From 931137c5a7f86422d8da6a8c39dee3447e69ef28 Mon Sep 17 00:00:00 2001 From: Gabor Bota Date: Wed, 3 Jul 2019 18:13:20 +0200 Subject: [PATCH 01/16] added integration tests for violation pairs. minor refactor --- .../hadoop/fs/s3a/s3guard/S3GuardFsck.java | 297 ++++++++ .../s3guard/S3GuardFsckViolationHandler.java | 298 ++++++++ .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 70 +- .../s3a/ITestS3GuardOutOfBandOperations.java | 11 +- .../apache/hadoop/fs/s3a/S3ATestUtils.java | 34 + .../s3a/s3guard/ITestS3GuardDraftingFsck.java | 177 +++++ .../fs/s3a/s3guard/ITestS3GuardFsck.java | 686 ++++++++++++++++++ .../s3a/s3guard/ITestS3GuardToolDynamoDB.java | 16 + .../fs/s3a/s3guard/MetadataStoreTestBase.java | 2 +- 9 files changed, 1583 insertions(+), 8 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDraftingFsck.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java new file mode 100644 index 0000000000000..b17b56f64dc10 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java @@ -0,0 +1,297 @@ +/* + * 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.Path; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.security.InvalidParameterException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Queue; +import java.util.Set; +import java.util.stream.Collectors; + +import static java.util.stream.Collectors.toList; + +/** + * Main class for the FSCK factored out from S3GuardTool + * The implementation uses fixed DynamoDBMetadataStore as the backing store + * for metadata. + * + * Functions: + *

+ */ +public class S3GuardFsck { + private static final Logger LOG = LoggerFactory.getLogger(S3GuardFsck.class); + public static final String ROOT_PATH_STRING = "/"; + + private S3AFileSystem rawFS; + private DynamoDBMetadataStore metadataStore; + + /** + * Creates an S3GuardFsck + * @param fs the filesystem to compare to + * @param ms metadatastore the metadatastore to compare with (dynamo) + * @param versionIdCheck if true than checks the versionId between S3 and + * the metadata store. + */ + S3GuardFsck(S3AFileSystem fs, MetadataStore ms, boolean versionIdCheck) + throws InvalidParameterException { + this.rawFS = fs; + + if (ms == null) { + throw new InvalidParameterException("S3AFileSystem should be guarded by" + + " a " + DynamoDBMetadataStore.class.getCanonicalName()); + } + this.metadataStore = (DynamoDBMetadataStore) ms; + + if(rawFS.hasMetadataStore()) { + throw new InvalidParameterException("Raw fs should not have a " + + "metadatastore."); + } + } + + /** + * Compares S3 to MS. + * Iterative breadth first walk on the S3 structure from a given root. + * Creates a list of pairs (metadata in S3 and in the MetadataStore) where + * the consistency or any rule is violated. + * Uses {@link S3GuardFsckViolationHandler} to handle violations. + * The violations are listed in Enums: {@link Violation} + * + * @param rootPath the root path to start the traversal + * @throws IOException + * @return + */ + public List compareS3toMs(final Path rootPath) throws IOException { + final S3AFileStatus root = + (S3AFileStatus) rawFS.getFileStatus(rootPath); + final List comparePairs = new ArrayList<>(); + final Queue queue = new ArrayDeque<>(); + boolean checkVersionId = false; + queue.add(root); + + while (!queue.isEmpty()) { + // pop front node from the queue + final S3AFileStatus currentDir = queue.poll(); + + // Get a listing of that dir from s3 and add just the files. + // (Each directory will be added as a root.) + // Files should be casted to S3AFileStatus instead of plain FileStatus + // to get the VersionID and Etag. + final Path currentDirPath = currentDir.getPath(); + // TODO Do we need to do a HEAD for each children in the path if we + // want the versionID? In the listing it is empty. + + final List children = + Arrays.asList(rawFS.listStatus(currentDirPath)).stream() + .filter(status -> !status.isDirectory()) + .map(S3AFileStatus.class::cast).collect(toList()); + + comparePairs.addAll( + compareS3DirToMs(currentDir, children).stream() + .filter(comparePair -> comparePair.containsViolation()) + .collect(Collectors.toList()) + ); + + // Add each dir to queue + children.stream().filter(pm -> pm.isDirectory()) + .forEach(pm -> queue.add(pm)); + } + + // Create a handler and handle each violated pairs + S3GuardFsckViolationHandler handler = + new S3GuardFsckViolationHandler(rawFS, metadataStore); + comparePairs.forEach(handler::handle); + + return comparePairs; + } + + protected List compareS3DirToMs(S3AFileStatus s3CurrentDir, + List children) throws IOException { + final Path path = s3CurrentDir.getPath(); + final PathMetadata pathMetadata = metadataStore.get(path); + List violationComparePairs = new ArrayList<>(); + + final ComparePair rootComparePair = + compareFileStatusToPathMetadata(s3CurrentDir, pathMetadata); + if (rootComparePair.containsViolation()) { + violationComparePairs.add(rootComparePair); + } + + children.forEach(s3ChildMeta -> { + try { + final PathMetadata msChildMeta = + metadataStore.get(s3ChildMeta.getPath()); + final ComparePair comparePair = + compareFileStatusToPathMetadata(s3ChildMeta, msChildMeta); + if (comparePair.containsViolation()) { + violationComparePairs.add(comparePair); + } + } catch (Exception e) { + e.printStackTrace(); + } + }); + + return violationComparePairs; + } + + protected ComparePair compareFileStatusToPathMetadata( + S3AFileStatus s3FileStatus, + PathMetadata msPathMetadata) throws IOException { + final Path path = s3FileStatus.getPath(); + System.out.println("== Path: " + path); + ComparePair comparePair = new ComparePair(s3FileStatus, msPathMetadata); + + if (!path.equals(path(ROOT_PATH_STRING))) { + final Path parentPath = path.getParent(); + final PathMetadata parentPm = metadataStore.get(parentPath); + + if (parentPm == null) { + comparePair.violations.add(Violation.NO_PARENT_ENTRY); + } else { + if (!parentPm.getFileStatus().isDirectory()) { + comparePair.violations.add(Violation.PARENT_IS_A_FILE); + } + if (parentPm.isDeleted()) { + comparePair.violations.add(Violation.PARENT_TOMBSTONED); + } + } + } else { + LOG.info("Entry is in the root, so there's no parent"); + } + + if(msPathMetadata == null) { + comparePair.violations.add(Violation.NO_METADATA_ENTRY); + return comparePair; + } + final S3AFileStatus msFileStatus = msPathMetadata.getFileStatus(); + if (s3FileStatus.isDirectory() && !msFileStatus.isDirectory()) { + comparePair.violations.add(Violation.DIR_IN_S3_FILE_IN_MS); + } + + /** + * Attribute check + */ + if(s3FileStatus.getLen() != msFileStatus.getLen()) { + comparePair.violations.add(Violation.LENGTH_MISMATCH); + } + + if(s3FileStatus.getModificationTime() != + msFileStatus.getModificationTime()) { + comparePair.violations.add(Violation.MOD_TIME_MISMATCH); + } + + if(s3FileStatus.getBlockSize() != msFileStatus.getBlockSize()) { + comparePair.violations.add(Violation.BLOCKSIZE_MISMATCH); + } + + if(s3FileStatus.getOwner() != msFileStatus.getOwner()) { + comparePair.violations.add(Violation.OWNER_MISMATCH); + } + + if(msPathMetadata.getFileStatus().getETag() == null) { + comparePair.violations.add(Violation.NO_ETAG); + } else if (s3FileStatus.getETag() != null && + !s3FileStatus.getETag().equals(msFileStatus.getETag())) { + comparePair.violations.add(Violation.ETAG_MISMATCH); + } + + if(msPathMetadata.getFileStatus().getVersionId() == null) { + comparePair.violations.add(Violation.NO_VERSIONID); + } else if(s3FileStatus.getVersionId() != msFileStatus.getVersionId()) { + comparePair.violations.add(Violation.VERSIONID_MISMATCH); + } + + return comparePair; + } + + private Path path(String s) { + return rawFS.makeQualified(new Path(s)); + } + + + public static class ComparePair { + private S3AFileStatus s3FileStatus; + private PathMetadata msPathMetadata; + + private Set violations = new HashSet<>(); + + ComparePair(S3AFileStatus status, PathMetadata pm) { + this.s3FileStatus = status; + this.msPathMetadata = pm; + } + + public S3AFileStatus getS3FileStatus() { + return s3FileStatus; + } + + public PathMetadata getMsPathMetadata() { + return msPathMetadata; + } + + public Set getViolations() { + return violations; + } + + public boolean containsViolation() { + return !violations.isEmpty(); + } + + @Override public String toString() { + return "ComparePair{" + "s3FileStatus=" + s3FileStatus + + ", msPathMetadata=" + msPathMetadata + ", violations=" + violations + + '}'; + } + } + + public enum Violation { + // No entry in metadatastore + NO_METADATA_ENTRY, + // A file or directory entry does not have a parent entry - excluding + // files and directories in the root. + NO_PARENT_ENTRY, + // An entry’s parent is a file + PARENT_IS_A_FILE, + // A file exists under a path for which there is a tombstone entry in the + // MS + PARENT_TOMBSTONED, + // A directory in S3 is a file entry in the MS + DIR_IN_S3_FILE_IN_MS, + // Attribute mismatch + LENGTH_MISMATCH, + MOD_TIME_MISMATCH, + BLOCKSIZE_MISMATCH, + OWNER_MISMATCH, + VERSIONID_MISMATCH, + ETAG_MISMATCH, + NO_ETAG, + NO_VERSIONID + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java new file mode 100644 index 0000000000000..75bb679c57ca1 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java @@ -0,0 +1,298 @@ +/* + * 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.commons.math3.ode.UnknownParameterException; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Violation handler for the S3Guard's fsck + * + */ +public class S3GuardFsckViolationHandler { + private static final Logger LOG = LoggerFactory.getLogger( + S3GuardFsckViolationHandler.class); + + private S3AFileSystem rawFs; + private DynamoDBMetadataStore metadataStore; + private static String NEWLINE = System.getProperty("line.separator"); + + public S3GuardFsckViolationHandler(S3AFileSystem fs, + DynamoDBMetadataStore ddbms) { + + this.metadataStore = ddbms; + this.rawFs = fs; + } + + public void handle(S3GuardFsck.ComparePair comparePair) { + if (!comparePair.containsViolation()) { + LOG.debug("There is no violation in the compare pair: " + toString()); + return; + } + + ViolationHandler handler; + + StringBuilder sB = new StringBuilder( + String.format("%sOn path: %s%n", NEWLINE, + comparePair.getS3FileStatus().getPath()) + ); + + for (S3GuardFsck.Violation violation : comparePair.getViolations()) { + switch (violation) { + case NO_METADATA_ENTRY: + handler = new NoMetadataEntryViolation(comparePair); + sB.append(handler.getError());; + break; + case NO_PARENT_ENTRY: + handler = new NoParentEntryViolation(comparePair); + sB.append(handler.getError());; + break; + case PARENT_IS_A_FILE: + handler = new ParentIsAFileViolation(comparePair); + sB.append(handler.getError());; + break; + case PARENT_TOMBSTONED: + handler = new ParentTombstonedViolation(comparePair); + sB.append(handler.getError());; + break; + case DIR_IN_S3_FILE_IN_MS: + handler = new DirInS3FileInMsViolation(comparePair); + sB.append(handler.getError());; + break; + case LENGTH_MISMATCH: + handler = new LengthMismatchViolation(comparePair); + sB.append(handler.getError());; + break; + case MOD_TIME_MISMATCH: + handler = new ModTimeMismatchViolation(comparePair); + sB.append(handler.getError());; + break; + case BLOCKSIZE_MISMATCH: + handler = new BlockSizeMismatchViolation(comparePair); + sB.append(handler.getError());; + break; + case OWNER_MISMATCH: + handler = new OwnerMismatchViolation(comparePair); + sB.append(handler.getError());; + break; + case VERSIONID_MISMATCH: + handler = new VersionIdMismatchViolation(comparePair); + sB.append(handler.getError());; + break; + case ETAG_MISMATCH: + handler = new EtagMismatchViolation(comparePair); + sB.append(handler.getError());; + break; + case NO_ETAG: + handler = new NoEtagViolation(comparePair); + sB.append(handler.getError());; + break; + case NO_VERSIONID: + handler = new NoVersionIdViolation(comparePair); + sB.append(handler.getError());; + break; + default: + LOG.error("UNKNOWN VIOLATION: {}", violation.toString()); + throw new UnknownParameterException("Unknown Violation: " + + violation.toString()); + } + sB.append(NEWLINE); + } + + LOG.error(sB.toString()); + } + + public static abstract class ViolationHandler { + final PathMetadata pathMetadata; + final S3AFileStatus s3FileStatus; + final S3AFileStatus msFileStatus; + + public ViolationHandler(S3GuardFsck.ComparePair comparePair) { + pathMetadata = comparePair.getMsPathMetadata(); + s3FileStatus = comparePair.getS3FileStatus(); + if (pathMetadata != null) { + msFileStatus = pathMetadata.getFileStatus(); + } else { + msFileStatus = null; + } + } + + abstract String getError(); + } + + public static class NoMetadataEntryViolation extends ViolationHandler { + + public NoMetadataEntryViolation(S3GuardFsck.ComparePair comparePair) { + super(comparePair); + } + + @Override + public String getError() { + return "No PathMetadata for this path in the MS."; + } + } + + public static class NoParentEntryViolation extends ViolationHandler { + + public NoParentEntryViolation(S3GuardFsck.ComparePair comparePair) { + super(comparePair); + } + + @Override + public String getError() { + return "Entry does not have a parent entry (not root)"; + } + } + + public static class ParentIsAFileViolation extends ViolationHandler { + + public ParentIsAFileViolation(S3GuardFsck.ComparePair comparePair) { + super(comparePair); + } + + @Override + public String getError() { + return "An entry’s parent is a file"; + } + } + + public static class ParentTombstonedViolation extends ViolationHandler { + + public ParentTombstonedViolation(S3GuardFsck.ComparePair comparePair) { + super(comparePair); + } + + @Override + public String getError() { + return "The entry's parent tombstoned"; + } + } + + public static class DirInS3FileInMsViolation extends ViolationHandler { + + public DirInS3FileInMsViolation(S3GuardFsck.ComparePair comparePair) { + super(comparePair); + } + + @Override + public String getError() { + return "A directory in S3 is a file entry in the MS"; + } + } + + public static class LengthMismatchViolation extends ViolationHandler { + + public LengthMismatchViolation(S3GuardFsck.ComparePair comparePair) { + super(comparePair); + } + + @Override public String getError() { + return String.format("getLen mismatch - s3: %s, ms: %s", + s3FileStatus.getLen(), msFileStatus.getLen()); + } + } + + public static class ModTimeMismatchViolation extends ViolationHandler { + + public ModTimeMismatchViolation(S3GuardFsck.ComparePair comparePair) { + super(comparePair); + } + + @Override public String getError() { + return String.format("getModificationTime mismatch - s3: %s, ms: %s", + s3FileStatus.getModificationTime(), + msFileStatus.getModificationTime()); + } + } + + public static class BlockSizeMismatchViolation extends ViolationHandler { + + public BlockSizeMismatchViolation(S3GuardFsck.ComparePair comparePair) { + super(comparePair); + } + + @Override public String getError() { + return String.format("getBlockSize mismatch - s3: %s, ms: %s", + s3FileStatus.getBlockSize(), msFileStatus.getBlockSize()); + } + } + + public static class OwnerMismatchViolation extends ViolationHandler { + + public OwnerMismatchViolation(S3GuardFsck.ComparePair comparePair) { + super(comparePair); + } + + @Override + public String getError() { + return String.format("getOwner mismatch - s3: %s, ms: %s", + s3FileStatus.getOwner(), msFileStatus.getOwner()); + } + } + + public static class VersionIdMismatchViolation extends ViolationHandler { + + public VersionIdMismatchViolation(S3GuardFsck.ComparePair comparePair) { + super(comparePair); + } + + @Override public String getError() { + return String.format("getVersionId mismatch - s3: %s, ms: %s", + s3FileStatus.getVersionId(), msFileStatus.getVersionId()); + } + } + + public static class EtagMismatchViolation extends ViolationHandler { + + public EtagMismatchViolation(S3GuardFsck.ComparePair comparePair) { + super(comparePair); + } + + @Override public String getError() { + return String.format("getETag mismatch - s3: %s, ms: %s", + s3FileStatus.getETag(), msFileStatus.getETag()); + } + } + + public static class NoEtagViolation extends ViolationHandler { + + public NoEtagViolation(S3GuardFsck.ComparePair comparePair) { + super(comparePair); + } + + @Override public String getError() { + return "No etag."; + } + } + + public static class NoVersionIdViolation extends ViolationHandler { + + public NoVersionIdViolation(S3GuardFsck.ComparePair comparePair) { + super(comparePair); + } + + @Override public String getError() { + return "No versionid."; + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java index 492c5660f341b..5812c02f45dae 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java @@ -94,7 +94,8 @@ public abstract class S3GuardTool extends Configured implements Tool { "\t" + Diff.NAME + " - " + Diff.PURPOSE + "\n" + "\t" + Prune.NAME + " - " + Prune.PURPOSE + "\n" + "\t" + SetCapacity.NAME + " - " + SetCapacity.PURPOSE + "\n" + - "\t" + SelectTool.NAME + " - " + SelectTool.PURPOSE + "\n"; + "\t" + SelectTool.NAME + " - " + SelectTool.PURPOSE + "\n" + + "\t" + Fsck.NAME + " - " + Fsck.PURPOSE + "\n"; private static final String DATA_IN_S3_IS_PRESERVED = "(all data in S3 is preserved)"; @@ -1485,6 +1486,70 @@ private void vprintln(PrintStream out, String format, Object... } } + /** + * Prune metadata that has not been modified recently. + */ + static class Fsck extends S3GuardTool { + public static final String CHECK_FLAG = "check"; + + public static final String NAME = "fsck"; + public static final String PURPOSE = "Compares S3 with MetadataStore, and " + + "returns a failure status if any rules or invariants are violated. " + + "Only works with DynamoDbMetadataStore."; + private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" + + "\t" + PURPOSE + "\n\n" + + "Common options:\n" + + " " + CHECK_FLAG + " Check the metadata store for errors, but do " + + "not fix any issues.\n"; + + Fsck(Configuration conf) { + super(conf, CHECK_FLAG); + addAgeOptions(); + } + + @Override + public String getName() { + return NAME; + } + + @Override + public String getUsage() { + return USAGE; + } + + public int run(String[] args, PrintStream out) throws + InterruptedException, IOException { + List paths = parseArgs(args); + try { + parseDynamoDBRegion(paths); + } catch (ExitUtil.ExitException e) { + errorln(USAGE); + throw e; + } + + if (paths.size() == 0) { + errorln(USAGE); + return ERROR; + } + + final MetadataStore ms = initMetadataStore(false); + String tableName = getConf().getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY); + initS3AFileSystem("s3a://" + tableName); + final S3AFileSystem fs = getFilesystem(); + + if (paths.get(0).equals(CHECK_FLAG)) { + // do the check + S3GuardFsck s3GuardFsck = new S3GuardFsck(fs, ms, false); + s3GuardFsck.compareS3toMs(fs.qualify(new Path("/"))); + } else { + errorln(USAGE); + return ERROR; + } + + return SUCCESS; + } + } + private static S3GuardTool command; /** @@ -1664,6 +1729,9 @@ public static int run(Configuration conf, String...args) throws // because this is the defacto S3 CLI. command = new SelectTool(conf); break; + case Fsck.NAME: + command = new Fsck(conf); + break; default: printHelp(); throw new ExitUtil.ExitException(E_USAGE, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java index f9e84e23567ae..2e5b78c8fc00d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java @@ -60,6 +60,11 @@ import static org.apache.hadoop.fs.s3a.Constants.RETRY_INTERVAL; import static org.apache.hadoop.fs.s3a.Constants.RETRY_LIMIT; import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.PROBE_INTERVAL_MILLIS; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.STABILIZATION_TIME; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.TIMESTAMP_SLEEP; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.awaitDeletedFileDisappearance; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.awaitFileStatus; import static org.apache.hadoop.fs.s3a.S3ATestUtils.checkListingContainsPath; import static org.apache.hadoop.fs.s3a.S3ATestUtils.checkListingDoesNotContainPath; import static org.apache.hadoop.fs.s3a.S3ATestUtils.metadataStorePersistsAuthoritativeBit; @@ -114,12 +119,6 @@ @RunWith(Parameterized.class) public class ITestS3GuardOutOfBandOperations extends AbstractS3ATestBase { - public static final int TIMESTAMP_SLEEP = 2000; - - public static final int STABILIZATION_TIME = 20_000; - - public static final int PROBE_INTERVAL_MILLIS = 2500; - private S3AFileSystem guardedFs; private S3AFileSystem rawFS; 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 e7f7f39fb866a..cac76edcd9e9e 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 @@ -54,6 +54,7 @@ import java.io.Closeable; import java.io.File; +import java.io.FileNotFoundException; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; @@ -71,6 +72,7 @@ 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.apache.hadoop.test.LambdaTestUtils.eventually; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC_COMMITTER_ENABLED; import static org.junit.Assert.*; @@ -91,6 +93,10 @@ public final class S3ATestUtils { public static final String UNSET_PROPERTY = "unset"; public static final int PURGE_DELAY_SECONDS = 60 * 60; + public static final int TIMESTAMP_SLEEP = 2000; + public static final int STABILIZATION_TIME = 20_000; + public static final int PROBE_INTERVAL_MILLIS = 500; + /** Add any deprecated keys. */ @SuppressWarnings("deprecation") private static void addDeprecatedKeys() { @@ -1307,4 +1313,32 @@ public static void checkListingContainsPath(S3AFileSystem fs, Path filePath) listStatusHasIt); } + /** + * Wait for a deleted file to no longer be visible. + * @param fs filesystem + * @param testFilePath path to query + * @throws Exception failure + */ + public static void awaitDeletedFileDisappearance(final S3AFileSystem fs, + final Path testFilePath) throws Exception { + eventually( + STABILIZATION_TIME, PROBE_INTERVAL_MILLIS, + () -> intercept(FileNotFoundException.class, + () -> fs.getFileStatus(testFilePath))); + } + + /** + * Wait for a file to be visible. + * @param fs filesystem + * @param testFilePath path to query + * @return the file status. + * @throws Exception failure + */ + public static S3AFileStatus awaitFileStatus(S3AFileSystem fs, + final Path testFilePath) + throws Exception { + return (S3AFileStatus) eventually( + STABILIZATION_TIME, PROBE_INTERVAL_MILLIS, + () -> fs.getFileStatus(testFilePath)); + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDraftingFsck.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDraftingFsck.java new file mode 100644 index 0000000000000..1bb727df01d3b --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDraftingFsck.java @@ -0,0 +1,177 @@ +/* + * 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.amazonaws.services.dynamodbv2.document.Item; +import com.amazonaws.services.dynamodbv2.document.ItemCollection; +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.spec.QuerySpec; +import com.amazonaws.services.dynamodbv2.document.utils.ValueMap; +import com.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder; +import com.amazonaws.services.dynamodbv2.xspec.ScanExpressionSpec; +import org.apache.hadoop.conf.Configuration; +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.s3a.AbstractS3ATestBase; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Queue; + +import static java.util.stream.Collectors.toList; +import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE; +import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.metadataStorePersistsAuthoritativeBit; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.itemToPathMetadata; +import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.pathToParentKeyAttribute; +import static org.junit.Assume.assumeTrue; + +public class ITestS3GuardDraftingFsck extends AbstractS3ATestBase { + + private S3AFileSystem guardedFs; + private S3AFileSystem rawFS; + + private MetadataStore metadataStore; + + @Before + public void setup() throws Exception { + super.setup(); + S3AFileSystem fs = getFileSystem(); + // These test will fail if no ms + assumeTrue("FS needs to have a metadatastore.", + fs.hasMetadataStore()); + assumeTrue("Metadatastore should persist authoritative bit", + metadataStorePersistsAuthoritativeBit(fs.getMetadataStore())); + + guardedFs = fs; + metadataStore = fs.getMetadataStore(); + + // create raw fs without s3guard + rawFS = createUnguardedFS(); + assertFalse("Raw FS still has S3Guard " + rawFS, + rawFS.hasMetadataStore()); + } + + /** + * Create a test filesystem which is always unguarded. + * This filesystem MUST be closed in test teardown. + * @return the new FS + */ + private S3AFileSystem createUnguardedFS() throws Exception { + S3AFileSystem testFS = getFileSystem(); + Configuration config = new Configuration(testFS.getConf()); + URI uri = testFS.getUri(); + + removeBaseAndBucketOverrides(uri.getHost(), config, + S3_METADATA_STORE_IMPL); + removeBaseAndBucketOverrides(uri.getHost(), config, + METADATASTORE_AUTHORITATIVE); + S3AFileSystem fs2 = new S3AFileSystem(); + fs2.initialize(uri, config); + return fs2; + } + + @Test + public void testTraverseS3() throws Exception { + final FileStatus root = rawFS.getFileStatus(path("/")); + + final Queue queue = new ArrayDeque<>(); + queue.add(root); + + System.out.println("Dir structure: "); + + while (!queue.isEmpty()) { + // pop front node from the queue and print it + final FileStatus currentDir = queue.poll(); + + System.out.println(currentDir.getPath()); + + // get a listing of that dir from s3 + final Path currentDirPath = currentDir.getPath(); + final List children = + Arrays.asList(rawFS.listStatus(currentDirPath)); + + // add each elem to queue + children.stream().filter(pm -> pm.isDirectory()) + .forEach(pm -> queue.add(pm)); + + } + } + + + @Test + public void testTraverseDynamo() throws Exception { + final PathMetadata root = metadataStore.get(path("/")); + + final Queue queue = new ArrayDeque<>(); + queue.add(root); + + System.out.println("Dir structure: "); + + while (!queue.isEmpty()) { + // pop front node from the queue and print it + final PathMetadata currentDir = queue.poll(); + + System.out.println(currentDir.getFileStatus().getPath()); + + // get a listing of that dir from dynamo + final Path currentDirPath = currentDir.getFileStatus().getPath(); + final Collection children = + metadataStore.listChildren(currentDirPath).getListing(); + + // add each elem to queue + children.stream().filter(pm -> pm.getFileStatus().isDirectory()) + .forEach(pm -> queue.add(pm)); + } + } + + @Test + public void testBuildGraphFromDynamo() throws Exception { + S3GuardTableAccess tableAccess = new S3GuardTableAccess( + (DynamoDBMetadataStore) metadataStore); + + ExpressionSpecBuilder builder = new ExpressionSpecBuilder(); + builder.withCondition( + ExpressionSpecBuilder.S("parent") + .beginsWith("/") + ); + final Iterable ddbPathMetadata = + tableAccess.scanMetadata(builder); + + ddbPathMetadata.iterator().forEachRemaining(pmd -> { + System.out.println(pmd.getFileStatus().getPath()); + }); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java new file mode 100644 index 0000000000000..3443d759b4abb --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java @@ -0,0 +1,686 @@ +/* + * 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.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.assertj.core.api.Assertions; +import org.junit.Before; +import org.junit.Test; + +import java.net.URI; +import java.util.ArrayDeque; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Queue; +import java.util.UUID; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; +import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE; +import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.awaitFileStatus; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.metadataStorePersistsAuthoritativeBit; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.junit.Assume.assumeNotNull; +import static org.junit.Assume.assumeTrue; + +public class ITestS3GuardFsck extends AbstractS3ATestBase { + + private S3AFileSystem guardedFs; + private S3AFileSystem rawFS; + + private MetadataStore metadataStore; + + @Before + public void setup() throws Exception { + super.setup(); + S3AFileSystem fs = getFileSystem(); + // These test will fail if no ms + assumeTrue("FS needs to have a metadatastore.", + fs.hasMetadataStore()); + assumeTrue("Metadatastore should persist authoritative bit", + metadataStorePersistsAuthoritativeBit(fs.getMetadataStore())); + + guardedFs = fs; + metadataStore = fs.getMetadataStore(); + + // create raw fs without s3guard + rawFS = createUnguardedFS(); + assertFalse("Raw FS still has S3Guard " + rawFS, + rawFS.hasMetadataStore()); + } + + /** + * Create a test filesystem which is always unguarded. + * This filesystem MUST be closed in test teardown. + * @return the new FS + */ + private S3AFileSystem createUnguardedFS() throws Exception { + S3AFileSystem testFS = getFileSystem(); + Configuration config = new Configuration(testFS.getConf()); + URI uri = testFS.getUri(); + + removeBaseAndBucketOverrides(uri.getHost(), config, + S3_METADATA_STORE_IMPL); + removeBaseAndBucketOverrides(uri.getHost(), config, + METADATASTORE_AUTHORITATIVE); + S3AFileSystem fs2 = new S3AFileSystem(); + fs2.initialize(uri, config); + return fs2; + } + + @Test + public void testIDetectNoMetadataEntry() throws Exception { + final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); + final Path file = new Path(cwd, "file"); + try { + touch(rawFS, file); + awaitFileStatus(rawFS, file); + + final S3GuardFsck s3GuardFsck = + new S3GuardFsck(rawFS, metadataStore, false); + + final List comparePairs = + s3GuardFsck.compareS3toMs(cwd); + + assertEquals("Number of pairs should be two.", 2, + comparePairs.size()); + final S3GuardFsck.ComparePair pair = comparePairs.get(0); + assumeTrue("The pair must contain a violation.", pair.containsViolation()); + assertEquals("The pair must contain only one violation", 1, + pair.getViolations().size()); + + final S3GuardFsck.Violation violation = + pair.getViolations().iterator().next(); + assertEquals("The violation should be that there is no violation entry.", + violation, S3GuardFsck.Violation.NO_METADATA_ENTRY); + } finally { + // delete the working directory with all of its contents + rawFS.delete(cwd, true); + metadataStore.forgetMetadata(file); + metadataStore.forgetMetadata(cwd); + } + } + + @Test + public void testIDetectNoParentEntry() throws Exception { + final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); + final Path file = new Path(cwd, "file"); + try { + // create a file with guarded fs + touch(guardedFs, file); + awaitFileStatus(guardedFs, file); + + // delete the parent from the MS + metadataStore.forgetMetadata(cwd); + + final S3GuardFsck s3GuardFsck = + new S3GuardFsck(rawFS, metadataStore, false); + + final List comparePairs = + s3GuardFsck.compareS3toMs(cwd); + + assertEquals("Number of pairs should be two. The cwd (parent) and the " + + "child.", 2, comparePairs.size()); + + // check the parent that it does not exist + final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(cwd)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", cwdPair); + assumeTrue("The cwdPair must contain a violation.", cwdPair.containsViolation()); + Assertions.assertThat(cwdPair.getViolations()) + .describedAs("Violations in the cwdPair") + .contains(S3GuardFsck.Violation.NO_METADATA_ENTRY); + + // check the child that there's no parent entry. + final S3GuardFsck.ComparePair childPair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(file)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", childPair); + assumeTrue("The childPair must contain a violation.", childPair.containsViolation()); + Assertions.assertThat(childPair.getViolations()) + .describedAs("Violations in the childPair") + .contains(S3GuardFsck.Violation.NO_PARENT_ENTRY); + } finally { + // delete the working directory with all of its contents + rawFS.delete(cwd, true); + metadataStore.forgetMetadata(file); + metadataStore.forgetMetadata(cwd); + } + } + + @Test + public void testIDetectParentIsAFile() throws Exception { + final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); + final Path file = new Path(cwd, "file"); + try { + // create a file with guarded fs + touch(guardedFs, file); + awaitFileStatus(guardedFs, file); + + // modify the cwd metadata and set that it's not a directory + final S3AFileStatus newParentFile = MetadataStoreTestBase + .basicFileStatus(cwd, 1, false, 1); + metadataStore.put(new PathMetadata(newParentFile)); + + final S3GuardFsck s3GuardFsck = + new S3GuardFsck(rawFS, metadataStore, false); + + final List comparePairs = + s3GuardFsck.compareS3toMs(cwd); + + assertEquals("Number of pairs should be two. The cwd (parent) and the " + + "child.", 2, comparePairs.size()); + + // check the parent that it does not exist + final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(cwd)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", cwdPair); + assumeTrue("The cwdPair must contain a violation.", cwdPair.containsViolation()); + Assertions.assertThat(cwdPair.getViolations()) + .describedAs("Violations in the cwdPair") + .contains(S3GuardFsck.Violation.DIR_IN_S3_FILE_IN_MS); + + // check the child that the parent is a file. + final S3GuardFsck.ComparePair childPair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(file)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", childPair); + assumeTrue("The childPair must contain a violation.", childPair.containsViolation()); + Assertions.assertThat(childPair.getViolations()) + .describedAs("Violations in the childPair") + .contains(S3GuardFsck.Violation.PARENT_IS_A_FILE); + } finally { + // delete the working directory with all of its contents + rawFS.delete(cwd, true); + metadataStore.forgetMetadata(file); + metadataStore.forgetMetadata(cwd); + } + } + + @Test + public void testIDetectParentTombstoned() throws Exception { + final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); + final Path file = new Path(cwd, "file"); + try { + // create a file with guarded fs + touch(guardedFs, file); + awaitFileStatus(guardedFs, file); + + // modify the parent metadata and set that it's not a directory + final PathMetadata cwdPmd = metadataStore.get(cwd); + cwdPmd.setIsDeleted(true); + metadataStore.put(cwdPmd); + + final S3GuardFsck s3GuardFsck = + new S3GuardFsck(rawFS, metadataStore, false); + + final List comparePairs = + s3GuardFsck.compareS3toMs(cwd); + + assertEquals("Number of pairs should be two. The cwd (parent) and the " + + "child.", 2, comparePairs.size()); + + // check the child that the parent is tombstoned + final S3GuardFsck.ComparePair childPair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(file)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", childPair); + assumeTrue("The childPair must contain a violation.", childPair.containsViolation()); + Assertions.assertThat(childPair.getViolations()) + .describedAs("Violations in the childPair") + .contains(S3GuardFsck.Violation.PARENT_TOMBSTONED); + } finally { + // delete the working directory with all of its contents + rawFS.delete(cwd, true); + metadataStore.forgetMetadata(file); + metadataStore.forgetMetadata(cwd); + } + } + + @Test + public void testIDetectDirInS3FileInMs() throws Exception { + final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); + try { + // create a file with guarded fs + mkdirs(cwd); + awaitFileStatus(guardedFs, cwd); + + // modify the cwd metadata and set that it's not a directory + final S3AFileStatus newParentFile = MetadataStoreTestBase + .basicFileStatus(cwd, 1, false, 1); + metadataStore.put(new PathMetadata(newParentFile)); + + final S3GuardFsck s3GuardFsck = + new S3GuardFsck(rawFS, metadataStore, false); + + final List comparePairs = + s3GuardFsck.compareS3toMs(cwd); + + assertEquals("Number of pair should be one. Only the cwd.", 1, + comparePairs.size()); + + // check the child that the dir in s3 is a file in the ms + final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(cwd)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", cwdPair); + assumeTrue("The cwdPair must contain a violation.", cwdPair.containsViolation()); + Assertions.assertThat(cwdPair.getViolations()) + .describedAs("Violations in the cwdPair") + .contains(S3GuardFsck.Violation.DIR_IN_S3_FILE_IN_MS); + } finally { + // delete the working directory with all of its contents + rawFS.delete(cwd, true); + metadataStore.forgetMetadata(cwd); + } + } + + @Test + public void testIDetectLengthMismatch() throws Exception { + final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); + final Path file = new Path(cwd, "file"); + try { + // create a file with guarded fs + touch(guardedFs, file); + awaitFileStatus(guardedFs, file); + + // modify the file metadata so the length will not match + final S3AFileStatus newFile = MetadataStoreTestBase + .basicFileStatus(file, 9999, false, 1); + metadataStore.put(new PathMetadata(newFile)); + + final S3GuardFsck s3GuardFsck = + new S3GuardFsck(rawFS, metadataStore, false); + + final List comparePairs = + s3GuardFsck.compareS3toMs(cwd); + + assertEquals("Number of pairs should be two. The cwd (parent) and the " + + "child.", 2, comparePairs.size()); + + // check the parent that it does not contain LENGTH_MISMATCH + final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(cwd)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", cwdPair); + assumeTrue("The cwdPair must contain a violation.", cwdPair.containsViolation()); + Assertions.assertThat(cwdPair.getViolations()) + .describedAs("Violations in the cwdPair") + .doesNotContain(S3GuardFsck.Violation.LENGTH_MISMATCH); + + // check the child that there's a LENGTH_MISMATCH + final S3GuardFsck.ComparePair childPair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(file)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", childPair); + assumeTrue("The childPair must contain a violation.", childPair.containsViolation()); + Assertions.assertThat(childPair.getViolations()) + .describedAs("Violations in the childPair") + .contains(S3GuardFsck.Violation.LENGTH_MISMATCH); + } finally { + // delete the working directory with all of its contents + rawFS.delete(cwd, true); + metadataStore.forgetMetadata(file); + metadataStore.forgetMetadata(cwd); + } + } + + @Test + public void testIDetectModTimeMismatch() throws Exception { + final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); + final Path file = new Path(cwd, "file"); + try { + // create a file with guarded fs + touch(guardedFs, file); + awaitFileStatus(guardedFs, file); + + // modify the file metadata so the length will not match + final S3AFileStatus newFileStatus = MetadataStoreTestBase + .basicFileStatus(file, 0, false, 1); + metadataStore.put(new PathMetadata(newFileStatus)); + + // modify the parent meta entry so the MOD_TIME will surely be up to date + final FileStatus oldCwdFileStatus = rawFS.getFileStatus(cwd); + final S3AFileStatus newCwdFileStatus = MetadataStoreTestBase + .basicFileStatus(file, 0, true, + oldCwdFileStatus.getModificationTime()); + metadataStore.put(new PathMetadata(newCwdFileStatus)); + + final S3GuardFsck s3GuardFsck = + new S3GuardFsck(rawFS, metadataStore, false); + + final List comparePairs = + s3GuardFsck.compareS3toMs(cwd); + + assertEquals("Number of pairs should be two. The cwd (parent) and the " + + "child.", 2, comparePairs.size()); + + // check the parent that it does not contain MOD_TIME_MISMATCH + final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(cwd)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", cwdPair); + assumeTrue("The cwdPair must contain a violation.", cwdPair.containsViolation()); + Assertions.assertThat(cwdPair.getViolations()) + .describedAs("Violations in the cwdPair") + .doesNotContain(S3GuardFsck.Violation.MOD_TIME_MISMATCH); + + // check the child that there's a MOD_TIME_MISMATCH + final S3GuardFsck.ComparePair childPair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(file)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", childPair); + assumeTrue("The childPair must contain a violation.", childPair.containsViolation()); + Assertions.assertThat(childPair.getViolations()) + .describedAs("Violations in the childPair") + .contains(S3GuardFsck.Violation.MOD_TIME_MISMATCH); + } finally { + // delete the working directory with all of its contents + rawFS.delete(cwd, true); + metadataStore.forgetMetadata(file); + metadataStore.forgetMetadata(cwd); + } + } + + @Test + public void testIBlockSizeMismatch() throws Exception { + final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); + final Path file = new Path(cwd, "file"); + try { + // create a file with guarded fs + touch(guardedFs, file); + awaitFileStatus(guardedFs, file); + + // modify the file metadata so the length will not match + final S3AFileStatus newFileStatus = MetadataStoreTestBase + .basicFileStatus(1, false, 999, 1, file); + metadataStore.put(new PathMetadata(newFileStatus)); + + final S3GuardFsck s3GuardFsck = + new S3GuardFsck(rawFS, metadataStore, false); + + final List comparePairs = + s3GuardFsck.compareS3toMs(cwd); + + assertEquals("Number of pairs should be two. The cwd (parent) and the " + + "child.", 2, comparePairs.size()); + + // check the parent that it does not contain BLOCKSIZE_MISMATCH + final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(cwd)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", cwdPair); + assumeTrue("The childPair must contain a violation.", cwdPair.containsViolation()); + Assertions.assertThat(cwdPair.getViolations()) + .describedAs("Violations in the cwdPair") + .doesNotContain(S3GuardFsck.Violation.BLOCKSIZE_MISMATCH); + + // check the child that there's a BLOCKSIZE_MISMATCH + final S3GuardFsck.ComparePair childPair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(file)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", childPair); + assumeTrue("The childPair must contain a violation.", childPair.containsViolation()); + Assertions.assertThat(childPair.getViolations()) + .describedAs("Violations in the childPair") + .contains(S3GuardFsck.Violation.BLOCKSIZE_MISMATCH); + } finally { + // delete the working directory with all of its contents + rawFS.delete(cwd, true); + metadataStore.forgetMetadata(file); + metadataStore.forgetMetadata(cwd); + } + } + + @Test + public void testIOwnerMismatch() throws Exception { + skip("We don't store the owner in dynamo, so there's no test for this."); + } + + @Test + public void testIVersionIdMismatch() throws Exception { + final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); + final Path file = new Path(cwd, "file"); + try { + // create a file with guarded fs + touch(guardedFs, file); + awaitFileStatus(guardedFs, file); + + // modify the file metadata so the versionId will not match + final S3AFileStatus newFileStatus = new S3AFileStatus(1, 1, file, 1, "" + , "etag", "versionId"); + metadataStore.put(new PathMetadata(newFileStatus)); + + final S3GuardFsck s3GuardFsck = + new S3GuardFsck(rawFS, metadataStore, false); + + final List comparePairs = + s3GuardFsck.compareS3toMs(cwd); + + assertEquals("Number of pairs should be two. The cwd (parent) and the " + + "child.", 2, comparePairs.size()); + + // check the parent that it does not contain BLOCKSIZE_MISMATCH + final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(cwd)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", cwdPair); + assumeTrue("The childPair must contain a violation.", cwdPair.containsViolation()); + Assertions.assertThat(cwdPair.getViolations()) + .describedAs("Violations in the cwdPair") + .doesNotContain(S3GuardFsck.Violation.VERSIONID_MISMATCH); + + // check the child that there's a BLOCKSIZE_MISMATCH + final S3GuardFsck.ComparePair childPair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(file)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", childPair); + assumeTrue("The childPair must contain a violation.", childPair.containsViolation()); + Assertions.assertThat(childPair.getViolations()) + .describedAs("Violations in the childPair") + .contains(S3GuardFsck.Violation.VERSIONID_MISMATCH); + } finally { + // delete the working directory with all of its contents + rawFS.delete(cwd, true); + metadataStore.forgetMetadata(file); + metadataStore.forgetMetadata(cwd); + } + } + + @Test + public void testIEtagMismatch() throws Exception { + final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); + final Path file = new Path(cwd, "file"); + try { + // create a file with guarded fs + touch(guardedFs, file); + awaitFileStatus(guardedFs, file); + + // modify the file metadata so the etag will not match + final S3AFileStatus newFileStatus = new S3AFileStatus(1, 1, file, 1, "" + , "etag", "versionId"); + metadataStore.put(new PathMetadata(newFileStatus)); + + final S3GuardFsck s3GuardFsck = + new S3GuardFsck(rawFS, metadataStore, false); + + final List comparePairs = + s3GuardFsck.compareS3toMs(cwd); + + assertEquals("Number of pairs should be two. The cwd (parent) and the " + + "child.", 2, comparePairs.size()); + + // check the parent that it does not contain BLOCKSIZE_MISMATCH + final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(cwd)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", cwdPair); + assumeTrue("The childPair must contain a violation.", cwdPair.containsViolation()); + Assertions.assertThat(cwdPair.getViolations()) + .describedAs("Violations in the cwdPair") + .doesNotContain(S3GuardFsck.Violation.ETAG_MISMATCH); + + // check the child that there's a BLOCKSIZE_MISMATCH + final S3GuardFsck.ComparePair childPair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(file)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", childPair); + assumeTrue("The childPair must contain a violation.", childPair.containsViolation()); + Assertions.assertThat(childPair.getViolations()) + .describedAs("Violations in the childPair") + .contains(S3GuardFsck.Violation.ETAG_MISMATCH); + } finally { + // delete the working directory with all of its contents + rawFS.delete(cwd, true); + metadataStore.forgetMetadata(file); + metadataStore.forgetMetadata(cwd); + } + } + + @Test + public void testINoEtag() throws Exception { + final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); + final Path file1 = new Path(cwd, "file1"); + final Path file2 = new Path(cwd, "file2"); + try { + // create a file1 with guarded fs + touch(guardedFs, file1); + touch(guardedFs, file2); + awaitFileStatus(guardedFs, file1); + awaitFileStatus(guardedFs, file2); + + // modify the file1 metadata so there's no etag + final S3AFileStatus newFile1Status = + new S3AFileStatus(1, 1, file1, 1, "", null, "versionId"); + final S3AFileStatus newFile2Status = + new S3AFileStatus(1, 1, file2, 1, "", "etag", "versionId"); + + metadataStore.put(new PathMetadata(newFile1Status)); + metadataStore.put(new PathMetadata(newFile2Status)); + + final S3GuardFsck s3GuardFsck = + new S3GuardFsck(rawFS, metadataStore, false); + + final List comparePairs = + s3GuardFsck.compareS3toMs(cwd); + + assertEquals("Number of pairs should be 3. The cwd (parent) and the " + + "2 children.", 3, comparePairs.size()); + + // check file 1 that there's NO_ETAG + final S3GuardFsck.ComparePair file1Pair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(file1)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", file1Pair); + assumeTrue("The file1Pair must contain a violation.", + file1Pair.containsViolation()); + Assertions.assertThat(file1Pair.getViolations()) + .describedAs("Violations in the file1Pair") + .contains(S3GuardFsck.Violation.NO_ETAG); + + // check the child that there's no NO_ETAG violation + final S3GuardFsck.ComparePair file2Pair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(file2)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", file2Pair); + assumeTrue("The file2Pair must contain a violation.", + file2Pair.containsViolation()); + Assertions.assertThat(file2Pair.getViolations()) + .describedAs("Violations in the file2Pair") + .doesNotContain(S3GuardFsck.Violation.NO_ETAG); + } finally { + // delete the working directory with all of its contents + rawFS.delete(cwd, true); + metadataStore.forgetMetadata(file1); + metadataStore.forgetMetadata(file2); + metadataStore.forgetMetadata(cwd); + } + } + + @Test + public void testINoVersionId() throws Exception { + final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); + final Path file1 = new Path(cwd, "file1"); + final Path file2 = new Path(cwd, "file2"); + try { + // create a file1 with guarded fs + touch(guardedFs, file1); + touch(guardedFs, file2); + awaitFileStatus(guardedFs, file1); + awaitFileStatus(guardedFs, file2); + + // modify the file1 metadata so there's no versionId + final S3AFileStatus newFile1Status = + new S3AFileStatus(1, 1, file1, 1, "", "etag", null); + // force add versionID to the second file + final S3AFileStatus newFile2Status = + new S3AFileStatus(1, 1, file2, 1, "", "etag", "notnull"); + + metadataStore.put(new PathMetadata(newFile1Status)); + metadataStore.put(new PathMetadata(newFile2Status)); + + final S3GuardFsck s3GuardFsck = + new S3GuardFsck(rawFS, metadataStore, false); + + final List comparePairs = + s3GuardFsck.compareS3toMs(cwd); + + assertEquals("Number of pairs should be 3. The cwd (parent) and the " + + "2 children.", 3, comparePairs.size()); + + // check file 1 that there's NO_ETAG + final S3GuardFsck.ComparePair file1Pair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(file1)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", file1Pair); + assumeTrue("The file1Pair must contain a violation.", + file1Pair.containsViolation()); + Assertions.assertThat(file1Pair.getViolations()) + .describedAs("Violations in the file1Pair") + .contains(S3GuardFsck.Violation.NO_VERSIONID); + + // check file 2 that there's no NO_ETAG violation + final S3GuardFsck.ComparePair file2Pair = comparePairs.stream() + .filter(p -> p.getS3FileStatus().getPath().equals(file2)) + .findFirst().get(); + assumeNotNull("The pair should not be null.", file2Pair); + assumeTrue("The file2Pair must contain a violation.", + file2Pair.containsViolation()); + Assertions.assertThat(file2Pair.getViolations()) + .describedAs("Violations in the file2Pair") + .doesNotContain(S3GuardFsck.Violation.NO_VERSIONID); + } finally { + // delete the working directory with all of its contents + rawFS.delete(cwd, true); + metadataStore.forgetMetadata(file1); + metadataStore.forgetMetadata(file2); + metadataStore.forgetMetadata(cwd); + } + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java index 9e0a08b93b071..6db44f344da70 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java @@ -289,4 +289,20 @@ public void testDestroyUnknownTable() throws Throwable { "-meta", "dynamodb://" + getTestTableName(DYNAMODB_TABLE)); } + @Test + public void testCLIFsckWithoutParam() throws Exception { + run(S3GuardTool.Fsck.NAME, "check"); + } + + @Test + public void testCLIFsckWithPathParam() throws Exception { + fail("TODO implement"); + run(S3GuardTool.Fsck.NAME, "check"); + } + + @Test + public void testCLIFsckWithVersionIdParam() throws Exception { + fail("TODO implement"); + run(S3GuardTool.Fsck.NAME, "check"); + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java index 198a2de51e986..e65ff8a9eafe2 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStoreTestBase.java @@ -1199,7 +1199,7 @@ protected S3AFileStatus basicFileStatus(Path path, int size, boolean isDir) return basicFileStatus(path, size, isDir, modTime); } - protected S3AFileStatus basicFileStatus(int size, boolean isDir, + public static S3AFileStatus basicFileStatus(int size, boolean isDir, long blockSize, long modificationTime, Path path) { if (isDir) { return new S3AFileStatus(Tristate.UNKNOWN, path, null); From 3cfd2c9a573b5d83754b62632e69941c8a0fd5df Mon Sep 17 00:00:00 2001 From: Gabor Bota Date: Wed, 7 Aug 2019 13:05:07 +0200 Subject: [PATCH 02/16] added AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH with test, removed versionID mismatch, added severity (not used enywhere yet); Change-Id: I4e327bb172663b5da247789d19053e6d54e88a1e --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 3 +- .../hadoop/fs/s3a/s3guard/S3GuardFsck.java | 159 ++++--- .../s3guard/S3GuardFsckViolationHandler.java | 114 +++-- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 4 +- .../fs/s3a/s3guard/ITestS3GuardFsck.java | 388 +++++++++--------- 5 files changed, 367 insertions(+), 301 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 6edbed7ef4bb9..dd26a0b5f6272 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 @@ -1491,8 +1491,7 @@ public boolean hasMetadataStore() { * @return true if there is a metadata store and the authoritative flag * is set for this filesystem. */ - @VisibleForTesting - boolean hasAuthoritativeMetadataStore() { + @VisibleForTesting public boolean hasAuthoritativeMetadataStore() { return hasMetadataStore() && allowAuthoritativeMetadataStore; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java index b17b56f64dc10..be7d9ebabea7d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.s3guard; +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; @@ -33,9 +34,9 @@ import java.util.List; import java.util.Queue; import java.util.Set; -import java.util.stream.Collectors; import static java.util.stream.Collectors.toList; +import static java.util.stream.Collectors.toSet; /** * Main class for the FSCK factored out from S3GuardTool @@ -58,10 +59,8 @@ public class S3GuardFsck { * Creates an S3GuardFsck * @param fs the filesystem to compare to * @param ms metadatastore the metadatastore to compare with (dynamo) - * @param versionIdCheck if true than checks the versionId between S3 and - * the metadata store. */ - S3GuardFsck(S3AFileSystem fs, MetadataStore ms, boolean versionIdCheck) + S3GuardFsck(S3AFileSystem fs, MetadataStore ms) throws InvalidParameterException { this.rawFS = fs; @@ -71,7 +70,7 @@ public class S3GuardFsck { } this.metadataStore = (DynamoDBMetadataStore) ms; - if(rawFS.hasMetadataStore()) { + if (rawFS.hasMetadataStore()) { throw new InvalidParameterException("Raw fs should not have a " + "metadatastore."); } @@ -89,12 +88,11 @@ public class S3GuardFsck { * @throws IOException * @return */ - public List compareS3toMs(final Path rootPath) throws IOException { + public List compareS3RootToMs(final Path rootPath) throws IOException { final S3AFileStatus root = (S3AFileStatus) rawFS.getFileStatus(rootPath); final List comparePairs = new ArrayList<>(); final Queue queue = new ArrayDeque<>(); - boolean checkVersionId = false; queue.add(root); while (!queue.isEmpty()) { @@ -106,19 +104,30 @@ public List compareS3toMs(final Path rootPath) throws IOException { // Files should be casted to S3AFileStatus instead of plain FileStatus // to get the VersionID and Etag. final Path currentDirPath = currentDir.getPath(); - // TODO Do we need to do a HEAD for each children in the path if we - // want the versionID? In the listing it is empty. + final FileStatus[] s3DirListing = rawFS.listStatus(currentDirPath); final List children = - Arrays.asList(rawFS.listStatus(currentDirPath)).stream() + Arrays.asList(s3DirListing).stream() .filter(status -> !status.isDirectory()) .map(S3AFileStatus.class::cast).collect(toList()); - comparePairs.addAll( + // Compare the directory contents if the listing is authoritative + final DirListingMetadata msDirListing = + metadataStore.listChildren(currentDirPath); + if (msDirListing.isAuthoritative()) { + final ComparePair cP = + compareAuthDirListing(s3DirListing, msDirListing); + if (cP.containsViolation()) { + comparePairs.add(cP); + } + } + + // Compare directory and contents, but not the listing + final List compareResult = compareS3DirToMs(currentDir, children).stream() .filter(comparePair -> comparePair.containsViolation()) - .collect(Collectors.toList()) - ); + .collect(toList()); + comparePairs.addAll(compareResult); // Add each dir to queue children.stream().filter(pm -> pm.isDirectory()) @@ -133,6 +142,29 @@ public List compareS3toMs(final Path rootPath) throws IOException { return comparePairs; } + private ComparePair compareAuthDirListing(FileStatus[] s3DirListing, + DirListingMetadata msDirListing) { + ComparePair cP = new ComparePair(s3DirListing, msDirListing); + + if (!msDirListing.isAuthoritative()) { + return cP; + } + + if (s3DirListing.length != msDirListing.numEntries()) { + cP.violations.add(Violation.AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH); + } else { + final Set msPaths = msDirListing.getListing().stream() + .map(pm -> pm.getFileStatus().getPath()).collect(toSet()); + final Set s3Paths = Arrays.stream(s3DirListing) + .map(pm -> pm.getPath()).collect(toSet()); + if (!s3Paths.equals(msPaths)) { + cP.violations.add(Violation.AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH); + } + } + + return cP; + } + protected List compareS3DirToMs(S3AFileStatus s3CurrentDir, List children) throws IOException { final Path path = s3CurrentDir.getPath(); @@ -187,7 +219,7 @@ protected ComparePair compareFileStatusToPathMetadata( LOG.info("Entry is in the root, so there's no parent"); } - if(msPathMetadata == null) { + if (msPathMetadata == null) { comparePair.violations.add(Violation.NO_METADATA_ENTRY); return comparePair; } @@ -195,40 +227,29 @@ protected ComparePair compareFileStatusToPathMetadata( if (s3FileStatus.isDirectory() && !msFileStatus.isDirectory()) { comparePair.violations.add(Violation.DIR_IN_S3_FILE_IN_MS); } + if (!s3FileStatus.isDirectory() && msFileStatus.isDirectory()) { + comparePair.violations.add(Violation.FILE_IN_S3_DIR_IN_MS); + } /** * Attribute check */ - if(s3FileStatus.getLen() != msFileStatus.getLen()) { + if (s3FileStatus.getLen() != msFileStatus.getLen()) { comparePair.violations.add(Violation.LENGTH_MISMATCH); } - if(s3FileStatus.getModificationTime() != + if (s3FileStatus.getModificationTime() != msFileStatus.getModificationTime()) { comparePair.violations.add(Violation.MOD_TIME_MISMATCH); } - if(s3FileStatus.getBlockSize() != msFileStatus.getBlockSize()) { - comparePair.violations.add(Violation.BLOCKSIZE_MISMATCH); - } - - if(s3FileStatus.getOwner() != msFileStatus.getOwner()) { - comparePair.violations.add(Violation.OWNER_MISMATCH); - } - - if(msPathMetadata.getFileStatus().getETag() == null) { + if (msPathMetadata.getFileStatus().getETag() == null) { comparePair.violations.add(Violation.NO_ETAG); } else if (s3FileStatus.getETag() != null && !s3FileStatus.getETag().equals(msFileStatus.getETag())) { comparePair.violations.add(Violation.ETAG_MISMATCH); } - if(msPathMetadata.getFileStatus().getVersionId() == null) { - comparePair.violations.add(Violation.NO_VERSIONID); - } else if(s3FileStatus.getVersionId() != msFileStatus.getVersionId()) { - comparePair.violations.add(Violation.VERSIONID_MISMATCH); - } - return comparePair; } @@ -238,14 +259,30 @@ private Path path(String s) { public static class ComparePair { - private S3AFileStatus s3FileStatus; - private PathMetadata msPathMetadata; + private final S3AFileStatus s3FileStatus; + private final PathMetadata msPathMetadata; + + private final FileStatus[] s3DirListing; + private final DirListingMetadata msDirListing; + + private final Path path; private Set violations = new HashSet<>(); ComparePair(S3AFileStatus status, PathMetadata pm) { this.s3FileStatus = status; this.msPathMetadata = pm; + this.s3DirListing = null; + this.msDirListing = null; + this.path = status.getPath(); + } + + ComparePair(FileStatus[] s3DirListing, DirListingMetadata msDirListing) { + this.s3DirListing = s3DirListing; + this.msDirListing = msDirListing; + this.s3FileStatus = null; + this.msPathMetadata = null; + this.path = msDirListing.getPath(); } public S3AFileStatus getS3FileStatus() { @@ -269,29 +306,55 @@ public boolean containsViolation() { + ", msPathMetadata=" + msPathMetadata + ", violations=" + violations + '}'; } + + public DirListingMetadata getMsDirListing() { + return msDirListing; + } + + public FileStatus[] getS3DirListing() { + return s3DirListing; + } + + public Path getPath() { + return path; + } } + /** + * Violation with severity: + * Defines the severity of the violation between 0-2 + * where 0 is the most severe and 2 is the least severe. + */ public enum Violation { // No entry in metadatastore - NO_METADATA_ENTRY, + NO_METADATA_ENTRY(1), // A file or directory entry does not have a parent entry - excluding // files and directories in the root. - NO_PARENT_ENTRY, + NO_PARENT_ENTRY(0), // An entry’s parent is a file - PARENT_IS_A_FILE, - // A file exists under a path for which there is a tombstone entry in the - // MS - PARENT_TOMBSTONED, + PARENT_IS_A_FILE(0), + // A file exists under a path for which there is + // a tombstone entry in the MS + PARENT_TOMBSTONED(0), // A directory in S3 is a file entry in the MS - DIR_IN_S3_FILE_IN_MS, + DIR_IN_S3_FILE_IN_MS(0), + // A file in S3 is a directory in the MS + FILE_IN_S3_DIR_IN_MS(0), + AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH(1), // Attribute mismatch - LENGTH_MISMATCH, - MOD_TIME_MISMATCH, - BLOCKSIZE_MISMATCH, - OWNER_MISMATCH, - VERSIONID_MISMATCH, - ETAG_MISMATCH, - NO_ETAG, - NO_VERSIONID + LENGTH_MISMATCH(0), + MOD_TIME_MISMATCH(2), + // If there's a versionID the mismatch is severe + VERSIONID_MISMATCH(0), + // If there's an etag the mismatch is severe + ETAG_MISMATCH(0), + // Don't worry too much if we don't have an etag + NO_ETAG(2); + + int severity; + + Violation(int s) { + this.severity = s; + } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java index 75bb679c57ca1..65e715ff7b528 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java @@ -18,12 +18,16 @@ package org.apache.hadoop.fs.s3a.s3guard; +import com.google.common.collect.Lists; import org.apache.commons.math3.ode.UnknownParameterException; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Arrays; + /** * Violation handler for the S3Guard's fsck * @@ -52,63 +56,58 @@ public void handle(S3GuardFsck.ComparePair comparePair) { ViolationHandler handler; StringBuilder sB = new StringBuilder( - String.format("%sOn path: %s%n", NEWLINE, - comparePair.getS3FileStatus().getPath()) + String.format("%sOn path: %s%n", NEWLINE, comparePair.getPath()) ); for (S3GuardFsck.Violation violation : comparePair.getViolations()) { switch (violation) { case NO_METADATA_ENTRY: handler = new NoMetadataEntryViolation(comparePair); - sB.append(handler.getError());; + sB.append(handler.getError()); break; case NO_PARENT_ENTRY: handler = new NoParentEntryViolation(comparePair); - sB.append(handler.getError());; + sB.append(handler.getError()); break; case PARENT_IS_A_FILE: handler = new ParentIsAFileViolation(comparePair); - sB.append(handler.getError());; + sB.append(handler.getError()); break; case PARENT_TOMBSTONED: handler = new ParentTombstonedViolation(comparePair); - sB.append(handler.getError());; + sB.append(handler.getError()); break; case DIR_IN_S3_FILE_IN_MS: handler = new DirInS3FileInMsViolation(comparePair); - sB.append(handler.getError());; + sB.append(handler.getError()); + break; + case FILE_IN_S3_DIR_IN_MS: + handler = new FileInS3DirInMsViolation(comparePair); + sB.append(handler.getError()); + break; + case AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH: + handler = new AuthDirContentMismatchViolation(comparePair); + sB.append(handler.getError()); break; case LENGTH_MISMATCH: handler = new LengthMismatchViolation(comparePair); - sB.append(handler.getError());; + sB.append(handler.getError()); break; case MOD_TIME_MISMATCH: handler = new ModTimeMismatchViolation(comparePair); - sB.append(handler.getError());; - break; - case BLOCKSIZE_MISMATCH: - handler = new BlockSizeMismatchViolation(comparePair); - sB.append(handler.getError());; - break; - case OWNER_MISMATCH: - handler = new OwnerMismatchViolation(comparePair); - sB.append(handler.getError());; + sB.append(handler.getError()); break; case VERSIONID_MISMATCH: handler = new VersionIdMismatchViolation(comparePair); - sB.append(handler.getError());; + sB.append(handler.getError()); break; case ETAG_MISMATCH: handler = new EtagMismatchViolation(comparePair); - sB.append(handler.getError());; + sB.append(handler.getError()); break; case NO_ETAG: handler = new NoEtagViolation(comparePair); - sB.append(handler.getError());; - break; - case NO_VERSIONID: - handler = new NoVersionIdViolation(comparePair); - sB.append(handler.getError());; + sB.append(handler.getError()); break; default: LOG.error("UNKNOWN VIOLATION: {}", violation.toString()); @@ -125,6 +124,8 @@ public static abstract class ViolationHandler { final PathMetadata pathMetadata; final S3AFileStatus s3FileStatus; final S3AFileStatus msFileStatus; + final FileStatus[] s3DirListing; + final DirListingMetadata msDirListing; public ViolationHandler(S3GuardFsck.ComparePair comparePair) { pathMetadata = comparePair.getMsPathMetadata(); @@ -134,6 +135,8 @@ public ViolationHandler(S3GuardFsck.ComparePair comparePair) { } else { msFileStatus = null; } + s3DirListing = comparePair.getS3DirListing(); + msDirListing = comparePair.getMsDirListing(); } abstract String getError(); @@ -199,53 +202,57 @@ public String getError() { } } - public static class LengthMismatchViolation extends ViolationHandler { + public static class FileInS3DirInMsViolation extends ViolationHandler { - public LengthMismatchViolation(S3GuardFsck.ComparePair comparePair) { + public FileInS3DirInMsViolation(S3GuardFsck.ComparePair comparePair) { super(comparePair); } - @Override public String getError() { - return String.format("getLen mismatch - s3: %s, ms: %s", - s3FileStatus.getLen(), msFileStatus.getLen()); + @Override + public String getError() { + return "A file in S3 is a directory entry in the MS"; } } - public static class ModTimeMismatchViolation extends ViolationHandler { + public static class AuthDirContentMismatchViolation extends ViolationHandler { - public ModTimeMismatchViolation(S3GuardFsck.ComparePair comparePair) { + public AuthDirContentMismatchViolation(S3GuardFsck.ComparePair comparePair) { super(comparePair); } - @Override public String getError() { - return String.format("getModificationTime mismatch - s3: %s, ms: %s", - s3FileStatus.getModificationTime(), - msFileStatus.getModificationTime()); + @Override + public String getError() { + final String str = String.format( + "The content of an authoritative directory listing does " + + "not match the content of the S3 listing. S3: %s, MS: %s", + Arrays.asList(s3DirListing), msDirListing.getListing()); + return str; } } - public static class BlockSizeMismatchViolation extends ViolationHandler { + public static class LengthMismatchViolation extends ViolationHandler { - public BlockSizeMismatchViolation(S3GuardFsck.ComparePair comparePair) { + public LengthMismatchViolation(S3GuardFsck.ComparePair comparePair) { super(comparePair); } @Override public String getError() { - return String.format("getBlockSize mismatch - s3: %s, ms: %s", - s3FileStatus.getBlockSize(), msFileStatus.getBlockSize()); + return String.format("getLen mismatch - s3: %s, ms: %s", + s3FileStatus.getLen(), msFileStatus.getLen()); } } - public static class OwnerMismatchViolation extends ViolationHandler { + public static class ModTimeMismatchViolation extends ViolationHandler { - public OwnerMismatchViolation(S3GuardFsck.ComparePair comparePair) { + public ModTimeMismatchViolation(S3GuardFsck.ComparePair comparePair) { super(comparePair); } @Override public String getError() { - return String.format("getOwner mismatch - s3: %s, ms: %s", - s3FileStatus.getOwner(), msFileStatus.getOwner()); + return String.format("getModificationTime mismatch - s3: %s, ms: %s", + s3FileStatus.getModificationTime(), + msFileStatus.getModificationTime()); } } @@ -255,7 +262,8 @@ public VersionIdMismatchViolation(S3GuardFsck.ComparePair comparePair) { super(comparePair); } - @Override public String getError() { + @Override + public String getError() { return String.format("getVersionId mismatch - s3: %s, ms: %s", s3FileStatus.getVersionId(), msFileStatus.getVersionId()); } @@ -267,7 +275,8 @@ public EtagMismatchViolation(S3GuardFsck.ComparePair comparePair) { super(comparePair); } - @Override public String getError() { + @Override + public String getError() { return String.format("getETag mismatch - s3: %s, ms: %s", s3FileStatus.getETag(), msFileStatus.getETag()); } @@ -279,20 +288,9 @@ public NoEtagViolation(S3GuardFsck.ComparePair comparePair) { super(comparePair); } - @Override public String getError() { + @Override + public String getError() { return "No etag."; } } - - public static class NoVersionIdViolation extends ViolationHandler { - - public NoVersionIdViolation(S3GuardFsck.ComparePair comparePair) { - super(comparePair); - } - - @Override public String getError() { - return "No versionid."; - } - } - } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java index 5812c02f45dae..9a31102d33a13 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java @@ -1539,8 +1539,8 @@ public int run(String[] args, PrintStream out) throws if (paths.get(0).equals(CHECK_FLAG)) { // do the check - S3GuardFsck s3GuardFsck = new S3GuardFsck(fs, ms, false); - s3GuardFsck.compareS3toMs(fs.qualify(new Path("/"))); + S3GuardFsck s3GuardFsck = new S3GuardFsck(fs, ms); + s3GuardFsck.compareS3RootToMs(fs.qualify(new Path("/"))); } else { errorln(USAGE); return ERROR; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java index 3443d759b4abb..e2f0ed0b0c5ce 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java @@ -18,7 +18,13 @@ package org.apache.hadoop.fs.s3a.s3guard; -import com.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder; + +import java.net.URI; +import java.util.List; +import java.util.UUID; +import org.junit.Before; +import org.junit.Test; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -26,25 +32,13 @@ import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.assertj.core.api.Assertions; -import org.junit.Before; -import org.junit.Test; -import java.net.URI; -import java.util.ArrayDeque; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.Queue; -import java.util.UUID; - -import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE; import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; import static org.apache.hadoop.fs.s3a.S3ATestUtils.awaitFileStatus; import static org.apache.hadoop.fs.s3a.S3ATestUtils.metadataStorePersistsAuthoritativeBit; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; -import static org.junit.Assume.assumeNotNull; import static org.junit.Assume.assumeTrue; public class ITestS3GuardFsck extends AbstractS3ATestBase { @@ -59,9 +53,9 @@ public void setup() throws Exception { super.setup(); S3AFileSystem fs = getFileSystem(); // These test will fail if no ms - assumeTrue("FS needs to have a metadatastore.", + assertTrue("FS needs to have a metadatastore.", fs.hasMetadataStore()); - assumeTrue("Metadatastore should persist authoritative bit", + assertTrue("Metadatastore should persist authoritative bit", metadataStorePersistsAuthoritativeBit(fs.getMetadataStore())); guardedFs = fs; @@ -101,15 +95,15 @@ public void testIDetectNoMetadataEntry() throws Exception { awaitFileStatus(rawFS, file); final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore, false); + new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3toMs(cwd); + s3GuardFsck.compareS3RootToMs(cwd); assertEquals("Number of pairs should be two.", 2, comparePairs.size()); final S3GuardFsck.ComparePair pair = comparePairs.get(0); - assumeTrue("The pair must contain a violation.", pair.containsViolation()); + assertTrue("The pair must contain a violation.", pair.containsViolation()); assertEquals("The pair must contain only one violation", 1, pair.getViolations().size()); @@ -138,30 +132,30 @@ public void testIDetectNoParentEntry() throws Exception { metadataStore.forgetMetadata(cwd); final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore, false); + new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3toMs(cwd); + s3GuardFsck.compareS3RootToMs(cwd); assertEquals("Number of pairs should be two. The cwd (parent) and the " + "child.", 2, comparePairs.size()); // check the parent that it does not exist final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(cwd)) + .filter(p -> p.getPath().equals(cwd)) .findFirst().get(); - assumeNotNull("The pair should not be null.", cwdPair); - assumeTrue("The cwdPair must contain a violation.", cwdPair.containsViolation()); + assertNotNull("The pair should not be null.", cwdPair); + assertTrue("The cwdPair must contain a violation.", cwdPair.containsViolation()); Assertions.assertThat(cwdPair.getViolations()) .describedAs("Violations in the cwdPair") .contains(S3GuardFsck.Violation.NO_METADATA_ENTRY); // check the child that there's no parent entry. final S3GuardFsck.ComparePair childPair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(file)) + .filter(p -> p.getPath().equals(file)) .findFirst().get(); - assumeNotNull("The pair should not be null.", childPair); - assumeTrue("The childPair must contain a violation.", childPair.containsViolation()); + assertNotNull("The pair should not be null.", childPair); + assertTrue("The childPair must contain a violation.", childPair.containsViolation()); Assertions.assertThat(childPair.getViolations()) .describedAs("Violations in the childPair") .contains(S3GuardFsck.Violation.NO_PARENT_ENTRY); @@ -188,30 +182,30 @@ public void testIDetectParentIsAFile() throws Exception { metadataStore.put(new PathMetadata(newParentFile)); final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore, false); + new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3toMs(cwd); + s3GuardFsck.compareS3RootToMs(cwd); assertEquals("Number of pairs should be two. The cwd (parent) and the " + "child.", 2, comparePairs.size()); // check the parent that it does not exist final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(cwd)) + .filter(p -> p.getPath().equals(cwd)) .findFirst().get(); - assumeNotNull("The pair should not be null.", cwdPair); - assumeTrue("The cwdPair must contain a violation.", cwdPair.containsViolation()); + assertNotNull("The pair should not be null.", cwdPair); + assertTrue("The cwdPair must contain a violation.", cwdPair.containsViolation()); Assertions.assertThat(cwdPair.getViolations()) .describedAs("Violations in the cwdPair") .contains(S3GuardFsck.Violation.DIR_IN_S3_FILE_IN_MS); // check the child that the parent is a file. final S3GuardFsck.ComparePair childPair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(file)) + .filter(p -> p.getPath().equals(file)) .findFirst().get(); - assumeNotNull("The pair should not be null.", childPair); - assumeTrue("The childPair must contain a violation.", childPair.containsViolation()); + assertNotNull("The pair should not be null.", childPair); + assertTrue("The childPair must contain a violation.", childPair.containsViolation()); Assertions.assertThat(childPair.getViolations()) .describedAs("Violations in the childPair") .contains(S3GuardFsck.Violation.PARENT_IS_A_FILE); @@ -238,20 +232,20 @@ public void testIDetectParentTombstoned() throws Exception { metadataStore.put(cwdPmd); final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore, false); + new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3toMs(cwd); + s3GuardFsck.compareS3RootToMs(cwd); assertEquals("Number of pairs should be two. The cwd (parent) and the " + "child.", 2, comparePairs.size()); // check the child that the parent is tombstoned final S3GuardFsck.ComparePair childPair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(file)) + .filter(p -> p.getPath().equals(file)) .findFirst().get(); - assumeNotNull("The pair should not be null.", childPair); - assumeTrue("The childPair must contain a violation.", childPair.containsViolation()); + assertNotNull("The pair should not be null.", childPair); + assertTrue("The childPair must contain a violation.", childPair.containsViolation()); Assertions.assertThat(childPair.getViolations()) .describedAs("Violations in the childPair") .contains(S3GuardFsck.Violation.PARENT_TOMBSTONED); @@ -277,20 +271,20 @@ public void testIDetectDirInS3FileInMs() throws Exception { metadataStore.put(new PathMetadata(newParentFile)); final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore, false); + new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3toMs(cwd); + s3GuardFsck.compareS3RootToMs(cwd); assertEquals("Number of pair should be one. Only the cwd.", 1, comparePairs.size()); // check the child that the dir in s3 is a file in the ms final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(cwd)) + .filter(p -> p.getPath().equals(cwd)) .findFirst().get(); - assumeNotNull("The pair should not be null.", cwdPair); - assumeTrue("The cwdPair must contain a violation.", cwdPair.containsViolation()); + assertNotNull("The pair should not be null.", cwdPair); + assertTrue("The cwdPair must contain a violation.", cwdPair.containsViolation()); Assertions.assertThat(cwdPair.getViolations()) .describedAs("Violations in the cwdPair") .contains(S3GuardFsck.Violation.DIR_IN_S3_FILE_IN_MS); @@ -302,7 +296,7 @@ public void testIDetectDirInS3FileInMs() throws Exception { } @Test - public void testIDetectLengthMismatch() throws Exception { + public void testIDetectFileInS3DirInMs() throws Exception { final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); final Path file = new Path(cwd, "file"); try { @@ -310,39 +304,39 @@ public void testIDetectLengthMismatch() throws Exception { touch(guardedFs, file); awaitFileStatus(guardedFs, file); - // modify the file metadata so the length will not match + // modify the cwd metadata and set that it's not a directory final S3AFileStatus newFile = MetadataStoreTestBase - .basicFileStatus(file, 9999, false, 1); + .basicFileStatus(file, 1, true, 1); metadataStore.put(new PathMetadata(newFile)); final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore, false); + new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3toMs(cwd); + s3GuardFsck.compareS3RootToMs(cwd); - assertEquals("Number of pairs should be two. The cwd (parent) and the " - + "child.", 2, comparePairs.size()); + assertEquals("Number of pairs should be two", 2, + comparePairs.size()); - // check the parent that it does not contain LENGTH_MISMATCH + // check the child that the dir in s3 is a file in the ms final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(cwd)) + .filter(p -> p.getPath().equals(cwd)) .findFirst().get(); - assumeNotNull("The pair should not be null.", cwdPair); - assumeTrue("The cwdPair must contain a violation.", cwdPair.containsViolation()); + assertNotNull("The pair should not be null.", cwdPair); Assertions.assertThat(cwdPair.getViolations()) .describedAs("Violations in the cwdPair") - .doesNotContain(S3GuardFsck.Violation.LENGTH_MISMATCH); + .doesNotContain(S3GuardFsck.Violation.FILE_IN_S3_DIR_IN_MS); - // check the child that there's a LENGTH_MISMATCH - final S3GuardFsck.ComparePair childPair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(file)) + // check the child that the dir in s3 is a file in the ms + final S3GuardFsck.ComparePair filePair = comparePairs.stream() + .filter(p -> p.getPath().equals(file)) .findFirst().get(); - assumeNotNull("The pair should not be null.", childPair); - assumeTrue("The childPair must contain a violation.", childPair.containsViolation()); - Assertions.assertThat(childPair.getViolations()) - .describedAs("Violations in the childPair") - .contains(S3GuardFsck.Violation.LENGTH_MISMATCH); + assertNotNull("The pair should not be null.", filePair); + assertTrue("The filePair must contain a violation.", + filePair.containsViolation()); + Assertions.assertThat(filePair.getViolations()) + .describedAs("Violations in the cwdPair") + .contains(S3GuardFsck.Violation.FILE_IN_S3_DIR_IN_MS); } finally { // delete the working directory with all of its contents rawFS.delete(cwd, true); @@ -352,7 +346,86 @@ public void testIDetectLengthMismatch() throws Exception { } @Test - public void testIDetectModTimeMismatch() throws Exception { + public void testIAuthoritativeDirectoryContentMismatch() throws Exception { + assumeTrue("Authoritative directory listings should be enabled for this " + + "test", guardedFs.hasAuthoritativeMetadataStore()); + // first dir listing will be correct + final Path cwdCorrect = path("/" + getMethodName() + "-" + UUID.randomUUID()); + final Path fileC1 = new Path(cwdCorrect, "fileC1"); + final Path fileC2 = new Path(cwdCorrect, "fileC2"); + + // second dir listing will be incorrect: missing entry from Dynamo + final Path cwdIncorrect = path("/" + getMethodName() + "-" + UUID.randomUUID()); + final Path fileIc1 = new Path(cwdIncorrect, "fileC1"); + final Path fileIc2 = new Path(cwdIncorrect, "fileC2"); + try { + touch(guardedFs, fileC1); + touch(guardedFs, fileC2); + touch(guardedFs, fileIc1); + + awaitFileStatus(rawFS, fileC1); + awaitFileStatus(rawFS, fileC2); + awaitFileStatus(rawFS, fileIc1); + + final S3GuardFsck s3GuardFsck = + new S3GuardFsck(rawFS, metadataStore); + + // get listing from ms and set it authoritative + final DirListingMetadata dlmC = + metadataStore.listChildren(cwdCorrect); + final DirListingMetadata dlmIc = + metadataStore.listChildren(cwdIncorrect); + dlmC.setAuthoritative(true); + dlmIc.setAuthoritative(true); + metadataStore.put(dlmC, null); + metadataStore.put(dlmIc, null); + + final DirListingMetadata dlmCa = + metadataStore.listChildren(cwdCorrect); + + // add a file raw so the listing will be different. + touch(rawFS, fileIc2); + awaitFileStatus(rawFS, fileIc2); + + final List pairsCorrect = + s3GuardFsck.compareS3RootToMs(cwdCorrect); + final List pairsIncorrect = + s3GuardFsck.compareS3RootToMs(cwdIncorrect); + + // check the parent that it does not contain LENGTH_MISMATCH + final S3GuardFsck.ComparePair cwdPair = pairsCorrect.stream() + .filter(p -> p.getPath().equals(cwdCorrect)) + .findFirst().get(); + assertNotNull("The pair should not be null.", cwdPair); + Assertions.assertThat(cwdPair.getViolations()) + .describedAs("Violations in the cwdPair") + .doesNotContain(S3GuardFsck.Violation.AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH); + + // check the child that there's a LENGTH_MISMATCH + final S3GuardFsck.ComparePair childPair = pairsIncorrect.stream() + .filter(p -> p.getPath().equals(cwdIncorrect)) + .findFirst().get(); + assertNotNull("The pair should not be null.", childPair); + assertTrue("The childPair must contain a violation.", childPair.containsViolation()); + Assertions.assertThat(childPair.getViolations()) + .describedAs("Violations in the childPair") + .contains(S3GuardFsck.Violation.AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH); + + } finally { + // cleanup + rawFS.delete(cwdCorrect, true); + rawFS.delete(cwdIncorrect, true); + metadataStore.forgetMetadata(fileIc1); + metadataStore.forgetMetadata(fileIc2); + metadataStore.forgetMetadata(fileC1); + metadataStore.forgetMetadata(fileC2); + metadataStore.forgetMetadata(cwdCorrect); + metadataStore.forgetMetadata(cwdIncorrect); + } + } + + @Test + public void testIDetectLengthMismatch() throws Exception { final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); final Path file = new Path(cwd, "file"); try { @@ -361,45 +434,37 @@ public void testIDetectModTimeMismatch() throws Exception { awaitFileStatus(guardedFs, file); // modify the file metadata so the length will not match - final S3AFileStatus newFileStatus = MetadataStoreTestBase - .basicFileStatus(file, 0, false, 1); - metadataStore.put(new PathMetadata(newFileStatus)); - - // modify the parent meta entry so the MOD_TIME will surely be up to date - final FileStatus oldCwdFileStatus = rawFS.getFileStatus(cwd); - final S3AFileStatus newCwdFileStatus = MetadataStoreTestBase - .basicFileStatus(file, 0, true, - oldCwdFileStatus.getModificationTime()); - metadataStore.put(new PathMetadata(newCwdFileStatus)); + final S3AFileStatus newFile = MetadataStoreTestBase + .basicFileStatus(file, 9999, false, 1); + metadataStore.put(new PathMetadata(newFile)); final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore, false); + new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3toMs(cwd); + s3GuardFsck.compareS3RootToMs(cwd); assertEquals("Number of pairs should be two. The cwd (parent) and the " + "child.", 2, comparePairs.size()); - // check the parent that it does not contain MOD_TIME_MISMATCH + // check the parent that it does not contain LENGTH_MISMATCH final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(cwd)) + .filter(p -> p.getPath().equals(cwd)) .findFirst().get(); - assumeNotNull("The pair should not be null.", cwdPair); - assumeTrue("The cwdPair must contain a violation.", cwdPair.containsViolation()); + assertNotNull("The pair should not be null.", cwdPair); Assertions.assertThat(cwdPair.getViolations()) .describedAs("Violations in the cwdPair") - .doesNotContain(S3GuardFsck.Violation.MOD_TIME_MISMATCH); + .doesNotContain(S3GuardFsck.Violation.LENGTH_MISMATCH); - // check the child that there's a MOD_TIME_MISMATCH + // check the child that there's a LENGTH_MISMATCH final S3GuardFsck.ComparePair childPair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(file)) + .filter(p -> p.getPath().equals(file)) .findFirst().get(); - assumeNotNull("The pair should not be null.", childPair); - assumeTrue("The childPair must contain a violation.", childPair.containsViolation()); + assertNotNull("The pair should not be null.", childPair); + assertTrue("The childPair must contain a violation.", childPair.containsViolation()); Assertions.assertThat(childPair.getViolations()) .describedAs("Violations in the childPair") - .contains(S3GuardFsck.Violation.MOD_TIME_MISMATCH); + .contains(S3GuardFsck.Violation.LENGTH_MISMATCH); } finally { // delete the working directory with all of its contents rawFS.delete(cwd, true); @@ -409,7 +474,7 @@ public void testIDetectModTimeMismatch() throws Exception { } @Test - public void testIBlockSizeMismatch() throws Exception { + public void testIDetectModTimeMismatch() throws Exception { final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); final Path file = new Path(cwd, "file"); try { @@ -419,37 +484,44 @@ public void testIBlockSizeMismatch() throws Exception { // modify the file metadata so the length will not match final S3AFileStatus newFileStatus = MetadataStoreTestBase - .basicFileStatus(1, false, 999, 1, file); + .basicFileStatus(file, 0, false, 1); metadataStore.put(new PathMetadata(newFileStatus)); + // modify the parent meta entry so the MOD_TIME will surely be up to date + final FileStatus oldCwdFileStatus = rawFS.getFileStatus(cwd); + final S3AFileStatus newCwdFileStatus = MetadataStoreTestBase + .basicFileStatus(file, 0, true, + oldCwdFileStatus.getModificationTime()); + metadataStore.put(new PathMetadata(newCwdFileStatus)); + final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore, false); + new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3toMs(cwd); + s3GuardFsck.compareS3RootToMs(cwd); assertEquals("Number of pairs should be two. The cwd (parent) and the " - + "child.", 2, comparePairs.size()); + + "child.", 2, comparePairs.size()); - // check the parent that it does not contain BLOCKSIZE_MISMATCH + // check the parent that it does not contain MOD_TIME_MISMATCH final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(cwd)) + .filter(p -> p.getPath().equals(cwd)) .findFirst().get(); - assumeNotNull("The pair should not be null.", cwdPair); - assumeTrue("The childPair must contain a violation.", cwdPair.containsViolation()); + assertNotNull("The pair should not be null.", cwdPair); Assertions.assertThat(cwdPair.getViolations()) .describedAs("Violations in the cwdPair") - .doesNotContain(S3GuardFsck.Violation.BLOCKSIZE_MISMATCH); + .doesNotContain(S3GuardFsck.Violation.MOD_TIME_MISMATCH); - // check the child that there's a BLOCKSIZE_MISMATCH + // check the child that there's a MOD_TIME_MISMATCH final S3GuardFsck.ComparePair childPair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(file)) + .filter(p -> p.getPath().equals(file)) .findFirst().get(); - assumeNotNull("The pair should not be null.", childPair); - assumeTrue("The childPair must contain a violation.", childPair.containsViolation()); + assertNotNull("The pair should not be null.", childPair); + assertTrue("The childPair must contain a violation.", + childPair.containsViolation()); Assertions.assertThat(childPair.getViolations()) .describedAs("Violations in the childPair") - .contains(S3GuardFsck.Violation.BLOCKSIZE_MISMATCH); + .contains(S3GuardFsck.Violation.MOD_TIME_MISMATCH); } finally { // delete the working directory with all of its contents rawFS.delete(cwd, true); @@ -458,11 +530,6 @@ public void testIBlockSizeMismatch() throws Exception { } } - @Test - public void testIOwnerMismatch() throws Exception { - skip("We don't store the owner in dynamo, so there's no test for this."); - } - @Test public void testIVersionIdMismatch() throws Exception { final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); @@ -478,30 +545,30 @@ public void testIVersionIdMismatch() throws Exception { metadataStore.put(new PathMetadata(newFileStatus)); final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore, false); + new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3toMs(cwd); + s3GuardFsck.compareS3RootToMs(cwd); assertEquals("Number of pairs should be two. The cwd (parent) and the " + "child.", 2, comparePairs.size()); // check the parent that it does not contain BLOCKSIZE_MISMATCH final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(cwd)) + .filter(p -> p.getPath().equals(cwd)) .findFirst().get(); - assumeNotNull("The pair should not be null.", cwdPair); - assumeTrue("The childPair must contain a violation.", cwdPair.containsViolation()); + assertNotNull("The pair should not be null.", cwdPair); Assertions.assertThat(cwdPair.getViolations()) .describedAs("Violations in the cwdPair") .doesNotContain(S3GuardFsck.Violation.VERSIONID_MISMATCH); // check the child that there's a BLOCKSIZE_MISMATCH final S3GuardFsck.ComparePair childPair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(file)) + .filter(p -> p.getPath().equals(file)) .findFirst().get(); - assumeNotNull("The pair should not be null.", childPair); - assumeTrue("The childPair must contain a violation.", childPair.containsViolation()); + assertNotNull("The pair should not be null.", childPair); + assertTrue("The childPair must contain a violation.", + childPair.containsViolation()); Assertions.assertThat(childPair.getViolations()) .describedAs("Violations in the childPair") .contains(S3GuardFsck.Violation.VERSIONID_MISMATCH); @@ -528,30 +595,30 @@ public void testIEtagMismatch() throws Exception { metadataStore.put(new PathMetadata(newFileStatus)); final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore, false); + new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3toMs(cwd); + s3GuardFsck.compareS3RootToMs(cwd); assertEquals("Number of pairs should be two. The cwd (parent) and the " + "child.", 2, comparePairs.size()); // check the parent that it does not contain BLOCKSIZE_MISMATCH final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(cwd)) + .filter(p -> p.getPath().equals(cwd)) .findFirst().get(); - assumeNotNull("The pair should not be null.", cwdPair); - assumeTrue("The childPair must contain a violation.", cwdPair.containsViolation()); + assertNotNull("The pair should not be null.", cwdPair); Assertions.assertThat(cwdPair.getViolations()) .describedAs("Violations in the cwdPair") .doesNotContain(S3GuardFsck.Violation.ETAG_MISMATCH); // check the child that there's a BLOCKSIZE_MISMATCH final S3GuardFsck.ComparePair childPair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(file)) + .filter(p -> p.getPath().equals(file)) .findFirst().get(); - assumeNotNull("The pair should not be null.", childPair); - assumeTrue("The childPair must contain a violation.", childPair.containsViolation()); + assertNotNull("The pair should not be null.", childPair); + assertTrue("The childPair must contain a violation.", + childPair.containsViolation()); Assertions.assertThat(childPair.getViolations()) .describedAs("Violations in the childPair") .contains(S3GuardFsck.Violation.ETAG_MISMATCH); @@ -585,20 +652,20 @@ public void testINoEtag() throws Exception { metadataStore.put(new PathMetadata(newFile2Status)); final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore, false); + new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3toMs(cwd); + s3GuardFsck.compareS3RootToMs(cwd); assertEquals("Number of pairs should be 3. The cwd (parent) and the " + "2 children.", 3, comparePairs.size()); // check file 1 that there's NO_ETAG final S3GuardFsck.ComparePair file1Pair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(file1)) + .filter(p -> p.getPath().equals(file1)) .findFirst().get(); - assumeNotNull("The pair should not be null.", file1Pair); - assumeTrue("The file1Pair must contain a violation.", + assertNotNull("The pair should not be null.", file1Pair); + assertTrue("The file1Pair must contain a violation.", file1Pair.containsViolation()); Assertions.assertThat(file1Pair.getViolations()) .describedAs("Violations in the file1Pair") @@ -606,11 +673,9 @@ public void testINoEtag() throws Exception { // check the child that there's no NO_ETAG violation final S3GuardFsck.ComparePair file2Pair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(file2)) + .filter(p -> p.getPath().equals(file2)) .findFirst().get(); - assumeNotNull("The pair should not be null.", file2Pair); - assumeTrue("The file2Pair must contain a violation.", - file2Pair.containsViolation()); + assertNotNull("The pair should not be null.", file2Pair); Assertions.assertThat(file2Pair.getViolations()) .describedAs("Violations in the file2Pair") .doesNotContain(S3GuardFsck.Violation.NO_ETAG); @@ -623,64 +688,5 @@ public void testINoEtag() throws Exception { } } - @Test - public void testINoVersionId() throws Exception { - final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); - final Path file1 = new Path(cwd, "file1"); - final Path file2 = new Path(cwd, "file2"); - try { - // create a file1 with guarded fs - touch(guardedFs, file1); - touch(guardedFs, file2); - awaitFileStatus(guardedFs, file1); - awaitFileStatus(guardedFs, file2); - - // modify the file1 metadata so there's no versionId - final S3AFileStatus newFile1Status = - new S3AFileStatus(1, 1, file1, 1, "", "etag", null); - // force add versionID to the second file - final S3AFileStatus newFile2Status = - new S3AFileStatus(1, 1, file2, 1, "", "etag", "notnull"); - - metadataStore.put(new PathMetadata(newFile1Status)); - metadataStore.put(new PathMetadata(newFile2Status)); - - final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore, false); - final List comparePairs = - s3GuardFsck.compareS3toMs(cwd); - - assertEquals("Number of pairs should be 3. The cwd (parent) and the " - + "2 children.", 3, comparePairs.size()); - - // check file 1 that there's NO_ETAG - final S3GuardFsck.ComparePair file1Pair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(file1)) - .findFirst().get(); - assumeNotNull("The pair should not be null.", file1Pair); - assumeTrue("The file1Pair must contain a violation.", - file1Pair.containsViolation()); - Assertions.assertThat(file1Pair.getViolations()) - .describedAs("Violations in the file1Pair") - .contains(S3GuardFsck.Violation.NO_VERSIONID); - - // check file 2 that there's no NO_ETAG violation - final S3GuardFsck.ComparePair file2Pair = comparePairs.stream() - .filter(p -> p.getS3FileStatus().getPath().equals(file2)) - .findFirst().get(); - assumeNotNull("The pair should not be null.", file2Pair); - assumeTrue("The file2Pair must contain a violation.", - file2Pair.containsViolation()); - Assertions.assertThat(file2Pair.getViolations()) - .describedAs("Violations in the file2Pair") - .doesNotContain(S3GuardFsck.Violation.NO_VERSIONID); - } finally { - // delete the working directory with all of its contents - rawFS.delete(cwd, true); - metadataStore.forgetMetadata(file1); - metadataStore.forgetMetadata(file2); - metadataStore.forgetMetadata(cwd); - } - } } \ No newline at end of file From 4016351462b6df3e128f64e2458980fb306ef96c Mon Sep 17 00:00:00 2001 From: Gabor Bota Date: Wed, 7 Aug 2019 14:58:59 +0200 Subject: [PATCH 03/16] factored out the big switch block from S3GuardFsckViolationHandler. Using the class defined in the enum instead. Change-Id: I2debc18e70af54ed08d0382bf42e0e11e3100603 --- .../hadoop/fs/s3a/s3guard/S3GuardFsck.java | 53 +++++--- .../s3guard/S3GuardFsckViolationHandler.java | 121 ++++++------------ 2 files changed, 77 insertions(+), 97 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java index be7d9ebabea7d..c9a3d81cc7d63 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java @@ -114,7 +114,8 @@ public List compareS3RootToMs(final Path rootPath) throws IOExcepti // Compare the directory contents if the listing is authoritative final DirListingMetadata msDirListing = metadataStore.listChildren(currentDirPath); - if (msDirListing.isAuthoritative()) { + if (msDirListing != null && + msDirListing.isAuthoritative()) { final ComparePair cP = compareAuthDirListing(s3DirListing, msDirListing); if (cP.containsViolation()) { @@ -243,6 +244,13 @@ protected ComparePair compareFileStatusToPathMetadata( comparePair.violations.add(Violation.MOD_TIME_MISMATCH); } + if(msPathMetadata.getFileStatus().getVersionId() == null) { + // we don't handle missing versionIDs + } else if(s3FileStatus.getVersionId() != msFileStatus.getVersionId()) { + comparePair.violations.add(Violation.VERSIONID_MISMATCH); + } + + if (msPathMetadata.getFileStatus().getETag() == null) { comparePair.violations.add(Violation.NO_ETAG); } else if (s3FileStatus.getETag() != null && @@ -321,40 +329,55 @@ public Path getPath() { } /** - * Violation with severity: + * Violation with severity and the handler. * Defines the severity of the violation between 0-2 * where 0 is the most severe and 2 is the least severe. */ public enum Violation { // No entry in metadatastore - NO_METADATA_ENTRY(1), + NO_METADATA_ENTRY(1, + S3GuardFsckViolationHandler.NoMetadataEntry.class), // A file or directory entry does not have a parent entry - excluding // files and directories in the root. - NO_PARENT_ENTRY(0), + NO_PARENT_ENTRY(0, + S3GuardFsckViolationHandler.NoParentEntry.class), // An entry’s parent is a file - PARENT_IS_A_FILE(0), + PARENT_IS_A_FILE(0, + S3GuardFsckViolationHandler.ParentIsAFile.class), // A file exists under a path for which there is // a tombstone entry in the MS - PARENT_TOMBSTONED(0), + PARENT_TOMBSTONED(0, + S3GuardFsckViolationHandler.ParentTombstoned.class), // A directory in S3 is a file entry in the MS - DIR_IN_S3_FILE_IN_MS(0), + DIR_IN_S3_FILE_IN_MS(0, + S3GuardFsckViolationHandler.DirInS3FileInMs.class), // A file in S3 is a directory in the MS - FILE_IN_S3_DIR_IN_MS(0), - AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH(1), + FILE_IN_S3_DIR_IN_MS(0, + S3GuardFsckViolationHandler.FileInS3DirInMs.class), + AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH(1, + S3GuardFsckViolationHandler.AuthDirContentMismatch.class), // Attribute mismatch - LENGTH_MISMATCH(0), - MOD_TIME_MISMATCH(2), + LENGTH_MISMATCH(0, + S3GuardFsckViolationHandler.LengthMismatch.class), + MOD_TIME_MISMATCH(2, + S3GuardFsckViolationHandler.ModTimeMismatch.class), // If there's a versionID the mismatch is severe - VERSIONID_MISMATCH(0), + VERSIONID_MISMATCH(0, + S3GuardFsckViolationHandler.VersionIdMismatch.class), // If there's an etag the mismatch is severe - ETAG_MISMATCH(0), + ETAG_MISMATCH(0, + S3GuardFsckViolationHandler.EtagMismatch.class), // Don't worry too much if we don't have an etag - NO_ETAG(2); + NO_ETAG(2, + S3GuardFsckViolationHandler.NoEtag.class); int severity; + Class handler; - Violation(int s) { + Violation(int s, + Class h) { this.severity = s; + this.handler = h; } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java index 65e715ff7b528..c1416cf2549f2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java @@ -18,19 +18,17 @@ package org.apache.hadoop.fs.s3a.s3guard; -import com.google.common.collect.Lists; -import org.apache.commons.math3.ode.UnknownParameterException; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.lang.reflect.InvocationTargetException; import java.util.Arrays; /** * Violation handler for the S3Guard's fsck - * */ public class S3GuardFsckViolationHandler { private static final Logger LOG = LoggerFactory.getLogger( @@ -53,67 +51,26 @@ public void handle(S3GuardFsck.ComparePair comparePair) { return; } - ViolationHandler handler; - StringBuilder sB = new StringBuilder( String.format("%sOn path: %s%n", NEWLINE, comparePair.getPath()) ); + // Create a new instance of the handler and use it. for (S3GuardFsck.Violation violation : comparePair.getViolations()) { - switch (violation) { - case NO_METADATA_ENTRY: - handler = new NoMetadataEntryViolation(comparePair); - sB.append(handler.getError()); - break; - case NO_PARENT_ENTRY: - handler = new NoParentEntryViolation(comparePair); - sB.append(handler.getError()); - break; - case PARENT_IS_A_FILE: - handler = new ParentIsAFileViolation(comparePair); - sB.append(handler.getError()); - break; - case PARENT_TOMBSTONED: - handler = new ParentTombstonedViolation(comparePair); - sB.append(handler.getError()); - break; - case DIR_IN_S3_FILE_IN_MS: - handler = new DirInS3FileInMsViolation(comparePair); - sB.append(handler.getError()); - break; - case FILE_IN_S3_DIR_IN_MS: - handler = new FileInS3DirInMsViolation(comparePair); - sB.append(handler.getError()); - break; - case AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH: - handler = new AuthDirContentMismatchViolation(comparePair); - sB.append(handler.getError()); - break; - case LENGTH_MISMATCH: - handler = new LengthMismatchViolation(comparePair); - sB.append(handler.getError()); - break; - case MOD_TIME_MISMATCH: - handler = new ModTimeMismatchViolation(comparePair); - sB.append(handler.getError()); - break; - case VERSIONID_MISMATCH: - handler = new VersionIdMismatchViolation(comparePair); - sB.append(handler.getError()); - break; - case ETAG_MISMATCH: - handler = new EtagMismatchViolation(comparePair); - sB.append(handler.getError()); - break; - case NO_ETAG: - handler = new NoEtagViolation(comparePair); - sB.append(handler.getError()); - break; - default: - LOG.error("UNKNOWN VIOLATION: {}", violation.toString()); - throw new UnknownParameterException("Unknown Violation: " + - violation.toString()); + try { + ViolationHandler handler = + violation.handler.getDeclaredConstructor(S3GuardFsck.ComparePair.class) + .newInstance(comparePair); + final String errorStr = handler.getError(); + sB.append(errorStr); + } catch (NoSuchMethodException e) { + LOG.error("Can not find declared constructor for handler: {}", + violation.handler); + } catch (IllegalAccessException | InstantiationException | InvocationTargetException e) { + LOG.error("Can not instantiate handler: {}", + violation.handler); } + sB.append(NEWLINE); } @@ -142,9 +99,9 @@ public ViolationHandler(S3GuardFsck.ComparePair comparePair) { abstract String getError(); } - public static class NoMetadataEntryViolation extends ViolationHandler { + public static class NoMetadataEntry extends ViolationHandler { - public NoMetadataEntryViolation(S3GuardFsck.ComparePair comparePair) { + public NoMetadataEntry(S3GuardFsck.ComparePair comparePair) { super(comparePair); } @@ -154,9 +111,9 @@ public String getError() { } } - public static class NoParentEntryViolation extends ViolationHandler { + public static class NoParentEntry extends ViolationHandler { - public NoParentEntryViolation(S3GuardFsck.ComparePair comparePair) { + public NoParentEntry(S3GuardFsck.ComparePair comparePair) { super(comparePair); } @@ -166,9 +123,9 @@ public String getError() { } } - public static class ParentIsAFileViolation extends ViolationHandler { + public static class ParentIsAFile extends ViolationHandler { - public ParentIsAFileViolation(S3GuardFsck.ComparePair comparePair) { + public ParentIsAFile(S3GuardFsck.ComparePair comparePair) { super(comparePair); } @@ -178,9 +135,9 @@ public String getError() { } } - public static class ParentTombstonedViolation extends ViolationHandler { + public static class ParentTombstoned extends ViolationHandler { - public ParentTombstonedViolation(S3GuardFsck.ComparePair comparePair) { + public ParentTombstoned(S3GuardFsck.ComparePair comparePair) { super(comparePair); } @@ -190,9 +147,9 @@ public String getError() { } } - public static class DirInS3FileInMsViolation extends ViolationHandler { + public static class DirInS3FileInMs extends ViolationHandler { - public DirInS3FileInMsViolation(S3GuardFsck.ComparePair comparePair) { + public DirInS3FileInMs(S3GuardFsck.ComparePair comparePair) { super(comparePair); } @@ -202,9 +159,9 @@ public String getError() { } } - public static class FileInS3DirInMsViolation extends ViolationHandler { + public static class FileInS3DirInMs extends ViolationHandler { - public FileInS3DirInMsViolation(S3GuardFsck.ComparePair comparePair) { + public FileInS3DirInMs(S3GuardFsck.ComparePair comparePair) { super(comparePair); } @@ -214,9 +171,9 @@ public String getError() { } } - public static class AuthDirContentMismatchViolation extends ViolationHandler { + public static class AuthDirContentMismatch extends ViolationHandler { - public AuthDirContentMismatchViolation(S3GuardFsck.ComparePair comparePair) { + public AuthDirContentMismatch(S3GuardFsck.ComparePair comparePair) { super(comparePair); } @@ -230,9 +187,9 @@ public String getError() { } } - public static class LengthMismatchViolation extends ViolationHandler { + public static class LengthMismatch extends ViolationHandler { - public LengthMismatchViolation(S3GuardFsck.ComparePair comparePair) { + public LengthMismatch(S3GuardFsck.ComparePair comparePair) { super(comparePair); } @@ -242,9 +199,9 @@ public LengthMismatchViolation(S3GuardFsck.ComparePair comparePair) { } } - public static class ModTimeMismatchViolation extends ViolationHandler { + public static class ModTimeMismatch extends ViolationHandler { - public ModTimeMismatchViolation(S3GuardFsck.ComparePair comparePair) { + public ModTimeMismatch(S3GuardFsck.ComparePair comparePair) { super(comparePair); } @@ -256,9 +213,9 @@ public String getError() { } } - public static class VersionIdMismatchViolation extends ViolationHandler { + public static class VersionIdMismatch extends ViolationHandler { - public VersionIdMismatchViolation(S3GuardFsck.ComparePair comparePair) { + public VersionIdMismatch(S3GuardFsck.ComparePair comparePair) { super(comparePair); } @@ -269,9 +226,9 @@ public String getError() { } } - public static class EtagMismatchViolation extends ViolationHandler { + public static class EtagMismatch extends ViolationHandler { - public EtagMismatchViolation(S3GuardFsck.ComparePair comparePair) { + public EtagMismatch(S3GuardFsck.ComparePair comparePair) { super(comparePair); } @@ -282,9 +239,9 @@ public String getError() { } } - public static class NoEtagViolation extends ViolationHandler { + public static class NoEtag extends ViolationHandler { - public NoEtagViolation(S3GuardFsck.ComparePair comparePair) { + public NoEtag(S3GuardFsck.ComparePair comparePair) { super(comparePair); } From debe4c351b3891627b145aad24d52c269b14b31d Mon Sep 17 00:00:00 2001 From: Gabor Bota Date: Wed, 7 Aug 2019 15:04:18 +0200 Subject: [PATCH 04/16] teardown added to itest Change-Id: Ifcaaf2ca2027e81f3be0dc1337b34aa315b8d5c1 --- .../apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java | 13 +++++++------ .../hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java | 11 +++++++++++ 2 files changed, 18 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java index c9a3d81cc7d63..fa64b2967b213 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java @@ -309,12 +309,6 @@ public boolean containsViolation() { return !violations.isEmpty(); } - @Override public String toString() { - return "ComparePair{" + "s3FileStatus=" + s3FileStatus - + ", msPathMetadata=" + msPathMetadata + ", violations=" + violations - + '}'; - } - public DirListingMetadata getMsDirListing() { return msDirListing; } @@ -326,6 +320,13 @@ public FileStatus[] getS3DirListing() { public Path getPath() { return path; } + + @Override public String toString() { + return "ComparePair{" + "s3FileStatus=" + s3FileStatus + + ", msPathMetadata=" + msPathMetadata + ", s3DirListing=" + Arrays + .toString(s3DirListing) + ", msDirListing=" + msDirListing + ", path=" + + path + ", violations=" + violations + '}'; + } } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java index e2f0ed0b0c5ce..3383c858b81c3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java @@ -22,6 +22,8 @@ import java.net.URI; import java.util.List; import java.util.UUID; + +import org.apache.hadoop.io.IOUtils; import org.junit.Before; import org.junit.Test; @@ -67,6 +69,15 @@ public void setup() throws Exception { rawFS.hasMetadataStore()); } + @Override + public void teardown() throws Exception { + if (guardedFs != null) { + IOUtils.cleanupWithLogger(LOG, guardedFs); + } + IOUtils.cleanupWithLogger(LOG, rawFS); + super.teardown(); + } + /** * Create a test filesystem which is always unguarded. * This filesystem MUST be closed in test teardown. From 6565328523f4a92c960051edaf421eaf6831e38f Mon Sep 17 00:00:00 2001 From: Gabor Bota Date: Mon, 12 Aug 2019 13:31:43 +0200 Subject: [PATCH 05/16] CLI test is working in org.apache.hadoop.fs.s3a.s3guard.ITestS3GuardToolDynamoDB#testCLIFsckWithParam Change-Id: I6bbb331b6c0a41c61043e482b95504fda8a50596 --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 3 +- .../hadoop/fs/s3a/s3guard/S3GuardFsck.java | 8 ++-- .../s3guard/S3GuardFsckViolationHandler.java | 10 ++--- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 43 +++++++++++++------ .../s3a/s3guard/ITestS3GuardDraftingFsck.java | 18 -------- .../s3a/s3guard/ITestS3GuardToolDynamoDB.java | 17 +++----- 6 files changed, 48 insertions(+), 51 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index dd26a0b5f6272..2cac72504a628 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 @@ -1491,7 +1491,8 @@ public boolean hasMetadataStore() { * @return true if there is a metadata store and the authoritative flag * is set for this filesystem. */ - @VisibleForTesting public boolean hasAuthoritativeMetadataStore() { + @VisibleForTesting + public boolean hasAuthoritativeMetadataStore() { return hasMetadataStore() && allowAuthoritativeMetadataStore; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java index fa64b2967b213..5581e0af3e637 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java @@ -84,11 +84,12 @@ public class S3GuardFsck { * Uses {@link S3GuardFsckViolationHandler} to handle violations. * The violations are listed in Enums: {@link Violation} * - * @param rootPath the root path to start the traversal + * @param p the root path to start the traversal * @throws IOException * @return */ - public List compareS3RootToMs(final Path rootPath) throws IOException { + public List compareS3RootToMs(Path p) throws IOException { + final Path rootPath = rawFS.qualify(p); final S3AFileStatus root = (S3AFileStatus) rawFS.getFileStatus(rootPath); final List comparePairs = new ArrayList<>(); @@ -114,8 +115,7 @@ public List compareS3RootToMs(final Path rootPath) throws IOExcepti // Compare the directory contents if the listing is authoritative final DirListingMetadata msDirListing = metadataStore.listChildren(currentDirPath); - if (msDirListing != null && - msDirListing.isAuthoritative()) { + if (msDirListing != null && msDirListing.isAuthoritative()) { final ComparePair cP = compareAuthDirListing(s3DirListing, msDirListing); if (cP.containsViolation()) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java index c1416cf2549f2..b5a73471af3e2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java @@ -28,7 +28,7 @@ import java.util.Arrays; /** - * Violation handler for the S3Guard's fsck + * Violation handler for the S3Guard's fsck. */ public class S3GuardFsckViolationHandler { private static final Logger LOG = LoggerFactory.getLogger( @@ -51,9 +51,9 @@ public void handle(S3GuardFsck.ComparePair comparePair) { return; } - StringBuilder sB = new StringBuilder( - String.format("%sOn path: %s%n", NEWLINE, comparePair.getPath()) - ); + StringBuilder sB = new StringBuilder(); + sB.append(NEWLINE) + .append("On path: ").append(comparePair.getPath()).append(NEWLINE); // Create a new instance of the handler and use it. for (S3GuardFsck.Violation violation : comparePair.getViolations()) { @@ -70,10 +70,8 @@ public void handle(S3GuardFsck.ComparePair comparePair) { LOG.error("Can not instantiate handler: {}", violation.handler); } - sB.append(NEWLINE); } - LOG.error(sB.toString()); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java index 9a31102d33a13..e4c2c2c1dbab5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java @@ -1504,7 +1504,6 @@ static class Fsck extends S3GuardTool { Fsck(Configuration conf) { super(conf, CHECK_FLAG); - addAgeOptions(); } @Override @@ -1520,32 +1519,52 @@ public String getUsage() { public int run(String[] args, PrintStream out) throws InterruptedException, IOException { List paths = parseArgs(args); - try { - parseDynamoDBRegion(paths); - } catch (ExitUtil.ExitException e) { - errorln(USAGE); - throw e; + if (paths.isEmpty()) { + out.println(USAGE); + throw invalidArgs("no arguments"); } - if (paths.size() == 0) { + String s3Path = paths.get(0); + try { + initS3AFileSystem(s3Path); + } catch (Exception e) { + errorln("Failed to initialize S3AFileSystem from path: " + s3Path); errorln(USAGE); return ERROR; } - final MetadataStore ms = initMetadataStore(false); - String tableName = getConf().getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY); - initS3AFileSystem("s3a://" + tableName); + URI uri = toUri(s3Path); + Path root; + if (uri.getPath().isEmpty()) { + root = new Path("/"); + } else { + root = new Path(uri.getPath()); + } + final S3AFileSystem fs = getFilesystem(); + initMetadataStore(false); + final MetadataStore ms = getStore(); + + if (ms == null || + !(ms instanceof DynamoDBMetadataStore)) { + errorln(s3Path + " path uses MS: " + ms.toString()); + errorln(NAME + " can be only used with a DynamoDB backed s3a bucket."); + errorln(USAGE); + return ERROR; + } - if (paths.get(0).equals(CHECK_FLAG)) { + final CommandFormat commandFormat = getCommandFormat(); + if (commandFormat.getOpt(CHECK_FLAG)) { // do the check S3GuardFsck s3GuardFsck = new S3GuardFsck(fs, ms); - s3GuardFsck.compareS3RootToMs(fs.qualify(new Path("/"))); + s3GuardFsck.compareS3RootToMs(fs.qualify(root)); } else { + errorln("No supported operation is selected."); errorln(USAGE); return ERROR; } + out.flush(); return SUCCESS; } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDraftingFsck.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDraftingFsck.java index 1bb727df01d3b..82030e7d03c7e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDraftingFsck.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDraftingFsck.java @@ -18,44 +18,26 @@ package org.apache.hadoop.fs.s3a.s3guard; -import com.amazonaws.services.dynamodbv2.document.Item; -import com.amazonaws.services.dynamodbv2.document.ItemCollection; -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.spec.QuerySpec; -import com.amazonaws.services.dynamodbv2.document.utils.ValueMap; import com.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder; -import com.amazonaws.services.dynamodbv2.xspec.ScanExpressionSpec; import org.apache.hadoop.conf.Configuration; 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.s3a.AbstractS3ATestBase; -import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.junit.Before; import org.junit.Test; -import java.io.File; -import java.io.IOException; import java.net.URI; import java.util.ArrayDeque; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Queue; -import static java.util.stream.Collectors.toList; import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE; import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; import static org.apache.hadoop.fs.s3a.S3ATestUtils.metadataStorePersistsAuthoritativeBit; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; -import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.itemToPathMetadata; -import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.pathToParentKeyAttribute; import static org.junit.Assume.assumeTrue; public class ITestS3GuardDraftingFsck extends AbstractS3ATestBase { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java index 6db44f344da70..493c876f8e73c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java @@ -32,6 +32,7 @@ import com.amazonaws.services.dynamodbv2.model.ResourceInUseException; import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException; import com.amazonaws.services.dynamodbv2.model.Tag; +import org.apache.hadoop.util.ExitUtil; import org.junit.Assert; import org.junit.Assume; import org.junit.AssumptionViolatedException; @@ -291,18 +292,14 @@ public void testDestroyUnknownTable() throws Throwable { @Test public void testCLIFsckWithoutParam() throws Exception { - run(S3GuardTool.Fsck.NAME, "check"); + intercept(ExitUtil.ExitException.class, () -> run(Fsck.NAME)); } @Test - public void testCLIFsckWithPathParam() throws Exception { - fail("TODO implement"); - run(S3GuardTool.Fsck.NAME, "check"); - } - - @Test - public void testCLIFsckWithVersionIdParam() throws Exception { - fail("TODO implement"); - run(S3GuardTool.Fsck.NAME, "check"); + public void testCLIFsckWithParam() throws Exception { + final int result = run(S3GuardTool.Fsck.NAME, "-check", + "s3a://" + getFileSystem().getBucket()); + assertEquals("The result should be success when fsck is running with " + + "correct parameters.", SUCCESS, result); } } From 7d32a407fd3d86f1c8600e3932acb448417de466 Mon Sep 17 00:00:00 2001 From: Gabor Bota Date: Mon, 12 Aug 2019 13:38:22 +0200 Subject: [PATCH 06/16] Remove ITestS3GuardDraftingFsck - move to a gist Change-Id: I611e7421ba061d1048bd6bb182f5238f810a400a --- .../s3a/s3guard/ITestS3GuardDraftingFsck.java | 159 ------------------ 1 file changed, 159 deletions(-) delete mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDraftingFsck.java diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDraftingFsck.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDraftingFsck.java deleted file mode 100644 index 82030e7d03c7e..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDraftingFsck.java +++ /dev/null @@ -1,159 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3a.s3guard; - -import com.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; -import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.junit.Before; -import org.junit.Test; - -import java.net.URI; -import java.util.ArrayDeque; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.Queue; - -import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE; -import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.metadataStorePersistsAuthoritativeBit; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; -import static org.junit.Assume.assumeTrue; - -public class ITestS3GuardDraftingFsck extends AbstractS3ATestBase { - - private S3AFileSystem guardedFs; - private S3AFileSystem rawFS; - - private MetadataStore metadataStore; - - @Before - public void setup() throws Exception { - super.setup(); - S3AFileSystem fs = getFileSystem(); - // These test will fail if no ms - assumeTrue("FS needs to have a metadatastore.", - fs.hasMetadataStore()); - assumeTrue("Metadatastore should persist authoritative bit", - metadataStorePersistsAuthoritativeBit(fs.getMetadataStore())); - - guardedFs = fs; - metadataStore = fs.getMetadataStore(); - - // create raw fs without s3guard - rawFS = createUnguardedFS(); - assertFalse("Raw FS still has S3Guard " + rawFS, - rawFS.hasMetadataStore()); - } - - /** - * Create a test filesystem which is always unguarded. - * This filesystem MUST be closed in test teardown. - * @return the new FS - */ - private S3AFileSystem createUnguardedFS() throws Exception { - S3AFileSystem testFS = getFileSystem(); - Configuration config = new Configuration(testFS.getConf()); - URI uri = testFS.getUri(); - - removeBaseAndBucketOverrides(uri.getHost(), config, - S3_METADATA_STORE_IMPL); - removeBaseAndBucketOverrides(uri.getHost(), config, - METADATASTORE_AUTHORITATIVE); - S3AFileSystem fs2 = new S3AFileSystem(); - fs2.initialize(uri, config); - return fs2; - } - - @Test - public void testTraverseS3() throws Exception { - final FileStatus root = rawFS.getFileStatus(path("/")); - - final Queue queue = new ArrayDeque<>(); - queue.add(root); - - System.out.println("Dir structure: "); - - while (!queue.isEmpty()) { - // pop front node from the queue and print it - final FileStatus currentDir = queue.poll(); - - System.out.println(currentDir.getPath()); - - // get a listing of that dir from s3 - final Path currentDirPath = currentDir.getPath(); - final List children = - Arrays.asList(rawFS.listStatus(currentDirPath)); - - // add each elem to queue - children.stream().filter(pm -> pm.isDirectory()) - .forEach(pm -> queue.add(pm)); - - } - } - - - @Test - public void testTraverseDynamo() throws Exception { - final PathMetadata root = metadataStore.get(path("/")); - - final Queue queue = new ArrayDeque<>(); - queue.add(root); - - System.out.println("Dir structure: "); - - while (!queue.isEmpty()) { - // pop front node from the queue and print it - final PathMetadata currentDir = queue.poll(); - - System.out.println(currentDir.getFileStatus().getPath()); - - // get a listing of that dir from dynamo - final Path currentDirPath = currentDir.getFileStatus().getPath(); - final Collection children = - metadataStore.listChildren(currentDirPath).getListing(); - - // add each elem to queue - children.stream().filter(pm -> pm.getFileStatus().isDirectory()) - .forEach(pm -> queue.add(pm)); - } - } - - @Test - public void testBuildGraphFromDynamo() throws Exception { - S3GuardTableAccess tableAccess = new S3GuardTableAccess( - (DynamoDBMetadataStore) metadataStore); - - ExpressionSpecBuilder builder = new ExpressionSpecBuilder(); - builder.withCondition( - ExpressionSpecBuilder.S("parent") - .beginsWith("/") - ); - final Iterable ddbPathMetadata = - tableAccess.scanMetadata(builder); - - ddbPathMetadata.iterator().forEachRemaining(pmd -> { - System.out.println(pmd.getFileStatus().getPath()); - }); - } -} \ No newline at end of file From fe0f7c44182bccd53ae621b2c1e8cccb9b8a2153 Mon Sep 17 00:00:00 2001 From: Gabor Bota Date: Tue, 13 Aug 2019 12:41:03 +0200 Subject: [PATCH 07/16] fix findbugs and checkstyle issues. long line lenght checks are skipped because of readability. Change-Id: I0660c181ec07e2c0addd906fdac41b92169283f5 --- .../hadoop/fs/s3a/s3guard/S3GuardFsck.java | 31 ++++-- .../s3guard/S3GuardFsckViolationHandler.java | 99 +++++++++++++++---- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 2 +- .../fs/s3a/s3guard/ITestS3GuardFsck.java | 12 ++- 4 files changed, 110 insertions(+), 34 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java index 5581e0af3e637..0a9f31a48e58f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java @@ -56,7 +56,7 @@ public class S3GuardFsck { private DynamoDBMetadataStore metadataStore; /** - * Creates an S3GuardFsck + * Creates an S3GuardFsck. * @param fs the filesystem to compare to * @param ms metadatastore the metadatastore to compare with (dynamo) */ @@ -245,7 +245,8 @@ protected ComparePair compareFileStatusToPathMetadata( } if(msPathMetadata.getFileStatus().getVersionId() == null) { - // we don't handle missing versionIDs + LOG.debug("Missing versionIDs are skipped right now. A HEAD request is " + + "required for each object to get the versionID."); } else if(s3FileStatus.getVersionId() != msFileStatus.getVersionId()) { comparePair.violations.add(Violation.VERSIONID_MISMATCH); } @@ -265,12 +266,14 @@ private Path path(String s) { return rawFS.makeQualified(new Path(s)); } - + /** + * A compare pair with the pair of metadata and the list of violations. + */ public static class ComparePair { private final S3AFileStatus s3FileStatus; private final PathMetadata msPathMetadata; - private final FileStatus[] s3DirListing; + private final List s3DirListing; private final DirListingMetadata msDirListing; private final Path path; @@ -286,7 +289,7 @@ public static class ComparePair { } ComparePair(FileStatus[] s3DirListing, DirListingMetadata msDirListing) { - this.s3DirListing = s3DirListing; + this.s3DirListing = Arrays.asList(s3DirListing); this.msDirListing = msDirListing; this.s3FileStatus = null; this.msPathMetadata = null; @@ -313,7 +316,7 @@ public DirListingMetadata getMsDirListing() { return msDirListing; } - public FileStatus[] getS3DirListing() { + public List getS3DirListing() { return s3DirListing; } @@ -323,8 +326,8 @@ public Path getPath() { @Override public String toString() { return "ComparePair{" + "s3FileStatus=" + s3FileStatus - + ", msPathMetadata=" + msPathMetadata + ", s3DirListing=" + Arrays - .toString(s3DirListing) + ", msDirListing=" + msDirListing + ", path=" + + ", msPathMetadata=" + msPathMetadata + ", s3DirListing=" + + s3DirListing + ", msDirListing=" + msDirListing + ", path=" + path + ", violations=" + violations + '}'; } } @@ -372,13 +375,21 @@ public enum Violation { NO_ETAG(2, S3GuardFsckViolationHandler.NoEtag.class); - int severity; - Class handler; + private int severity; + private Class handler; Violation(int s, Class h) { this.severity = s; this.handler = h; } + + public int getSeverity() { + return severity; + } + + public Class getHandler() { + return handler; + } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java index b5a73471af3e2..d44ee5bcfb957 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java @@ -26,6 +26,7 @@ import java.lang.reflect.InvocationTargetException; import java.util.Arrays; +import java.util.List; /** * Violation handler for the S3Guard's fsck. @@ -36,7 +37,7 @@ public class S3GuardFsckViolationHandler { private S3AFileSystem rawFs; private DynamoDBMetadataStore metadataStore; - private static String NEWLINE = System.getProperty("line.separator"); + private static String newLine = System.getProperty("line.separator"); public S3GuardFsckViolationHandler(S3AFileSystem fs, DynamoDBMetadataStore ddbms) { @@ -52,35 +53,39 @@ public void handle(S3GuardFsck.ComparePair comparePair) { } StringBuilder sB = new StringBuilder(); - sB.append(NEWLINE) - .append("On path: ").append(comparePair.getPath()).append(NEWLINE); + sB.append(newLine) + .append("On path: ").append(comparePair.getPath()).append(newLine); // Create a new instance of the handler and use it. for (S3GuardFsck.Violation violation : comparePair.getViolations()) { try { - ViolationHandler handler = - violation.handler.getDeclaredConstructor(S3GuardFsck.ComparePair.class) + ViolationHandler handler = violation.getHandler() + .getDeclaredConstructor(S3GuardFsck.ComparePair.class) .newInstance(comparePair); final String errorStr = handler.getError(); sB.append(errorStr); } catch (NoSuchMethodException e) { LOG.error("Can not find declared constructor for handler: {}", - violation.handler); + violation.getHandler()); } catch (IllegalAccessException | InstantiationException | InvocationTargetException e) { LOG.error("Can not instantiate handler: {}", - violation.handler); + violation.getHandler()); } - sB.append(NEWLINE); + sB.append(newLine); } LOG.error(sB.toString()); } + /** + * Violation handler abstract class. + * This class should be extended for violation handlers. + */ public static abstract class ViolationHandler { - final PathMetadata pathMetadata; - final S3AFileStatus s3FileStatus; - final S3AFileStatus msFileStatus; - final FileStatus[] s3DirListing; - final DirListingMetadata msDirListing; + private final PathMetadata pathMetadata; + private final S3AFileStatus s3FileStatus; + private final S3AFileStatus msFileStatus; + private final List s3DirListing; + private final DirListingMetadata msDirListing; public ViolationHandler(S3GuardFsck.ComparePair comparePair) { pathMetadata = comparePair.getMsPathMetadata(); @@ -95,8 +100,31 @@ public ViolationHandler(S3GuardFsck.ComparePair comparePair) { } abstract String getError(); + + public PathMetadata getPathMetadata() { + return pathMetadata; + } + + public S3AFileStatus getS3FileStatus() { + return s3FileStatus; + } + + public S3AFileStatus getMsFileStatus() { + return msFileStatus; + } + + public List getS3DirListing() { + return s3DirListing; + } + + public DirListingMetadata getMsDirListing() { + return msDirListing; + } } + /** + * The violation handler when there's no matching metadata entry in the MS. + */ public static class NoMetadataEntry extends ViolationHandler { public NoMetadataEntry(S3GuardFsck.ComparePair comparePair) { @@ -109,6 +137,9 @@ public String getError() { } } + /** + * The violation handler when there's no parent entry. + */ public static class NoParentEntry extends ViolationHandler { public NoParentEntry(S3GuardFsck.ComparePair comparePair) { @@ -121,6 +152,9 @@ public String getError() { } } + /** + * The violation handler when the parent of an entry is a file. + */ public static class ParentIsAFile extends ViolationHandler { public ParentIsAFile(S3GuardFsck.ComparePair comparePair) { @@ -133,6 +167,9 @@ public String getError() { } } + /** + * The violation handler when the parent of an entry is tombstoned. + */ public static class ParentTombstoned extends ViolationHandler { public ParentTombstoned(S3GuardFsck.ComparePair comparePair) { @@ -145,6 +182,9 @@ public String getError() { } } + /** + * The violation handler when there's a directory is a file metadata in MS. + */ public static class DirInS3FileInMs extends ViolationHandler { public DirInS3FileInMs(S3GuardFsck.ComparePair comparePair) { @@ -157,6 +197,9 @@ public String getError() { } } + /** + * The violation handler when a file metadata is a directory in MS. + */ public static class FileInS3DirInMs extends ViolationHandler { public FileInS3DirInMs(S3GuardFsck.ComparePair comparePair) { @@ -169,6 +212,9 @@ public String getError() { } } + /** + * The violation handler when there's a directory listing content mismatch. + */ public static class AuthDirContentMismatch extends ViolationHandler { public AuthDirContentMismatch(S3GuardFsck.ComparePair comparePair) { @@ -180,11 +226,14 @@ public String getError() { final String str = String.format( "The content of an authoritative directory listing does " + "not match the content of the S3 listing. S3: %s, MS: %s", - Arrays.asList(s3DirListing), msDirListing.getListing()); + Arrays.asList(getS3DirListing()), getMsDirListing().getListing()); return str; } } + /** + * The violation handler when there's a length mismatch. + */ public static class LengthMismatch extends ViolationHandler { public LengthMismatch(S3GuardFsck.ComparePair comparePair) { @@ -193,10 +242,13 @@ public LengthMismatch(S3GuardFsck.ComparePair comparePair) { @Override public String getError() { return String.format("getLen mismatch - s3: %s, ms: %s", - s3FileStatus.getLen(), msFileStatus.getLen()); + getS3FileStatus().getLen(), getMsFileStatus().getLen()); } } + /** + * The violation handler when there's a modtime mismatch. + */ public static class ModTimeMismatch extends ViolationHandler { public ModTimeMismatch(S3GuardFsck.ComparePair comparePair) { @@ -206,11 +258,14 @@ public ModTimeMismatch(S3GuardFsck.ComparePair comparePair) { @Override public String getError() { return String.format("getModificationTime mismatch - s3: %s, ms: %s", - s3FileStatus.getModificationTime(), - msFileStatus.getModificationTime()); + getS3FileStatus().getModificationTime(), + getMsFileStatus().getModificationTime()); } } + /** + * The violation handler when there's a version id mismatch + */ public static class VersionIdMismatch extends ViolationHandler { public VersionIdMismatch(S3GuardFsck.ComparePair comparePair) { @@ -220,10 +275,13 @@ public VersionIdMismatch(S3GuardFsck.ComparePair comparePair) { @Override public String getError() { return String.format("getVersionId mismatch - s3: %s, ms: %s", - s3FileStatus.getVersionId(), msFileStatus.getVersionId()); + getS3FileStatus().getVersionId(), getMsFileStatus().getVersionId()); } } + /** + * The violation handler when there's an etag mismatch. + */ public static class EtagMismatch extends ViolationHandler { public EtagMismatch(S3GuardFsck.ComparePair comparePair) { @@ -233,10 +291,13 @@ public EtagMismatch(S3GuardFsck.ComparePair comparePair) { @Override public String getError() { return String.format("getETag mismatch - s3: %s, ms: %s", - s3FileStatus.getETag(), msFileStatus.getETag()); + getS3FileStatus().getETag(), getMsFileStatus().getETag()); } } + /** + * The violation handler when there's no etag. + */ public static class NoEtag extends ViolationHandler { public NoEtag(S3GuardFsck.ComparePair comparePair) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java index e4c2c2c1dbab5..5da92f7e3c453 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java @@ -1547,7 +1547,7 @@ public int run(String[] args, PrintStream out) throws if (ms == null || !(ms instanceof DynamoDBMetadataStore)) { - errorln(s3Path + " path uses MS: " + ms.toString()); + errorln(s3Path + " path uses MS: " + ms); errorln(NAME + " can be only used with a DynamoDB backed s3a bucket."); errorln(USAGE); return ERROR; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java index 3383c858b81c3..a88ef0e8a883f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java @@ -43,6 +43,10 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; import static org.junit.Assume.assumeTrue; +/** + * Integration tests for the S3Guard Fsck against a dyamodb backed metadata + * store. + */ public class ITestS3GuardFsck extends AbstractS3ATestBase { private S3AFileSystem guardedFs; @@ -551,8 +555,8 @@ public void testIVersionIdMismatch() throws Exception { awaitFileStatus(guardedFs, file); // modify the file metadata so the versionId will not match - final S3AFileStatus newFileStatus = new S3AFileStatus(1, 1, file, 1, "" - , "etag", "versionId"); + final S3AFileStatus newFileStatus = new S3AFileStatus(1, 1, file, 1, "", + "etag", "versionId"); metadataStore.put(new PathMetadata(newFileStatus)); final S3GuardFsck s3GuardFsck = @@ -601,8 +605,8 @@ public void testIEtagMismatch() throws Exception { awaitFileStatus(guardedFs, file); // modify the file metadata so the etag will not match - final S3AFileStatus newFileStatus = new S3AFileStatus(1, 1, file, 1, "" - , "etag", "versionId"); + final S3AFileStatus newFileStatus = new S3AFileStatus(1, 1, file, 1, "", + "etag", "versionId"); metadataStore.put(new PathMetadata(newFileStatus)); final S3GuardFsck s3GuardFsck = From 60a25d38da26705486889617b95244cd89e1c0a7 Mon Sep 17 00:00:00 2001 From: Gabor Bota Date: Tue, 13 Aug 2019 12:51:58 +0200 Subject: [PATCH 08/16] some more comments Change-Id: I751b1520070836894b0667eff7861d0eb760a4a3 --- .../java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java | 2 +- .../hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java index 0a9f31a48e58f..cee0dbc9e502a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java @@ -86,7 +86,7 @@ public class S3GuardFsck { * * @param p the root path to start the traversal * @throws IOException - * @return + * @return a list of {@link ComparePair} */ public List compareS3RootToMs(Path p) throws IOException { final Path rootPath = rawFS.qualify(p); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java index d44ee5bcfb957..49ef3cfef63f4 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java @@ -35,8 +35,11 @@ public class S3GuardFsckViolationHandler { private static final Logger LOG = LoggerFactory.getLogger( S3GuardFsckViolationHandler.class); + // The rawFS and metadataStore are here to prepare when the ViolationHandlers + // will not just log, but fix the violations, so they will have access. private S3AFileSystem rawFs; private DynamoDBMetadataStore metadataStore; + private static String newLine = System.getProperty("line.separator"); public S3GuardFsckViolationHandler(S3AFileSystem fs, From 398c5d8f60ec9e12924481dac9c63bf2341c1e8f Mon Sep 17 00:00:00 2001 From: Gabor Bota Date: Wed, 4 Sep 2019 16:53:00 +0200 Subject: [PATCH 09/16] fix treewalk, add some tests, address Steve's comments. Tested in CLI and working. May need some more fine-tuning. Change-Id: I11df42693da9911738dbc031e74f418d487b2460 --- .../hadoop/fs/s3a/s3guard/S3GuardFsck.java | 125 +++++++++++------- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 10 +- .../fs/s3a/s3guard/ITestS3GuardFsck.java | 26 ++-- .../s3a/s3guard/ITestS3GuardToolDynamoDB.java | 15 +++ 4 files changed, 109 insertions(+), 67 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java index cee0dbc9e502a..78515d2ade594 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java @@ -20,8 +20,11 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.AWSBadRequestException; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; + +import com.google.common.base.Stopwatch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,6 +37,7 @@ import java.util.List; import java.util.Queue; import java.util.Set; +import java.util.concurrent.TimeUnit; import static java.util.stream.Collectors.toList; import static java.util.stream.Collectors.toSet; @@ -88,82 +92,91 @@ public class S3GuardFsck { * @throws IOException * @return a list of {@link ComparePair} */ - public List compareS3RootToMs(Path p) throws IOException { + public List compareS3ToMs(Path p) throws IOException { + Stopwatch stopwatch = Stopwatch.createStarted(); + int scannedItems = 0; + final Path rootPath = rawFS.qualify(p); - final S3AFileStatus root = - (S3AFileStatus) rawFS.getFileStatus(rootPath); + S3AFileStatus root = null; + try { + root = (S3AFileStatus) rawFS.getFileStatus(rootPath); + } catch (AWSBadRequestException e) { + throw new IOException(e.getMessage()); + } final List comparePairs = new ArrayList<>(); final Queue queue = new ArrayDeque<>(); queue.add(root); while (!queue.isEmpty()) { - // pop front node from the queue final S3AFileStatus currentDir = queue.poll(); + scannedItems++; - // Get a listing of that dir from s3 and add just the files. - // (Each directory will be added as a root.) - // Files should be casted to S3AFileStatus instead of plain FileStatus - // to get the VersionID and Etag. final Path currentDirPath = currentDir.getPath(); - - final FileStatus[] s3DirListing = rawFS.listStatus(currentDirPath); - final List children = - Arrays.asList(s3DirListing).stream() + List s3DirListing = Arrays.asList(rawFS.listStatus(currentDirPath)); + + // DIRECTORIES + // Check directory authoritativeness consistency + compareAuthoritativeDirectoryFlag(comparePairs, currentDirPath, s3DirListing); + // Add all descendant directory to the queue + s3DirListing.stream().filter(pm -> pm.isDirectory()) + .map(S3AFileStatus.class::cast) + .forEach(pm -> queue.add(pm)); + + // FILES + // check files for consistency + final List children = s3DirListing.stream() .filter(status -> !status.isDirectory()) .map(S3AFileStatus.class::cast).collect(toList()); - - // Compare the directory contents if the listing is authoritative - final DirListingMetadata msDirListing = - metadataStore.listChildren(currentDirPath); - if (msDirListing != null && msDirListing.isAuthoritative()) { - final ComparePair cP = - compareAuthDirListing(s3DirListing, msDirListing); - if (cP.containsViolation()) { - comparePairs.add(cP); - } - } - - // Compare directory and contents, but not the listing final List compareResult = compareS3DirToMs(currentDir, children).stream() .filter(comparePair -> comparePair.containsViolation()) .collect(toList()); comparePairs.addAll(compareResult); - - // Add each dir to queue - children.stream().filter(pm -> pm.isDirectory()) - .forEach(pm -> queue.add(pm)); + scannedItems += children.size(); } + stopwatch.stop(); // Create a handler and handle each violated pairs S3GuardFsckViolationHandler handler = new S3GuardFsckViolationHandler(rawFS, metadataStore); comparePairs.forEach(handler::handle); + LOG.info("Total scan time: {}s", stopwatch.elapsed(TimeUnit.SECONDS)); + LOG.info("Scanned entries: {}", scannedItems); + return comparePairs; } - private ComparePair compareAuthDirListing(FileStatus[] s3DirListing, - DirListingMetadata msDirListing) { - ComparePair cP = new ComparePair(s3DirListing, msDirListing); - - if (!msDirListing.isAuthoritative()) { - return cP; - } - - if (s3DirListing.length != msDirListing.numEntries()) { - cP.violations.add(Violation.AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH); - } else { - final Set msPaths = msDirListing.getListing().stream() - .map(pm -> pm.getFileStatus().getPath()).collect(toSet()); - final Set s3Paths = Arrays.stream(s3DirListing) - .map(pm -> pm.getPath()).collect(toSet()); - if (!s3Paths.equals(msPaths)) { + /** + * Compare the directory contents if the listing is authoritative + * @param comparePairs the list of compare pairs to add to if it contains a violation + * @param currentDirPath the current directory path + * @param s3DirListing the s3 directory listing to compare with + * @throws IOException + */ + private void compareAuthoritativeDirectoryFlag(List comparePairs, Path currentDirPath, + List s3DirListing) throws IOException { + final DirListingMetadata msDirListing = + metadataStore.listChildren(currentDirPath); + if (msDirListing != null && msDirListing.isAuthoritative()) { + ComparePair cP = new ComparePair(s3DirListing, msDirListing); + + if (s3DirListing.size() != msDirListing.numEntries()) { cP.violations.add(Violation.AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH); + } else { + final Set msPaths = msDirListing.getListing().stream() + .map(pm -> pm.getFileStatus().getPath()).collect(toSet()); + final Set s3Paths = s3DirListing.stream() + .map(pm -> pm.getPath()).collect(toSet()); + if (!s3Paths.equals(msPaths)) { + cP.violations.add(Violation.AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH); + } } - } - return cP; + if (cP.containsViolation()) { + comparePairs.add(cP); + } + } } protected List compareS3DirToMs(S3AFileStatus s3CurrentDir, @@ -199,7 +212,17 @@ protected ComparePair compareFileStatusToPathMetadata( S3AFileStatus s3FileStatus, PathMetadata msPathMetadata) throws IOException { final Path path = s3FileStatus.getPath(); - System.out.println("== Path: " + path); + + if(msPathMetadata != null) { + LOG.info("Path: {} - Length S3: {}, MS: {} - Etag S3: {}, MS: {}", + path, + s3FileStatus.getLen(), msPathMetadata.getFileStatus().getLen(), + s3FileStatus.getETag(), msPathMetadata.getFileStatus().getETag()); + } else { + LOG.info("Path: {} - Length S3: {} - Etag S3: {}, no record in MS.", + path, s3FileStatus.getLen(), s3FileStatus.getETag()); + } + ComparePair comparePair = new ComparePair(s3FileStatus, msPathMetadata); if (!path.equals(path(ROOT_PATH_STRING))) { @@ -217,7 +240,7 @@ protected ComparePair compareFileStatusToPathMetadata( } } } else { - LOG.info("Entry is in the root, so there's no parent"); + LOG.debug("Entry is in the root, so there's no parent"); } if (msPathMetadata == null) { @@ -288,8 +311,8 @@ public static class ComparePair { this.path = status.getPath(); } - ComparePair(FileStatus[] s3DirListing, DirListingMetadata msDirListing) { - this.s3DirListing = Arrays.asList(s3DirListing); + ComparePair(List s3DirListing, DirListingMetadata msDirListing) { + this.s3DirListing = s3DirListing; this.msDirListing = msDirListing; this.s3FileStatus = null; this.msPathMetadata = null; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java index 5da92f7e3c453..870d6ebbf8e61 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java @@ -1529,8 +1529,7 @@ public int run(String[] args, PrintStream out) throws initS3AFileSystem(s3Path); } catch (Exception e) { errorln("Failed to initialize S3AFileSystem from path: " + s3Path); - errorln(USAGE); - return ERROR; + throw e; } URI uri = toUri(s3Path); @@ -1557,7 +1556,12 @@ public int run(String[] args, PrintStream out) throws if (commandFormat.getOpt(CHECK_FLAG)) { // do the check S3GuardFsck s3GuardFsck = new S3GuardFsck(fs, ms); - s3GuardFsck.compareS3RootToMs(fs.qualify(root)); + try { + s3GuardFsck.compareS3ToMs(fs.qualify(root)); + } catch (IOException e) { + errorln("Error while running the check: compareS3ToMs"); + throw e; + } } else { errorln("No supported operation is selected."); errorln(USAGE); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java index a88ef0e8a883f..d5a5365d8c92b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java @@ -113,7 +113,7 @@ public void testIDetectNoMetadataEntry() throws Exception { new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3RootToMs(cwd); + s3GuardFsck.compareS3ToMs(cwd); assertEquals("Number of pairs should be two.", 2, comparePairs.size()); @@ -150,7 +150,7 @@ public void testIDetectNoParentEntry() throws Exception { new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3RootToMs(cwd); + s3GuardFsck.compareS3ToMs(cwd); assertEquals("Number of pairs should be two. The cwd (parent) and the " + "child.", 2, comparePairs.size()); @@ -200,7 +200,7 @@ public void testIDetectParentIsAFile() throws Exception { new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3RootToMs(cwd); + s3GuardFsck.compareS3ToMs(cwd); assertEquals("Number of pairs should be two. The cwd (parent) and the " + "child.", 2, comparePairs.size()); @@ -250,7 +250,7 @@ public void testIDetectParentTombstoned() throws Exception { new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3RootToMs(cwd); + s3GuardFsck.compareS3ToMs(cwd); assertEquals("Number of pairs should be two. The cwd (parent) and the " + "child.", 2, comparePairs.size()); @@ -289,7 +289,7 @@ public void testIDetectDirInS3FileInMs() throws Exception { new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3RootToMs(cwd); + s3GuardFsck.compareS3ToMs(cwd); assertEquals("Number of pair should be one. Only the cwd.", 1, comparePairs.size()); @@ -328,7 +328,7 @@ public void testIDetectFileInS3DirInMs() throws Exception { new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3RootToMs(cwd); + s3GuardFsck.compareS3ToMs(cwd); assertEquals("Number of pairs should be two", 2, comparePairs.size()); @@ -403,9 +403,9 @@ public void testIAuthoritativeDirectoryContentMismatch() throws Exception { awaitFileStatus(rawFS, fileIc2); final List pairsCorrect = - s3GuardFsck.compareS3RootToMs(cwdCorrect); + s3GuardFsck.compareS3ToMs(cwdCorrect); final List pairsIncorrect = - s3GuardFsck.compareS3RootToMs(cwdIncorrect); + s3GuardFsck.compareS3ToMs(cwdIncorrect); // check the parent that it does not contain LENGTH_MISMATCH final S3GuardFsck.ComparePair cwdPair = pairsCorrect.stream() @@ -457,7 +457,7 @@ public void testIDetectLengthMismatch() throws Exception { new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3RootToMs(cwd); + s3GuardFsck.compareS3ToMs(cwd); assertEquals("Number of pairs should be two. The cwd (parent) and the " + "child.", 2, comparePairs.size()); @@ -513,7 +513,7 @@ public void testIDetectModTimeMismatch() throws Exception { new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3RootToMs(cwd); + s3GuardFsck.compareS3ToMs(cwd); assertEquals("Number of pairs should be two. The cwd (parent) and the " + "child.", 2, comparePairs.size()); @@ -563,7 +563,7 @@ public void testIVersionIdMismatch() throws Exception { new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3RootToMs(cwd); + s3GuardFsck.compareS3ToMs(cwd); assertEquals("Number of pairs should be two. The cwd (parent) and the " + "child.", 2, comparePairs.size()); @@ -613,7 +613,7 @@ public void testIEtagMismatch() throws Exception { new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3RootToMs(cwd); + s3GuardFsck.compareS3ToMs(cwd); assertEquals("Number of pairs should be two. The cwd (parent) and the " + "child.", 2, comparePairs.size()); @@ -670,7 +670,7 @@ public void testINoEtag() throws Exception { new S3GuardFsck(rawFS, metadataStore); final List comparePairs = - s3GuardFsck.compareS3RootToMs(cwd); + s3GuardFsck.compareS3ToMs(cwd); assertEquals("Number of pairs should be 3. The cwd (parent) and the " + "2 children.", 3, comparePairs.size()); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java index 493c876f8e73c..f84b60af21273 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.s3guard; +import java.io.FileNotFoundException; import java.io.IOException; import java.util.HashMap; import java.util.List; @@ -302,4 +303,18 @@ public void testCLIFsckWithParam() throws Exception { assertEquals("The result should be success when fsck is running with " + "correct parameters.", SUCCESS, result); } + + @Test + public void testCLIFsckWithParamParentOfRoot() throws Exception { + intercept(IOException.class, "Invalid URI", + () -> run(S3GuardTool.Fsck.NAME, "-check", + "s3a://" + getFileSystem().getBucket() + "/..")); + } + + @Test + public void testCLIFsckFailInitializeFs() throws Exception { + intercept(FileNotFoundException.class, "does not exist", + () -> run(S3GuardTool.Fsck.NAME, "-check", + "s3a://this-bucket-does-not-exist-" + UUID.randomUUID())); + } } From 61bbe4faac1f4808479cddcfbb8ea6024ed13a2d Mon Sep 17 00:00:00 2001 From: Gabor Bota Date: Fri, 6 Sep 2019 15:32:13 +0200 Subject: [PATCH 10/16] fix wrong missing versionid handling Change-Id: I8330f9c562ab3e3335a2aea7a85446643ce4fa8c --- .../java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java index 78515d2ade594..9373f02a5c25d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java @@ -267,10 +267,11 @@ protected ComparePair compareFileStatusToPathMetadata( comparePair.violations.add(Violation.MOD_TIME_MISMATCH); } - if(msPathMetadata.getFileStatus().getVersionId() == null) { - LOG.debug("Missing versionIDs are skipped right now. A HEAD request is " + if(msPathMetadata.getFileStatus().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() != msFileStatus.getVersionId()) { + } else if(!s3FileStatus.getVersionId().equals(msFileStatus.getVersionId())) { comparePair.violations.add(Violation.VERSIONID_MISMATCH); } From 7f904bfa6a9f16c1f82de09c8a2c96df2fcc50ca Mon Sep 17 00:00:00 2001 From: Gabor Bota Date: Fri, 6 Sep 2019 16:13:50 +0200 Subject: [PATCH 11/16] check etag only for files, and not directories Change-Id: Iaff8875a7ca238639c105537c3268bfb212189e2 --- .../apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java index 9373f02a5c25d..eccfd8b182369 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java @@ -275,12 +275,14 @@ protected ComparePair compareFileStatusToPathMetadata( comparePair.violations.add(Violation.VERSIONID_MISMATCH); } - - if (msPathMetadata.getFileStatus().getETag() == null) { - comparePair.violations.add(Violation.NO_ETAG); - } else if (s3FileStatus.getETag() != null && - !s3FileStatus.getETag().equals(msFileStatus.getETag())) { - comparePair.violations.add(Violation.ETAG_MISMATCH); + // check etag only for files, and not directories + if (!s3FileStatus.isDirectory()) { + if (msPathMetadata.getFileStatus().getETag() == null) { + comparePair.violations.add(Violation.NO_ETAG); + } else if (s3FileStatus.getETag() != null && + !s3FileStatus.getETag().equals(msFileStatus.getETag())) { + comparePair.violations.add(Violation.ETAG_MISMATCH); + } } return comparePair; From d4e0c73bb9303c1fbbcb00712ba6f3b6f26fd518 Mon Sep 17 00:00:00 2001 From: Gabor Bota Date: Tue, 10 Sep 2019 16:20:39 +0200 Subject: [PATCH 12/16] fixed based on comments part1 - there will be another part soon Change-Id: Ife89007fdc028aa49abe0ed6441f95e08078688f --- .../hadoop/fs/s3a/s3guard/S3GuardFsck.java | 197 +++++++++++------- .../s3guard/S3GuardFsckViolationHandler.java | 49 +++-- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 7 +- .../fs/s3a/s3guard/ITestS3GuardFsck.java | 161 ++++---------- .../s3a/s3guard/ITestS3GuardToolDynamoDB.java | 3 +- 5 files changed, 196 insertions(+), 221 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java index eccfd8b182369..c9f9a184c33f7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java @@ -18,16 +18,7 @@ package org.apache.hadoop.fs.s3a.s3guard; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.s3a.AWSBadRequestException; -import org.apache.hadoop.fs.s3a.S3AFileStatus; -import org.apache.hadoop.fs.s3a.S3AFileSystem; - -import com.google.common.base.Stopwatch; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - +import java.io.FileNotFoundException; import java.io.IOException; import java.security.InvalidParameterException; import java.util.ArrayDeque; @@ -39,6 +30,16 @@ import java.util.Set; import java.util.concurrent.TimeUnit; +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import com.google.common.base.Stopwatch; + +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 static java.util.stream.Collectors.toList; import static java.util.stream.Collectors.toSet; @@ -56,28 +57,27 @@ public class S3GuardFsck { private static final Logger LOG = LoggerFactory.getLogger(S3GuardFsck.class); public static final String ROOT_PATH_STRING = "/"; - private S3AFileSystem rawFS; - private DynamoDBMetadataStore metadataStore; + private final S3AFileSystem rawFS; + private final DynamoDBMetadataStore metadataStore; /** * Creates an S3GuardFsck. * @param fs the filesystem to compare to * @param ms metadatastore the metadatastore to compare with (dynamo) */ - S3GuardFsck(S3AFileSystem fs, MetadataStore ms) + public S3GuardFsck(S3AFileSystem fs, MetadataStore ms) throws InvalidParameterException { this.rawFS = fs; if (ms == null) { - throw new InvalidParameterException("S3AFileSystem should be guarded by" - + " a " + DynamoDBMetadataStore.class.getCanonicalName()); + throw new InvalidParameterException("S3A Bucket " + fs.getBucket() + + " should be guarded by a " + + DynamoDBMetadataStore.class.getCanonicalName()); } this.metadataStore = (DynamoDBMetadataStore) ms; - if (rawFS.hasMetadataStore()) { - throw new InvalidParameterException("Raw fs should not have a " - + "metadatastore."); - } + Preconditions.checkArgument(!rawFS.hasMetadataStore(), + "Raw fs should not have a metadatastore."); } /** @@ -89,50 +89,53 @@ public class S3GuardFsck { * The violations are listed in Enums: {@link Violation} * * @param p the root path to start the traversal - * @throws IOException * @return a list of {@link ComparePair} + * @throws IOException */ public List compareS3ToMs(Path p) throws IOException { Stopwatch stopwatch = Stopwatch.createStarted(); int scannedItems = 0; final Path rootPath = rawFS.qualify(p); - S3AFileStatus root = null; - try { - root = (S3AFileStatus) rawFS.getFileStatus(rootPath); - } catch (AWSBadRequestException e) { - throw new IOException(e.getMessage()); - } + S3AFileStatus root = (S3AFileStatus) rawFS.getFileStatus(rootPath); final List comparePairs = new ArrayList<>(); final Queue queue = new ArrayDeque<>(); queue.add(root); while (!queue.isEmpty()) { final S3AFileStatus currentDir = queue.poll(); - scannedItems++; + final Path currentDirPath = currentDir.getPath(); - List s3DirListing = Arrays.asList(rawFS.listStatus(currentDirPath)); - - // DIRECTORIES - // Check directory authoritativeness consistency - compareAuthoritativeDirectoryFlag(comparePairs, currentDirPath, s3DirListing); - // Add all descendant directory to the queue - s3DirListing.stream().filter(pm -> pm.isDirectory()) - .map(S3AFileStatus.class::cast) - .forEach(pm -> queue.add(pm)); - - // FILES - // check files for consistency - final List children = s3DirListing.stream() - .filter(status -> !status.isDirectory()) - .map(S3AFileStatus.class::cast).collect(toList()); - final List compareResult = - compareS3DirToMs(currentDir, children).stream() - .filter(comparePair -> comparePair.containsViolation()) - .collect(toList()); - comparePairs.addAll(compareResult); - scannedItems += children.size(); + try { + List s3DirListing = Arrays.asList( + rawFS.listStatus(currentDirPath)); + + // Check authoritative directory flag. + compareAuthoritativeDirectoryFlag(comparePairs, currentDirPath, + s3DirListing); + // Add all descendant directory to the queue + s3DirListing.stream().filter(pm -> pm.isDirectory()) + .map(S3AFileStatus.class::cast) + .forEach(pm -> queue.add(pm)); + + // Check file and directory metadata for consistency. + final List children = s3DirListing.stream() + .filter(status -> !status.isDirectory()) + .map(S3AFileStatus.class::cast).collect(toList()); + final List compareResult = + compareS3DirContentToMs(currentDir, children); + comparePairs.addAll(compareResult); + + // Increase the scanned file size. + // One for the directory, one for the children. + scannedItems++; + scannedItems += children.size(); + } catch (FileNotFoundException e) { + LOG.error("The path has been deleted since it was queued: " + + currentDirPath, e); + } + } stopwatch.stop(); @@ -148,14 +151,16 @@ public List compareS3ToMs(Path p) throws IOException { } /** - * Compare the directory contents if the listing is authoritative - * @param comparePairs the list of compare pairs to add to if it contains a violation + * Compare the directory contents if the listing is authoritative. + * + * @param comparePairs the list of compare pairs to add to + * if it contains a violation * @param currentDirPath the current directory path * @param s3DirListing the s3 directory listing to compare with * @throws IOException */ - private void compareAuthoritativeDirectoryFlag(List comparePairs, Path currentDirPath, - List s3DirListing) throws IOException { + private void compareAuthoritativeDirectoryFlag(List comparePairs, + Path currentDirPath, List s3DirListing) throws IOException { final DirListingMetadata msDirListing = metadataStore.listChildren(currentDirPath); if (msDirListing != null && msDirListing.isAuthoritative()) { @@ -179,7 +184,16 @@ private void compareAuthoritativeDirectoryFlag(List comparePairs, P } } - protected List compareS3DirToMs(S3AFileStatus s3CurrentDir, + /** + * Compares S3 directory content to the metadata store. + * + * @param s3CurrentDir file status of the current directory + * @param children the contents of the directory + * @return the compare pairs with violations of consistency + * @throws IOException + */ + protected List compareS3DirContentToMs( + S3AFileStatus s3CurrentDir, List children) throws IOException { final Path path = s3CurrentDir.getPath(); final PathMetadata pathMetadata = metadataStore.get(path); @@ -201,23 +215,36 @@ protected List compareS3DirToMs(S3AFileStatus s3CurrentDir, violationComparePairs.add(comparePair); } } catch (Exception e) { - e.printStackTrace(); + LOG.error(e.getMessage(), e); } }); return violationComparePairs; } + /** + * Compares a {@link S3AFileStatus} from S3 to a {@link PathMetadata} + * from the metadata store. Finds violated invariants and consistency + * issues. + * + * @param s3FileStatus the file status from S3 + * @param msPathMetadata the path metadata from metadatastore + * @return {@link ComparePair} with the found issues + * @throws IOException + */ protected ComparePair compareFileStatusToPathMetadata( S3AFileStatus s3FileStatus, PathMetadata msPathMetadata) throws IOException { final Path path = s3FileStatus.getPath(); - if(msPathMetadata != null) { - LOG.info("Path: {} - Length S3: {}, MS: {} - Etag S3: {}, MS: {}", - path, - s3FileStatus.getLen(), msPathMetadata.getFileStatus().getLen(), - s3FileStatus.getETag(), msPathMetadata.getFileStatus().getETag()); + if (msPathMetadata != null) { + LOG.info("Path: {} - Length S3: {}, MS: {} " + + "- Etag S3: {}, MS: {} " + + "- VersionId: S3: {}, MS: {}", + path, + s3FileStatus.getLen(), msPathMetadata.getFileStatus().getLen(), + s3FileStatus.getETag(), msPathMetadata.getFileStatus().getETag(), + s3FileStatus.getVersionId(), msPathMetadata.getFileStatus().getVersionId()); } else { LOG.info("Path: {} - Length S3: {} - Etag S3: {}, no record in MS.", path, s3FileStatus.getLen(), s3FileStatus.getETag()); @@ -240,7 +267,7 @@ protected ComparePair compareFileStatusToPathMetadata( } } } else { - LOG.debug("Entry is in the root, so there's no parent"); + LOG.debug("Entry is in the root directory, so there's no parent"); } if (msPathMetadata == null) { @@ -304,7 +331,7 @@ public static class ComparePair { private final Path path; - private Set violations = new HashSet<>(); + private final Set violations = new HashSet<>(); ComparePair(S3AFileStatus status, PathMetadata pm) { this.s3FileStatus = status; @@ -364,45 +391,65 @@ public Path getPath() { * where 0 is the most severe and 2 is the least severe. */ public enum Violation { - // No entry in metadatastore + /** + * No entry in metadatastore. + */ NO_METADATA_ENTRY(1, S3GuardFsckViolationHandler.NoMetadataEntry.class), - // A file or directory entry does not have a parent entry - excluding - // files and directories in the root. + /** + * A file or directory entry does not have a parent entry - excluding + * files and directories in the root. + */ NO_PARENT_ENTRY(0, S3GuardFsckViolationHandler.NoParentEntry.class), - // An entry’s parent is a file + /** + * An entry’s parent is a file. + */ PARENT_IS_A_FILE(0, S3GuardFsckViolationHandler.ParentIsAFile.class), - // A file exists under a path for which there is - // a tombstone entry in the MS + /** + * A file exists under a path for which there is a + * tombstone entry in the MS. + */ PARENT_TOMBSTONED(0, S3GuardFsckViolationHandler.ParentTombstoned.class), - // A directory in S3 is a file entry in the MS + /** + * A directory in S3 is a file entry in the MS. + */ DIR_IN_S3_FILE_IN_MS(0, S3GuardFsckViolationHandler.DirInS3FileInMs.class), - // A file in S3 is a directory in the MS + /** + * A file in S3 is a directory in the MS. + */ FILE_IN_S3_DIR_IN_MS(0, S3GuardFsckViolationHandler.FileInS3DirInMs.class), AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH(1, S3GuardFsckViolationHandler.AuthDirContentMismatch.class), - // Attribute mismatch + /** + * Attribute mismatch. + */ LENGTH_MISMATCH(0, S3GuardFsckViolationHandler.LengthMismatch.class), MOD_TIME_MISMATCH(2, S3GuardFsckViolationHandler.ModTimeMismatch.class), - // If there's a versionID the mismatch is severe + /** + * If there's a versionID the mismatch is severe. + */ VERSIONID_MISMATCH(0, S3GuardFsckViolationHandler.VersionIdMismatch.class), - // If there's an etag the mismatch is severe + /** + * If there's an etag the mismatch is severe. + */ ETAG_MISMATCH(0, S3GuardFsckViolationHandler.EtagMismatch.class), - // Don't worry too much if we don't have an etag + /** + * Don't worry too much if we don't have an etag. + */ NO_ETAG(2, S3GuardFsckViolationHandler.NoEtag.class); - private int severity; - private Class handler; + private final int severity; + private final Class handler; Violation(int s, Class h) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java index 49ef3cfef63f4..d30fa016cbe67 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java @@ -18,16 +18,17 @@ package org.apache.hadoop.fs.s3a.s3guard; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.s3a.S3AFileStatus; -import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.lang.reflect.InvocationTargetException; import java.util.Arrays; import java.util.List; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.S3AFileSystem; + /** * Violation handler for the S3Guard's fsck. */ @@ -37,8 +38,8 @@ public class S3GuardFsckViolationHandler { // The rawFS and metadataStore are here to prepare when the ViolationHandlers // will not just log, but fix the violations, so they will have access. - private S3AFileSystem rawFs; - private DynamoDBMetadataStore metadataStore; + private final S3AFileSystem rawFs; + private final DynamoDBMetadataStore metadataStore; private static String newLine = System.getProperty("line.separator"); @@ -51,7 +52,7 @@ public S3GuardFsckViolationHandler(S3AFileSystem fs, public void handle(S3GuardFsck.ComparePair comparePair) { if (!comparePair.containsViolation()) { - LOG.debug("There is no violation in the compare pair: " + toString()); + LOG.debug("There is no violation in the compare pair: {}", comparePair); return; } @@ -59,7 +60,21 @@ public void handle(S3GuardFsck.ComparePair comparePair) { sB.append(newLine) .append("On path: ").append(comparePair.getPath()).append(newLine); - // Create a new instance of the handler and use it. + handleComparePair(comparePair, sB); + + LOG.error(sB.toString()); + } + + /** + * Create a new instance of the violation handler for all the violations + * found in the compare pair and use it. + * + * @param comparePair the compare pair with violations + * @param sB StringBuilder to append error strings from violations. + */ + protected static void handleComparePair(S3GuardFsck.ComparePair comparePair, + StringBuilder sB) { + for (S3GuardFsck.Violation violation : comparePair.getViolations()) { try { ViolationHandler handler = violation.getHandler() @@ -76,7 +91,6 @@ public void handle(S3GuardFsck.ComparePair comparePair) { } sB.append(newLine); } - LOG.error(sB.toString()); } /** @@ -102,7 +116,7 @@ public ViolationHandler(S3GuardFsck.ComparePair comparePair) { msDirListing = comparePair.getMsDirListing(); } - abstract String getError(); + public abstract String getError(); public PathMetadata getPathMetadata() { return pathMetadata; @@ -166,7 +180,7 @@ public ParentIsAFile(S3GuardFsck.ComparePair comparePair) { @Override public String getError() { - return "An entry’s parent is a file"; + return "The entry's parent in the metastore database is a file."; } } @@ -181,7 +195,8 @@ public ParentTombstoned(S3GuardFsck.ComparePair comparePair) { @Override public String getError() { - return "The entry's parent tombstoned"; + return "The entry in the metastore database has a parent entry " + + "which is a tombstone marker"; } } @@ -244,7 +259,7 @@ public LengthMismatch(S3GuardFsck.ComparePair comparePair) { } @Override public String getError() { - return String.format("getLen mismatch - s3: %s, ms: %s", + return String.format("File length mismatch - s3: %s, ms: %s", getS3FileStatus().getLen(), getMsFileStatus().getLen()); } } @@ -260,7 +275,7 @@ public ModTimeMismatch(S3GuardFsck.ComparePair comparePair) { @Override public String getError() { - return String.format("getModificationTime mismatch - s3: %s, ms: %s", + return String.format("File timestamp mismatch - s3: %s, ms: %s", getS3FileStatus().getModificationTime(), getMsFileStatus().getModificationTime()); } @@ -293,7 +308,7 @@ public EtagMismatch(S3GuardFsck.ComparePair comparePair) { @Override public String getError() { - return String.format("getETag mismatch - s3: %s, ms: %s", + return String.format("Etag mismatch - s3: %s, ms: %s", getS3FileStatus().getETag(), getMsFileStatus().getETag()); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java index 870d6ebbf8e61..223c9d4af8d7c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java @@ -1487,7 +1487,7 @@ private void vprintln(PrintStream out, String format, Object... } /** - * Prune metadata that has not been modified recently. + * Fsck - check for consistency between S3 and the metadatastore. */ static class Fsck extends S3GuardTool { public static final String CHECK_FLAG = "check"; @@ -1495,11 +1495,11 @@ static class Fsck extends S3GuardTool { public static final String NAME = "fsck"; public static final String PURPOSE = "Compares S3 with MetadataStore, and " + "returns a failure status if any rules or invariants are violated. " - + "Only works with DynamoDbMetadataStore."; + + "Only works with DynamoDB metadata stores."; private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" + "\t" + PURPOSE + "\n\n" + "Common options:\n" + - " " + CHECK_FLAG + " Check the metadata store for errors, but do " + " -" + CHECK_FLAG + " Check the metadata store for errors, but do " + "not fix any issues.\n"; Fsck(Configuration conf) { @@ -1559,7 +1559,6 @@ public int run(String[] args, PrintStream out) throws try { s3GuardFsck.compareS3ToMs(fs.qualify(root)); } catch (IOException e) { - errorln("Error while running the check: compareS3ToMs"); throw e; } } else { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java index d5a5365d8c92b..6e11952516331 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java @@ -23,25 +23,26 @@ import java.util.List; import java.util.UUID; -import org.apache.hadoop.io.IOUtils; +import org.assertj.core.api.Assertions; import org.junit.Before; import org.junit.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.assertj.core.api.Assertions; +import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH; +import static org.junit.Assume.assumeTrue; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE; import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; import static org.apache.hadoop.fs.s3a.S3ATestUtils.awaitFileStatus; import static org.apache.hadoop.fs.s3a.S3ATestUtils.metadataStorePersistsAuthoritativeBit; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; -import static org.junit.Assume.assumeTrue; /** * Integration tests for the S3Guard Fsck against a dyamodb backed metadata @@ -59,9 +60,9 @@ public void setup() throws Exception { super.setup(); S3AFileSystem fs = getFileSystem(); // These test will fail if no ms - assertTrue("FS needs to have a metadatastore.", + assumeTrue("FS needs to have a metadatastore.", fs.hasMetadataStore()); - assertTrue("Metadatastore should persist authoritative bit", + assumeTrue("Metadatastore should persist authoritative bit", metadataStorePersistsAuthoritativeBit(fs.getMetadataStore())); guardedFs = fs; @@ -93,9 +94,8 @@ private S3AFileSystem createUnguardedFS() throws Exception { URI uri = testFS.getUri(); removeBaseAndBucketOverrides(uri.getHost(), config, - S3_METADATA_STORE_IMPL); - removeBaseAndBucketOverrides(uri.getHost(), config, - METADATASTORE_AUTHORITATIVE); + S3_METADATA_STORE_IMPL, METADATASTORE_AUTHORITATIVE, + AUTHORITATIVE_PATH); S3AFileSystem fs2 = new S3AFileSystem(); fs2.initialize(uri, config); return fs2; @@ -115,8 +115,8 @@ public void testIDetectNoMetadataEntry() throws Exception { final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); - assertEquals("Number of pairs should be two.", 2, - comparePairs.size()); + assertComparePairsSize(comparePairs, 2); + final S3GuardFsck.ComparePair pair = comparePairs.get(0); assertTrue("The pair must contain a violation.", pair.containsViolation()); assertEquals("The pair must contain only one violation", 1, @@ -152,8 +152,7 @@ public void testIDetectNoParentEntry() throws Exception { final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); - assertEquals("Number of pairs should be two. The cwd (parent) and the " - + "child.", 2, comparePairs.size()); + assertComparePairsSize(comparePairs, 2); // check the parent that it does not exist final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() @@ -202,8 +201,7 @@ public void testIDetectParentIsAFile() throws Exception { final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); - assertEquals("Number of pairs should be two. The cwd (parent) and the " - + "child.", 2, comparePairs.size()); + assertComparePairsSize(comparePairs, 2); // check the parent that it does not exist final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() @@ -252,8 +250,7 @@ public void testIDetectParentTombstoned() throws Exception { final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); - assertEquals("Number of pairs should be two. The cwd (parent) and the " - + "child.", 2, comparePairs.size()); + assertComparePairsSize(comparePairs, 1); // check the child that the parent is tombstoned final S3GuardFsck.ComparePair childPair = comparePairs.stream() @@ -291,8 +288,7 @@ public void testIDetectDirInS3FileInMs() throws Exception { final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); - assertEquals("Number of pair should be one. Only the cwd.", 1, - comparePairs.size()); + assertComparePairsSize(comparePairs, 1); // check the child that the dir in s3 is a file in the ms final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() @@ -330,17 +326,7 @@ public void testIDetectFileInS3DirInMs() throws Exception { final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); - assertEquals("Number of pairs should be two", 2, - comparePairs.size()); - - // check the child that the dir in s3 is a file in the ms - final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() - .filter(p -> p.getPath().equals(cwd)) - .findFirst().get(); - assertNotNull("The pair should not be null.", cwdPair); - Assertions.assertThat(cwdPair.getViolations()) - .describedAs("Violations in the cwdPair") - .doesNotContain(S3GuardFsck.Violation.FILE_IN_S3_DIR_IN_MS); + assertComparePairsSize(comparePairs, 1); // check the child that the dir in s3 is a file in the ms final S3GuardFsck.ComparePair filePair = comparePairs.stream() @@ -407,16 +393,11 @@ public void testIAuthoritativeDirectoryContentMismatch() throws Exception { final List pairsIncorrect = s3GuardFsck.compareS3ToMs(cwdIncorrect); - // check the parent that it does not contain LENGTH_MISMATCH - final S3GuardFsck.ComparePair cwdPair = pairsCorrect.stream() - .filter(p -> p.getPath().equals(cwdCorrect)) - .findFirst().get(); - assertNotNull("The pair should not be null.", cwdPair); - Assertions.assertThat(cwdPair.getViolations()) - .describedAs("Violations in the cwdPair") - .doesNotContain(S3GuardFsck.Violation.AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH); + // Assert that the correct dir does not contain the violation. + assertTrue(pairsCorrect.stream() + .noneMatch(p -> p.getPath().equals(cwdCorrect))); - // check the child that there's a LENGTH_MISMATCH + // Assert that the incorrect listing contains the violation. final S3GuardFsck.ComparePair childPair = pairsIncorrect.stream() .filter(p -> p.getPath().equals(cwdIncorrect)) .findFirst().get(); @@ -459,19 +440,13 @@ public void testIDetectLengthMismatch() throws Exception { final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); - assertEquals("Number of pairs should be two. The cwd (parent) and the " - + "child.", 2, comparePairs.size()); + assertComparePairsSize(comparePairs, 1); - // check the parent that it does not contain LENGTH_MISMATCH - final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() - .filter(p -> p.getPath().equals(cwd)) - .findFirst().get(); - assertNotNull("The pair should not be null.", cwdPair); - Assertions.assertThat(cwdPair.getViolations()) - .describedAs("Violations in the cwdPair") - .doesNotContain(S3GuardFsck.Violation.LENGTH_MISMATCH); + // Assert that the correct dir does not contain the violation. + assertTrue(comparePairs.stream() + .noneMatch(p -> p.getPath().equals(cwd))); - // check the child that there's a LENGTH_MISMATCH + // Assert that the incorrect file meta contains the violation. final S3GuardFsck.ComparePair childPair = comparePairs.stream() .filter(p -> p.getPath().equals(file)) .findFirst().get(); @@ -515,19 +490,13 @@ public void testIDetectModTimeMismatch() throws Exception { final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); - assertEquals("Number of pairs should be two. The cwd (parent) and the " - + "child.", 2, comparePairs.size()); + assertComparePairsSize(comparePairs, 1); - // check the parent that it does not contain MOD_TIME_MISMATCH - final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() - .filter(p -> p.getPath().equals(cwd)) - .findFirst().get(); - assertNotNull("The pair should not be null.", cwdPair); - Assertions.assertThat(cwdPair.getViolations()) - .describedAs("Violations in the cwdPair") - .doesNotContain(S3GuardFsck.Violation.MOD_TIME_MISMATCH); + // Assert that the correct dir does not contain the violation. + assertTrue(comparePairs.stream() + .noneMatch(p -> p.getPath().equals(cwd))); - // check the child that there's a MOD_TIME_MISMATCH + // check the file meta that there's a violation. final S3GuardFsck.ComparePair childPair = comparePairs.stream() .filter(p -> p.getPath().equals(file)) .findFirst().get(); @@ -545,56 +514,6 @@ public void testIDetectModTimeMismatch() throws Exception { } } - @Test - public void testIVersionIdMismatch() throws Exception { - final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); - final Path file = new Path(cwd, "file"); - try { - // create a file with guarded fs - touch(guardedFs, file); - awaitFileStatus(guardedFs, file); - - // modify the file metadata so the versionId will not match - final S3AFileStatus newFileStatus = new S3AFileStatus(1, 1, file, 1, "", - "etag", "versionId"); - metadataStore.put(new PathMetadata(newFileStatus)); - - final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore); - - final List comparePairs = - s3GuardFsck.compareS3ToMs(cwd); - - assertEquals("Number of pairs should be two. The cwd (parent) and the " - + "child.", 2, comparePairs.size()); - - // check the parent that it does not contain BLOCKSIZE_MISMATCH - final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() - .filter(p -> p.getPath().equals(cwd)) - .findFirst().get(); - assertNotNull("The pair should not be null.", cwdPair); - Assertions.assertThat(cwdPair.getViolations()) - .describedAs("Violations in the cwdPair") - .doesNotContain(S3GuardFsck.Violation.VERSIONID_MISMATCH); - - // check the child that there's a BLOCKSIZE_MISMATCH - final S3GuardFsck.ComparePair childPair = comparePairs.stream() - .filter(p -> p.getPath().equals(file)) - .findFirst().get(); - assertNotNull("The pair should not be null.", childPair); - assertTrue("The childPair must contain a violation.", - childPair.containsViolation()); - Assertions.assertThat(childPair.getViolations()) - .describedAs("Violations in the childPair") - .contains(S3GuardFsck.Violation.VERSIONID_MISMATCH); - } finally { - // delete the working directory with all of its contents - rawFS.delete(cwd, true); - metadataStore.forgetMetadata(file); - metadataStore.forgetMetadata(cwd); - } - } - @Test public void testIEtagMismatch() throws Exception { final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); @@ -615,17 +534,7 @@ public void testIEtagMismatch() throws Exception { final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); - assertEquals("Number of pairs should be two. The cwd (parent) and the " - + "child.", 2, comparePairs.size()); - - // check the parent that it does not contain BLOCKSIZE_MISMATCH - final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() - .filter(p -> p.getPath().equals(cwd)) - .findFirst().get(); - assertNotNull("The pair should not be null.", cwdPair); - Assertions.assertThat(cwdPair.getViolations()) - .describedAs("Violations in the cwdPair") - .doesNotContain(S3GuardFsck.Violation.ETAG_MISMATCH); + assertComparePairsSize(comparePairs, 1); // check the child that there's a BLOCKSIZE_MISMATCH final S3GuardFsck.ComparePair childPair = comparePairs.stream() @@ -672,8 +581,7 @@ public void testINoEtag() throws Exception { final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); - assertEquals("Number of pairs should be 3. The cwd (parent) and the " - + "2 children.", 3, comparePairs.size()); + assertComparePairsSize(comparePairs, 2); // check file 1 that there's NO_ETAG final S3GuardFsck.ComparePair file1Pair = comparePairs.stream() @@ -703,5 +611,10 @@ public void testINoEtag() throws Exception { } } - + protected void assertComparePairsSize( + List comparePairs, int num) { + Assertions.assertThat(comparePairs.size()) + .describedAs("Number of compare pairs") + .isEqualTo(num); + } } \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java index f84b60af21273..6132612ac6568 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java @@ -33,7 +33,7 @@ import com.amazonaws.services.dynamodbv2.model.ResourceInUseException; import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException; import com.amazonaws.services.dynamodbv2.model.Tag; -import org.apache.hadoop.util.ExitUtil; + import org.junit.Assert; import org.junit.Assume; import org.junit.AssumptionViolatedException; @@ -44,6 +44,7 @@ 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.util.ExitUtil; import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_REGION_KEY; import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_NAME_KEY; From 637816599633f48748eb18384694e18e9504f391 Mon Sep 17 00:00:00 2001 From: Gabor Bota Date: Tue, 10 Sep 2019 17:23:52 +0200 Subject: [PATCH 13/16] ITestS3GuardFsck refactor Change-Id: I2ce69d66e348c4c0aded9bc8cf273e0c3a44f580 --- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 9 +- .../fs/s3a/s3guard/ITestS3GuardFsck.java | 393 ++++++------------ 2 files changed, 132 insertions(+), 270 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java index 223c9d4af8d7c..25a0cb04bb4a1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java @@ -1523,6 +1523,7 @@ public int run(String[] args, PrintStream out) throws out.println(USAGE); throw invalidArgs("no arguments"); } + int exitValue = EXIT_SUCCESS; String s3Path = paths.get(0); try { @@ -1557,7 +1558,11 @@ public int run(String[] args, PrintStream out) throws // do the check S3GuardFsck s3GuardFsck = new S3GuardFsck(fs, ms); try { - s3GuardFsck.compareS3ToMs(fs.qualify(root)); + final List comparePairs + = s3GuardFsck.compareS3ToMs(fs.qualify(root)); + if (comparePairs.size() > 0) { + exitValue = EXIT_FAIL; + } } catch (IOException e) { throw e; } @@ -1568,7 +1573,7 @@ public int run(String[] args, PrintStream out) throws } out.flush(); - return SUCCESS; + return exitValue; } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java index 6e11952516331..79ff34f2a3afb 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.s3a.s3guard; +import java.io.IOException; import java.net.URI; import java.util.List; import java.util.UUID; @@ -29,6 +30,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; @@ -51,7 +53,7 @@ public class ITestS3GuardFsck extends AbstractS3ATestBase { private S3AFileSystem guardedFs; - private S3AFileSystem rawFS; + private S3AFileSystem rawFs; private MetadataStore metadataStore; @@ -69,9 +71,9 @@ public void setup() throws Exception { metadataStore = fs.getMetadataStore(); // create raw fs without s3guard - rawFS = createUnguardedFS(); - assertFalse("Raw FS still has S3Guard " + rawFS, - rawFS.hasMetadataStore()); + rawFs = createUnguardedFS(); + assertFalse("Raw FS still has S3Guard " + rawFs, + rawFs.hasMetadataStore()); } @Override @@ -79,7 +81,7 @@ public void teardown() throws Exception { if (guardedFs != null) { IOUtils.cleanupWithLogger(LOG, guardedFs); } - IOUtils.cleanupWithLogger(LOG, rawFS); + IOUtils.cleanupWithLogger(LOG, rawFs); super.teardown(); } @@ -106,31 +108,19 @@ public void testIDetectNoMetadataEntry() throws Exception { final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); final Path file = new Path(cwd, "file"); try { - touch(rawFS, file); - awaitFileStatus(rawFS, file); - - final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore); + touchRawAndWaitRaw(file); + final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore); final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); assertComparePairsSize(comparePairs, 2); - final S3GuardFsck.ComparePair pair = comparePairs.get(0); - assertTrue("The pair must contain a violation.", pair.containsViolation()); - assertEquals("The pair must contain only one violation", 1, - pair.getViolations().size()); - - final S3GuardFsck.Violation violation = - pair.getViolations().iterator().next(); - assertEquals("The violation should be that there is no violation entry.", - violation, S3GuardFsck.Violation.NO_METADATA_ENTRY); + checkForViolationInPairs(file, comparePairs, + S3GuardFsck.Violation.NO_METADATA_ENTRY); } finally { // delete the working directory with all of its contents - rawFS.delete(cwd, true); - metadataStore.forgetMetadata(file); - metadataStore.forgetMetadata(cwd); + cleanup(file, cwd); } } @@ -139,45 +129,23 @@ public void testIDetectNoParentEntry() throws Exception { final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); final Path file = new Path(cwd, "file"); try { - // create a file with guarded fs - touch(guardedFs, file); - awaitFileStatus(guardedFs, file); - + touchGuardedAndWaitRaw(file); // delete the parent from the MS metadataStore.forgetMetadata(cwd); - final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore); - + final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore); final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); assertComparePairsSize(comparePairs, 2); - // check the parent that it does not exist - final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() - .filter(p -> p.getPath().equals(cwd)) - .findFirst().get(); - assertNotNull("The pair should not be null.", cwdPair); - assertTrue("The cwdPair must contain a violation.", cwdPair.containsViolation()); - Assertions.assertThat(cwdPair.getViolations()) - .describedAs("Violations in the cwdPair") - .contains(S3GuardFsck.Violation.NO_METADATA_ENTRY); - + checkForViolationInPairs(cwd, comparePairs, + S3GuardFsck.Violation.NO_METADATA_ENTRY); // check the child that there's no parent entry. - final S3GuardFsck.ComparePair childPair = comparePairs.stream() - .filter(p -> p.getPath().equals(file)) - .findFirst().get(); - assertNotNull("The pair should not be null.", childPair); - assertTrue("The childPair must contain a violation.", childPair.containsViolation()); - Assertions.assertThat(childPair.getViolations()) - .describedAs("Violations in the childPair") - .contains(S3GuardFsck.Violation.NO_PARENT_ENTRY); + checkForViolationInPairs(file, comparePairs, + S3GuardFsck.Violation.NO_PARENT_ENTRY); } finally { - // delete the working directory with all of its contents - rawFS.delete(cwd, true); - metadataStore.forgetMetadata(file); - metadataStore.forgetMetadata(cwd); + cleanup(file, cwd); } } @@ -186,47 +154,25 @@ public void testIDetectParentIsAFile() throws Exception { final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); final Path file = new Path(cwd, "file"); try { - // create a file with guarded fs - touch(guardedFs, file); - awaitFileStatus(guardedFs, file); - + touchGuardedAndWaitRaw(file); // modify the cwd metadata and set that it's not a directory final S3AFileStatus newParentFile = MetadataStoreTestBase .basicFileStatus(cwd, 1, false, 1); metadataStore.put(new PathMetadata(newParentFile)); - final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore); - + final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore); final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); assertComparePairsSize(comparePairs, 2); - // check the parent that it does not exist - final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() - .filter(p -> p.getPath().equals(cwd)) - .findFirst().get(); - assertNotNull("The pair should not be null.", cwdPair); - assertTrue("The cwdPair must contain a violation.", cwdPair.containsViolation()); - Assertions.assertThat(cwdPair.getViolations()) - .describedAs("Violations in the cwdPair") - .contains(S3GuardFsck.Violation.DIR_IN_S3_FILE_IN_MS); - + checkForViolationInPairs(cwd, comparePairs, + S3GuardFsck.Violation.DIR_IN_S3_FILE_IN_MS); // check the child that the parent is a file. - final S3GuardFsck.ComparePair childPair = comparePairs.stream() - .filter(p -> p.getPath().equals(file)) - .findFirst().get(); - assertNotNull("The pair should not be null.", childPair); - assertTrue("The childPair must contain a violation.", childPair.containsViolation()); - Assertions.assertThat(childPair.getViolations()) - .describedAs("Violations in the childPair") - .contains(S3GuardFsck.Violation.PARENT_IS_A_FILE); + checkForViolationInPairs(file, comparePairs, + S3GuardFsck.Violation.PARENT_IS_A_FILE); } finally { - // delete the working directory with all of its contents - rawFS.delete(cwd, true); - metadataStore.forgetMetadata(file); - metadataStore.forgetMetadata(cwd); + cleanup(file, cwd); } } @@ -235,37 +181,22 @@ public void testIDetectParentTombstoned() throws Exception { final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); final Path file = new Path(cwd, "file"); try { - // create a file with guarded fs - touch(guardedFs, file); - awaitFileStatus(guardedFs, file); - + touchGuardedAndWaitRaw(file); // modify the parent metadata and set that it's not a directory final PathMetadata cwdPmd = metadataStore.get(cwd); cwdPmd.setIsDeleted(true); metadataStore.put(cwdPmd); - final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore); - + final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore); final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); assertComparePairsSize(comparePairs, 1); - // check the child that the parent is tombstoned - final S3GuardFsck.ComparePair childPair = comparePairs.stream() - .filter(p -> p.getPath().equals(file)) - .findFirst().get(); - assertNotNull("The pair should not be null.", childPair); - assertTrue("The childPair must contain a violation.", childPair.containsViolation()); - Assertions.assertThat(childPair.getViolations()) - .describedAs("Violations in the childPair") - .contains(S3GuardFsck.Violation.PARENT_TOMBSTONED); + checkForViolationInPairs(file, comparePairs, + S3GuardFsck.Violation.PARENT_TOMBSTONED); } finally { - // delete the working directory with all of its contents - rawFS.delete(cwd, true); - metadataStore.forgetMetadata(file); - metadataStore.forgetMetadata(cwd); + cleanup(file, cwd); } } @@ -276,33 +207,21 @@ public void testIDetectDirInS3FileInMs() throws Exception { // create a file with guarded fs mkdirs(cwd); awaitFileStatus(guardedFs, cwd); - // modify the cwd metadata and set that it's not a directory final S3AFileStatus newParentFile = MetadataStoreTestBase .basicFileStatus(cwd, 1, false, 1); metadataStore.put(new PathMetadata(newParentFile)); - final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore); - + final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore); final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); - assertComparePairsSize(comparePairs, 1); // check the child that the dir in s3 is a file in the ms - final S3GuardFsck.ComparePair cwdPair = comparePairs.stream() - .filter(p -> p.getPath().equals(cwd)) - .findFirst().get(); - assertNotNull("The pair should not be null.", cwdPair); - assertTrue("The cwdPair must contain a violation.", cwdPair.containsViolation()); - Assertions.assertThat(cwdPair.getViolations()) - .describedAs("Violations in the cwdPair") - .contains(S3GuardFsck.Violation.DIR_IN_S3_FILE_IN_MS); + checkForViolationInPairs(cwd, comparePairs, + S3GuardFsck.Violation.DIR_IN_S3_FILE_IN_MS); } finally { - // delete the working directory with all of its contents - rawFS.delete(cwd, true); - metadataStore.forgetMetadata(cwd); + cleanup(cwd); } } @@ -311,38 +230,22 @@ public void testIDetectFileInS3DirInMs() throws Exception { final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); final Path file = new Path(cwd, "file"); try { - // create a file with guarded fs - touch(guardedFs, file); - awaitFileStatus(guardedFs, file); - + touchGuardedAndWaitRaw(file); // modify the cwd metadata and set that it's not a directory final S3AFileStatus newFile = MetadataStoreTestBase .basicFileStatus(file, 1, true, 1); metadataStore.put(new PathMetadata(newFile)); - final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore); - + final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore); final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); assertComparePairsSize(comparePairs, 1); - // check the child that the dir in s3 is a file in the ms - final S3GuardFsck.ComparePair filePair = comparePairs.stream() - .filter(p -> p.getPath().equals(file)) - .findFirst().get(); - assertNotNull("The pair should not be null.", filePair); - assertTrue("The filePair must contain a violation.", - filePair.containsViolation()); - Assertions.assertThat(filePair.getViolations()) - .describedAs("Violations in the cwdPair") - .contains(S3GuardFsck.Violation.FILE_IN_S3_DIR_IN_MS); + checkForViolationInPairs(file, comparePairs, + S3GuardFsck.Violation.FILE_IN_S3_DIR_IN_MS); } finally { - // delete the working directory with all of its contents - rawFS.delete(cwd, true); - metadataStore.forgetMetadata(file); - metadataStore.forgetMetadata(cwd); + cleanup(file, cwd); } } @@ -360,34 +263,22 @@ public void testIAuthoritativeDirectoryContentMismatch() throws Exception { final Path fileIc1 = new Path(cwdIncorrect, "fileC1"); final Path fileIc2 = new Path(cwdIncorrect, "fileC2"); try { - touch(guardedFs, fileC1); - touch(guardedFs, fileC2); - touch(guardedFs, fileIc1); - - awaitFileStatus(rawFS, fileC1); - awaitFileStatus(rawFS, fileC2); - awaitFileStatus(rawFS, fileIc1); - - final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore); + touchGuardedAndWaitRaw(fileC1); + touchGuardedAndWaitRaw(fileC2); + touchGuardedAndWaitRaw(fileIc1); // get listing from ms and set it authoritative - final DirListingMetadata dlmC = - metadataStore.listChildren(cwdCorrect); - final DirListingMetadata dlmIc = - metadataStore.listChildren(cwdIncorrect); + final DirListingMetadata dlmC = metadataStore.listChildren(cwdCorrect); + final DirListingMetadata dlmIc = metadataStore.listChildren(cwdIncorrect); dlmC.setAuthoritative(true); dlmIc.setAuthoritative(true); metadataStore.put(dlmC, null); metadataStore.put(dlmIc, null); - final DirListingMetadata dlmCa = - metadataStore.listChildren(cwdCorrect); - // add a file raw so the listing will be different. - touch(rawFS, fileIc2); - awaitFileStatus(rawFS, fileIc2); + touchRawAndWaitRaw(fileIc2); + final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore); final List pairsCorrect = s3GuardFsck.compareS3ToMs(cwdCorrect); final List pairsIncorrect = @@ -398,25 +289,10 @@ public void testIAuthoritativeDirectoryContentMismatch() throws Exception { .noneMatch(p -> p.getPath().equals(cwdCorrect))); // Assert that the incorrect listing contains the violation. - final S3GuardFsck.ComparePair childPair = pairsIncorrect.stream() - .filter(p -> p.getPath().equals(cwdIncorrect)) - .findFirst().get(); - assertNotNull("The pair should not be null.", childPair); - assertTrue("The childPair must contain a violation.", childPair.containsViolation()); - Assertions.assertThat(childPair.getViolations()) - .describedAs("Violations in the childPair") - .contains(S3GuardFsck.Violation.AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH); - + checkForViolationInPairs(cwdIncorrect, pairsIncorrect, + S3GuardFsck.Violation.AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH); } finally { - // cleanup - rawFS.delete(cwdCorrect, true); - rawFS.delete(cwdIncorrect, true); - metadataStore.forgetMetadata(fileIc1); - metadataStore.forgetMetadata(fileIc2); - metadataStore.forgetMetadata(fileC1); - metadataStore.forgetMetadata(fileC2); - metadataStore.forgetMetadata(cwdCorrect); - metadataStore.forgetMetadata(cwdIncorrect); + cleanup(fileC1, fileC2, fileIc1, fileIc2, cwdCorrect, cwdIncorrect); } } @@ -426,40 +302,26 @@ public void testIDetectLengthMismatch() throws Exception { final Path file = new Path(cwd, "file"); try { // create a file with guarded fs - touch(guardedFs, file); - awaitFileStatus(guardedFs, file); + touchGuardedAndWaitRaw(file); // modify the file metadata so the length will not match final S3AFileStatus newFile = MetadataStoreTestBase .basicFileStatus(file, 9999, false, 1); metadataStore.put(new PathMetadata(newFile)); - final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore); - + final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore); final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); assertComparePairsSize(comparePairs, 1); - // Assert that the correct dir does not contain the violation. assertTrue(comparePairs.stream() .noneMatch(p -> p.getPath().equals(cwd))); - // Assert that the incorrect file meta contains the violation. - final S3GuardFsck.ComparePair childPair = comparePairs.stream() - .filter(p -> p.getPath().equals(file)) - .findFirst().get(); - assertNotNull("The pair should not be null.", childPair); - assertTrue("The childPair must contain a violation.", childPair.containsViolation()); - Assertions.assertThat(childPair.getViolations()) - .describedAs("Violations in the childPair") - .contains(S3GuardFsck.Violation.LENGTH_MISMATCH); + checkForViolationInPairs(file, comparePairs, + S3GuardFsck.Violation.LENGTH_MISMATCH); } finally { - // delete the working directory with all of its contents - rawFS.delete(cwd, true); - metadataStore.forgetMetadata(file); - metadataStore.forgetMetadata(cwd); + cleanup(file, cwd); } } @@ -469,48 +331,31 @@ public void testIDetectModTimeMismatch() throws Exception { final Path file = new Path(cwd, "file"); try { // create a file with guarded fs - touch(guardedFs, file); - awaitFileStatus(guardedFs, file); - + touchGuardedAndWaitRaw(file); // modify the file metadata so the length will not match final S3AFileStatus newFileStatus = MetadataStoreTestBase .basicFileStatus(file, 0, false, 1); metadataStore.put(new PathMetadata(newFileStatus)); - // modify the parent meta entry so the MOD_TIME will surely be up to date - final FileStatus oldCwdFileStatus = rawFS.getFileStatus(cwd); + final FileStatus oldCwdFileStatus = rawFs.getFileStatus(cwd); final S3AFileStatus newCwdFileStatus = MetadataStoreTestBase .basicFileStatus(file, 0, true, oldCwdFileStatus.getModificationTime()); metadataStore.put(new PathMetadata(newCwdFileStatus)); - final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore); - + final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore); final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); assertComparePairsSize(comparePairs, 1); - // Assert that the correct dir does not contain the violation. assertTrue(comparePairs.stream() .noneMatch(p -> p.getPath().equals(cwd))); - // check the file meta that there's a violation. - final S3GuardFsck.ComparePair childPair = comparePairs.stream() - .filter(p -> p.getPath().equals(file)) - .findFirst().get(); - assertNotNull("The pair should not be null.", childPair); - assertTrue("The childPair must contain a violation.", - childPair.containsViolation()); - Assertions.assertThat(childPair.getViolations()) - .describedAs("Violations in the childPair") - .contains(S3GuardFsck.Violation.MOD_TIME_MISMATCH); + checkForViolationInPairs(file, comparePairs, + S3GuardFsck.Violation.MOD_TIME_MISMATCH); } finally { - // delete the working directory with all of its contents - rawFS.delete(cwd, true); - metadataStore.forgetMetadata(file); - metadataStore.forgetMetadata(cwd); + cleanup(file, cwd); } } @@ -519,38 +364,22 @@ public void testIEtagMismatch() throws Exception { final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); final Path file = new Path(cwd, "file"); try { - // create a file with guarded fs - touch(guardedFs, file); - awaitFileStatus(guardedFs, file); - + touchGuardedAndWaitRaw(file); // modify the file metadata so the etag will not match final S3AFileStatus newFileStatus = new S3AFileStatus(1, 1, file, 1, "", "etag", "versionId"); metadataStore.put(new PathMetadata(newFileStatus)); - final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore); - + final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore); final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); assertComparePairsSize(comparePairs, 1); - // check the child that there's a BLOCKSIZE_MISMATCH - final S3GuardFsck.ComparePair childPair = comparePairs.stream() - .filter(p -> p.getPath().equals(file)) - .findFirst().get(); - assertNotNull("The pair should not be null.", childPair); - assertTrue("The childPair must contain a violation.", - childPair.containsViolation()); - Assertions.assertThat(childPair.getViolations()) - .describedAs("Violations in the childPair") - .contains(S3GuardFsck.Violation.ETAG_MISMATCH); + checkForViolationInPairs(file, comparePairs, + S3GuardFsck.Violation.ETAG_MISMATCH); } finally { - // delete the working directory with all of its contents - rawFS.delete(cwd, true); - metadataStore.forgetMetadata(file); - metadataStore.forgetMetadata(cwd); + cleanup(file, cwd); } } @@ -561,53 +390,30 @@ public void testINoEtag() throws Exception { final Path file2 = new Path(cwd, "file2"); try { // create a file1 with guarded fs - touch(guardedFs, file1); - touch(guardedFs, file2); - awaitFileStatus(guardedFs, file1); - awaitFileStatus(guardedFs, file2); - + touchGuardedAndWaitRaw(file1); + touchGuardedAndWaitRaw(file2); // modify the file1 metadata so there's no etag final S3AFileStatus newFile1Status = new S3AFileStatus(1, 1, file1, 1, "", null, "versionId"); final S3AFileStatus newFile2Status = new S3AFileStatus(1, 1, file2, 1, "", "etag", "versionId"); - metadataStore.put(new PathMetadata(newFile1Status)); metadataStore.put(new PathMetadata(newFile2Status)); - final S3GuardFsck s3GuardFsck = - new S3GuardFsck(rawFS, metadataStore); - + final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore); final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); assertComparePairsSize(comparePairs, 2); // check file 1 that there's NO_ETAG - final S3GuardFsck.ComparePair file1Pair = comparePairs.stream() - .filter(p -> p.getPath().equals(file1)) - .findFirst().get(); - assertNotNull("The pair should not be null.", file1Pair); - assertTrue("The file1Pair must contain a violation.", - file1Pair.containsViolation()); - Assertions.assertThat(file1Pair.getViolations()) - .describedAs("Violations in the file1Pair") - .contains(S3GuardFsck.Violation.NO_ETAG); - + checkForViolationInPairs(file1, comparePairs, + S3GuardFsck.Violation.NO_ETAG); // check the child that there's no NO_ETAG violation - final S3GuardFsck.ComparePair file2Pair = comparePairs.stream() - .filter(p -> p.getPath().equals(file2)) - .findFirst().get(); - assertNotNull("The pair should not be null.", file2Pair); - Assertions.assertThat(file2Pair.getViolations()) - .describedAs("Violations in the file2Pair") - .doesNotContain(S3GuardFsck.Violation.NO_ETAG); + checkNoViolationInPairs(file2, comparePairs, + S3GuardFsck.Violation.NO_ETAG); } finally { - // delete the working directory with all of its contents - rawFS.delete(cwd, true); - metadataStore.forgetMetadata(file1); - metadataStore.forgetMetadata(file2); - metadataStore.forgetMetadata(cwd); + cleanup(file1, file2, cwd); } } @@ -617,4 +423,55 @@ protected void assertComparePairsSize( .describedAs("Number of compare pairs") .isEqualTo(num); } + + private void touchGuardedAndWaitRaw(Path file) throws Exception { + touchAndWait(guardedFs, rawFs, file); + } + + private void touchRawAndWaitRaw(Path file) throws Exception { + touchAndWait(rawFs, rawFs, file); + } + + private void touchAndWait(FileSystem forTouch, FileSystem forWait, Path file) + throws IOException { + touch(forTouch, file); + touch(forWait, file); + } + + private void checkForViolationInPairs(Path file, + List comparePairs, + S3GuardFsck.Violation violation) { + final S3GuardFsck.ComparePair childPair = comparePairs.stream() + .filter(p -> p.getPath().equals(file)) + .findFirst().get(); + assertNotNull("The pair should not be null.", childPair); + assertTrue("The pair must contain a violation.", + childPair.containsViolation()); + Assertions.assertThat(childPair.getViolations()) + .describedAs("Violations in the pair") + .contains(violation); + } + + private void checkNoViolationInPairs(Path file2, + List comparePairs, + S3GuardFsck.Violation violation) { + final S3GuardFsck.ComparePair file2Pair = comparePairs.stream() + .filter(p -> p.getPath().equals(file2)) + .findFirst().get(); + assertNotNull("The pair should not be null.", file2Pair); + Assertions.assertThat(file2Pair.getViolations()) + .describedAs("Violations in the pair") + .doesNotContain(violation); + } + + private void cleanup(Path... paths) { + for (Path path : paths) { + try { + metadataStore.forgetMetadata(path); + rawFs.delete(path, true); + } catch (IOException e) { + LOG.error("Error during cleanup.", e); + } + } + } } \ No newline at end of file From a9e9eb41183fa069fed7396941d9c7e5cba59632 Mon Sep 17 00:00:00 2001 From: Gabor Bota Date: Tue, 10 Sep 2019 18:36:03 +0200 Subject: [PATCH 14/16] MOD_TIME_MISMATCH allowed within range Change-Id: Ib933b0cfee6fd5dd9da0a062b5f81c26e94d383c --- .../apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java | 15 +++++++++------ .../hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java | 11 ++++++----- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java index c9f9a184c33f7..0c81c8ba1fb4e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java @@ -60,6 +60,8 @@ public class S3GuardFsck { private final S3AFileSystem rawFS; private final DynamoDBMetadataStore metadataStore; + private final long MOD_TIME_RANGE = 2000L; + /** * Creates an S3GuardFsck. * @param fs the filesystem to compare to @@ -239,12 +241,10 @@ protected ComparePair compareFileStatusToPathMetadata( if (msPathMetadata != null) { LOG.info("Path: {} - Length S3: {}, MS: {} " + - "- Etag S3: {}, MS: {} " + - "- VersionId: S3: {}, MS: {}", + "- Etag S3: {}, MS: {} ", path, s3FileStatus.getLen(), msPathMetadata.getFileStatus().getLen(), - s3FileStatus.getETag(), msPathMetadata.getFileStatus().getETag(), - s3FileStatus.getVersionId(), msPathMetadata.getFileStatus().getVersionId()); + s3FileStatus.getETag(), msPathMetadata.getFileStatus().getETag()); } else { LOG.info("Path: {} - Length S3: {} - Etag S3: {}, no record in MS.", path, s3FileStatus.getLen(), s3FileStatus.getETag()); @@ -289,8 +289,11 @@ protected ComparePair compareFileStatusToPathMetadata( comparePair.violations.add(Violation.LENGTH_MISMATCH); } - if (s3FileStatus.getModificationTime() != - msFileStatus.getModificationTime()) { + // ModTime should be in the accuracy range defined. + long modTimeDiff = Math.abs( + s3FileStatus.getModificationTime() - msFileStatus.getModificationTime() + ); + if (modTimeDiff > MOD_TIME_RANGE) { comparePair.violations.add(Violation.MOD_TIME_MISMATCH); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java index 79ff34f2a3afb..b1da6f035125b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java @@ -332,17 +332,18 @@ public void testIDetectModTimeMismatch() throws Exception { try { // create a file with guarded fs touchGuardedAndWaitRaw(file); - // modify the file metadata so the length will not match - final S3AFileStatus newFileStatus = MetadataStoreTestBase - .basicFileStatus(file, 0, false, 1); - metadataStore.put(new PathMetadata(newFileStatus)); // modify the parent meta entry so the MOD_TIME will surely be up to date final FileStatus oldCwdFileStatus = rawFs.getFileStatus(cwd); final S3AFileStatus newCwdFileStatus = MetadataStoreTestBase - .basicFileStatus(file, 0, true, + .basicFileStatus(cwd, 0, true, oldCwdFileStatus.getModificationTime()); metadataStore.put(new PathMetadata(newCwdFileStatus)); + // modify the file metadata so the length will not match + final S3AFileStatus newFileStatus = MetadataStoreTestBase + .basicFileStatus(file, 0, false, 1); + metadataStore.put(new PathMetadata(newFileStatus)); + final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore); final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); From 633d6cd09eb734fa135865ac8e1ab53e2eef869a Mon Sep 17 00:00:00 2001 From: Gabor Bota Date: Wed, 11 Sep 2019 13:46:54 +0200 Subject: [PATCH 15/16] added TOMBSTONED_IN_MS_NOT_DELETED_IN_S3, fixed some checkstyle issues Change-Id: I98da340813d826acdf21e13698942c9cde09f192 --- .../hadoop/fs/s3a/s3guard/S3GuardFsck.java | 14 +++++++++- .../s3guard/S3GuardFsckViolationHandler.java | 26 ++++++++++++++---- .../fs/s3a/s3guard/ITestS3GuardFsck.java | 27 +++++++++++++++++++ .../s3a/s3guard/ITestS3GuardToolDynamoDB.java | 11 ++++---- 4 files changed, 67 insertions(+), 11 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java index 0c81c8ba1fb4e..a9925df521851 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsck.java @@ -60,7 +60,7 @@ public class S3GuardFsck { private final S3AFileSystem rawFS; private final DynamoDBMetadataStore metadataStore; - private final long MOD_TIME_RANGE = 2000L; + private static final long MOD_TIME_RANGE = 2000L; /** * Creates an S3GuardFsck. @@ -270,10 +270,13 @@ protected ComparePair compareFileStatusToPathMetadata( LOG.debug("Entry is in the root directory, so there's no parent"); } + // If the msPathMetadata is null, we RETURN because + // there is no metadata compare with if (msPathMetadata == null) { comparePair.violations.add(Violation.NO_METADATA_ENTRY); return comparePair; } + final S3AFileStatus msFileStatus = msPathMetadata.getFileStatus(); if (s3FileStatus.isDirectory() && !msFileStatus.isDirectory()) { comparePair.violations.add(Violation.DIR_IN_S3_FILE_IN_MS); @@ -282,6 +285,10 @@ protected ComparePair compareFileStatusToPathMetadata( comparePair.violations.add(Violation.FILE_IN_S3_DIR_IN_MS); } + if(msPathMetadata.isDeleted()) { + comparePair.violations.add(Violation.TOMBSTONED_IN_MS_NOT_DELETED_IN_S3); + } + /** * Attribute check */ @@ -428,6 +435,11 @@ public enum Violation { S3GuardFsckViolationHandler.FileInS3DirInMs.class), AUTHORITATIVE_DIRECTORY_CONTENT_MISMATCH(1, S3GuardFsckViolationHandler.AuthDirContentMismatch.class), + /** + * 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), /** * Attribute mismatch. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java index d30fa016cbe67..ca0ff4037b090 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java @@ -259,7 +259,7 @@ public LengthMismatch(S3GuardFsck.ComparePair comparePair) { } @Override public String getError() { - return String.format("File length mismatch - s3: %s, ms: %s", + return String.format("File length mismatch - S3: %s, MS: %s", getS3FileStatus().getLen(), getMsFileStatus().getLen()); } } @@ -275,14 +275,14 @@ public ModTimeMismatch(S3GuardFsck.ComparePair comparePair) { @Override public String getError() { - return String.format("File timestamp mismatch - s3: %s, ms: %s", + return String.format("File timestamp mismatch - S3: %s, MS: %s", getS3FileStatus().getModificationTime(), getMsFileStatus().getModificationTime()); } } /** - * The violation handler when there's a version id mismatch + * The violation handler when there's a version id mismatch. */ public static class VersionIdMismatch extends ViolationHandler { @@ -292,7 +292,7 @@ public VersionIdMismatch(S3GuardFsck.ComparePair comparePair) { @Override public String getError() { - return String.format("getVersionId mismatch - s3: %s, ms: %s", + return String.format("getVersionId mismatch - S3: %s, MS: %s", getS3FileStatus().getVersionId(), getMsFileStatus().getVersionId()); } } @@ -308,7 +308,7 @@ public EtagMismatch(S3GuardFsck.ComparePair comparePair) { @Override public String getError() { - return String.format("Etag mismatch - s3: %s, ms: %s", + return String.format("Etag mismatch - S3: %s, MS: %s", getS3FileStatus().getETag(), getMsFileStatus().getETag()); } } @@ -327,4 +327,20 @@ public String getError() { return "No etag."; } } + + /** + * The violation handler when there's a tombstoned entry in the ms is + * present, but the object is not deleted in S3. + */ + public class TombstonedInMsNotDeletedInS3 extends ViolationHandler { + + public TombstonedInMsNotDeletedInS3(S3GuardFsck.ComparePair comparePair) { + super(comparePair); + } + + @Override + public String getError() { + return "The entry for the path is tombstoned in the MS."; + } + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java index b1da6f035125b..a582a6b917010 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java @@ -418,6 +418,33 @@ public void testINoEtag() throws Exception { } } + @Test + public void testTombstonedInMsNotDeletedInS3() throws Exception { + final Path cwd = path("/" + getMethodName() + "-" + UUID.randomUUID()); + final Path file = new Path(cwd, "file"); + try { + // create a file with guarded fs + touchGuardedAndWaitRaw(file); + // set isDeleted flag in ms to true (tombstone item) + final PathMetadata fileMeta = metadataStore.get(file); + fileMeta.setIsDeleted(true); + metadataStore.put(fileMeta); + + final S3GuardFsck s3GuardFsck = new S3GuardFsck(rawFs, metadataStore); + final List comparePairs = + s3GuardFsck.compareS3ToMs(cwd); + + assertComparePairsSize(comparePairs, 1); + + // check fil1 that there's the violation + 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); + } + } + protected void assertComparePairsSize( List comparePairs, int num) { Assertions.assertThat(comparePairs.size()) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java index 6132612ac6568..205eb65a1c919 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java @@ -301,21 +301,22 @@ public void testCLIFsckWithoutParam() throws Exception { public void testCLIFsckWithParam() throws Exception { final int result = run(S3GuardTool.Fsck.NAME, "-check", "s3a://" + getFileSystem().getBucket()); - assertEquals("The result should be success when fsck is running with " - + "correct parameters.", SUCCESS, result); + 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); } @Test public void testCLIFsckWithParamParentOfRoot() throws Exception { intercept(IOException.class, "Invalid URI", () -> run(S3GuardTool.Fsck.NAME, "-check", - "s3a://" + getFileSystem().getBucket() + "/..")); + "s3a://" + getFileSystem().getBucket() + "/..")); } @Test public void testCLIFsckFailInitializeFs() throws Exception { intercept(FileNotFoundException.class, "does not exist", - () -> run(S3GuardTool.Fsck.NAME, "-check", - "s3a://this-bucket-does-not-exist-" + UUID.randomUUID())); + () -> run(S3GuardTool.Fsck.NAME, "-check", + "s3a://this-bucket-does-not-exist-" + UUID.randomUUID())); } } From 1dfc3a1c31da2610080d4d9ff4d10f5adf83e838 Mon Sep 17 00:00:00 2001 From: Gabor Bota Date: Wed, 11 Sep 2019 15:53:56 +0200 Subject: [PATCH 16/16] fixed test failures Change-Id: Ic31cbd5925b92df6c421012a3b91497d16aa6bef --- .../hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java | 2 +- .../org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java | 7 +++---- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java index ca0ff4037b090..97e6fe644a23a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardFsckViolationHandler.java @@ -332,7 +332,7 @@ public String getError() { * The violation handler when there's a tombstoned entry in the ms is * present, but the object is not deleted in S3. */ - public class TombstonedInMsNotDeletedInS3 extends ViolationHandler { + public static class TombstonedInMsNotDeletedInS3 extends ViolationHandler { public TombstonedInMsNotDeletedInS3(S3GuardFsck.ComparePair comparePair) { super(comparePair); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java index a582a6b917010..ea92f694ec974 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsck.java @@ -191,7 +191,6 @@ public void testIDetectParentTombstoned() throws Exception { final List comparePairs = s3GuardFsck.compareS3ToMs(cwd); - assertComparePairsSize(comparePairs, 1); // check the child that the parent is tombstoned checkForViolationInPairs(file, comparePairs, S3GuardFsck.Violation.PARENT_TOMBSTONED); @@ -447,9 +446,9 @@ public void testTombstonedInMsNotDeletedInS3() throws Exception { protected void assertComparePairsSize( List comparePairs, int num) { - Assertions.assertThat(comparePairs.size()) + Assertions.assertThat(comparePairs) .describedAs("Number of compare pairs") - .isEqualTo(num); + .hasSize(num); } private void touchGuardedAndWaitRaw(Path file) throws Exception { @@ -502,4 +501,4 @@ private void cleanup(Path... paths) { } } } -} \ No newline at end of file +}