From 142fde417520d9f0144053c205ebd98c3c25f56e Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 21 Jun 2019 13:32:58 +0100 Subject: [PATCH 01/23] HADOOP-16384: prune resilience. Step one: a new root test to execute prune against the live store and DDB table. To follow: debug the conditions leading to prune fail on my store :) Change-Id: I25d138ed7b42f2b5eda12c2cd736f8c054b69820 --- hadoop-tools/hadoop-aws/pom.xml | 4 + .../PathMetadataDynamoDBTranslation.java | 3 +- .../site/markdown/tools/hadoop-aws/s3guard.md | 2 +- .../hadoop/fs/s3a/AbstractS3ATestBase.java | 7 + .../s3guard/AbstractS3GuardToolTestBase.java | 2 +- .../s3guard/ITestS3GuardRootOperations.java | 138 ++++++++++++++++++ 6 files changed, 153 insertions(+), 3 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index dbe593d437670..f79136d1416cf 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -187,6 +187,8 @@ **/ITestDynamoDBMetadataStoreScale.java **/ITestTerasort*.java + **/ITestS3GuardRootOperations*.java + @@ -224,6 +226,8 @@ **/ITestTerasort*.java + + **/ITestS3GuardRootOperations*.java diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java index 7d4980a06fc29..ba691d63479bb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java @@ -293,7 +293,8 @@ static KeyAttribute pathToParentKeyAttribute(Path path) { @VisibleForTesting public static String pathToParentKey(Path path) { Preconditions.checkNotNull(path); - Preconditions.checkArgument(path.isUriPathAbsolute(), "Path not absolute"); + Preconditions.checkArgument(path.isUriPathAbsolute(), + "Path not absolute: '%s'", path); URI uri = path.toUri(); String bucket = uri.getHost(); Preconditions.checkArgument(!StringUtils.isEmpty(bucket), diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md index c2f37483732a2..1e83e5435d352 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md @@ -796,7 +796,7 @@ time" is older than the specified age. ```bash hadoop s3guard prune [-days DAYS] [-hours HOURS] [-minutes MINUTES] - [-seconds SECONDS] [-m URI] ( -region REGION | s3a://BUCKET ) + [-seconds SECONDS] [-meta URI] ( -region REGION | s3a://BUCKET ) ``` A time value of hours, minutes and/or seconds must be supplied. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java index c1942a944353b..b05bbb48bab02 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java @@ -49,8 +49,15 @@ protected AbstractFSContract createContract(Configuration conf) { return new S3AContract(conf); } + @Override + public void setup() throws Exception { + Thread.currentThread().setName("setup"); + super.setup(); + } + @Override public void teardown() throws Exception { + Thread.currentThread().setName("teardown"); super.teardown(); describe("closing file system"); IOUtils.closeStream(getFileSystem()); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java index eaaa50c1c0f3a..5315474013541 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java @@ -105,7 +105,7 @@ protected static void expectResult(int expected, * @return the output of any successful run * @throws Exception failure */ - protected static String expectSuccess( + public static String expectSuccess( String message, S3GuardTool tool, String... args) throws Exception { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java new file mode 100644 index 0000000000000..70f9623a2816b --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.s3guard; + +import java.net.URI; +import java.util.concurrent.TimeUnit; + +import org.assertj.core.api.Assertions; +import org.junit.FixMethodOrder; +import org.junit.Test; +import org.junit.runners.MethodSorters; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.S3ATestUtils; +import org.apache.hadoop.fs.s3a.impl.StoreContext; + +import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.hadoop.fs.s3a.Constants.ENABLE_MULTI_DELETE; +import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; + +/** + * This test run against the root of the FS, and operations which span the DDB + * table. + * For this reason, these tests are executed in the sequential phase of the + * integration tests. + *

+ * The tests only run if DynamoDB is the metastore. + */ +@FixMethodOrder(MethodSorters.NAME_ASCENDING) +public class ITestS3GuardRootOperations extends AbstractS3ATestBase { + + private StoreContext storeContext; + + private String fsUriStr; + + private DynamoDBMetadataStore metastore; + + private String metastoreUriStr; + + /** + * The test timeout is increased in case previous tests have created + * many tombstone markers which now need to be purged. + * @return the test timeout. + */ + @Override + protected int getTestTimeoutMillis() { + return SCALE_TEST_TIMEOUT_SECONDS * 1000; + } + + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + String bucketName = getTestBucketName(conf); + + // set a sleep time of 0 on pruning, for speedier test runs. + removeBucketOverrides(bucketName, conf, ENABLE_MULTI_DELETE); + conf.setTimeDuration( + S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY, + 0, + TimeUnit.MILLISECONDS); + return conf; + } + + @Override + public void setup() throws Exception { + super.setup(); + S3AFileSystem fs = getFileSystem(); + Configuration conf = fs.getConf(); + S3ATestUtils.assumeS3GuardState(true, conf); + storeContext = fs.createStoreContext(); + assume("Filesystem isn't running DDB", + storeContext.getMetadataStore() instanceof DynamoDBMetadataStore); + metastore = (DynamoDBMetadataStore) storeContext.getMetadataStore(); + URI fsURI = storeContext.getFsURI(); + fsUriStr = fsURI.toString(); + if (!fsUriStr.endsWith("/")) { + fsUriStr = fsUriStr + "/"; + } + metastoreUriStr = "dynamodb://" + metastore.getTableName() + "/"; + } + + @Override + public void teardown() throws Exception { + Thread.currentThread().setName("teardown"); + super.teardown(); + } + + @Test + public void test_100_FilesystemPrune() throws Throwable { + describe("Execute prune against a filesystem URI"); + S3AFileSystem fs = getFileSystem(); + Configuration conf = fs.getConf(); + int result = S3GuardTool.run(conf, + S3GuardTool.Prune.NAME, + fsUriStr); + Assertions.assertThat(result) + .describedAs("Result of prune %s", fsUriStr) + .isEqualTo(0); + } + + + @Test + public void test_200_MetastorePrune() throws Throwable { + describe("Execute prune against a dynamo URL"); + S3AFileSystem fs = getFileSystem(); + Configuration conf = fs.getConf(); + S3GuardTool.Prune cmd = new S3GuardTool.Prune(conf); + int result = S3GuardTool.run(conf, + S3GuardTool.Prune.NAME, + "-meta", checkNotNull(metastoreUriStr), + "-seconds", "1"); + Assertions.assertThat(result) + .describedAs("Result of prune %s", fsUriStr) + .isEqualTo(0); + } + +} From 23dad58547542d64143115838d8bd3ce6340ccfb Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 25 Jun 2019 22:47:50 +0100 Subject: [PATCH 02/23] HADOOP-16384: prune debugging. This adds a new (documented in s3guard testing) entry point which does a treewalk down the table and prints out the path entries. example: hadoop org.apache.hadoop.fs.s3a.s3guard.DumpS3GuardTable s3a://bucket-x/ out.csv This uses the service launcher entry point, which I (finally) fixed up to be better for execute operations, primarily reporting failures better. Change-Id: Ic11335e66d6b32f1766067bdcf89c23a2682b757 Tested: S3 ireland --- .../launcher/ServiceLaunchException.java | 13 + .../service/launcher/ServiceLauncher.java | 70 ++- .../fs/s3a/s3guard/DumpS3GuardTable.java | 406 ++++++++++++++++++ .../fs/s3a/s3guard/PathOrderComparators.java | 4 +- .../site/markdown/tools/hadoop-aws/testing.md | 53 ++- .../s3a/s3guard/ITestS3GuardToolDynamoDB.java | 23 + 6 files changed, 550 insertions(+), 19 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLaunchException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLaunchException.java index 1243a1fabfa50..d62496547daea 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLaunchException.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLaunchException.java @@ -78,4 +78,17 @@ public ServiceLaunchException(int exitCode, String format, Object... args) { } } + /** + * Create a formatted exception. + *

+ * This uses {@link String#format(String, Object...)} + * to build the formatted exception in the ENGLISH locale. + * @param exitCode exit code + * @param cause inner cause + * @param format format for message to use in exception + * @param args list of arguments + */ + public ServiceLaunchException(int exitCode, Throwable cause, String format, Object... args) { + super(exitCode, String.format(Locale.ENGLISH, format, args), cause); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java index da91a3d0e6c70..5e8a1f4eb21fb 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java @@ -200,7 +200,7 @@ public ServiceLauncher(String serviceName, String serviceClassName) { * Get the service. * * Null until - * {@link #coreServiceLaunch(Configuration, List, boolean, boolean)} + * {@link #coreServiceLaunch(Configuration, Service, List, boolean, boolean)} * has completed. * @return the service */ @@ -303,7 +303,7 @@ public void launchServiceAndExit(List args) { exitException = e; noteException(exitException); } - if (exitException.getExitCode() != 0) { + if (exitException.getExitCode() == LauncherExitCodes.EXIT_USAGE) { // something went wrong. Print the usage and commands System.err.println(getUsageMessage()); System.err.println("Command: " + argumentString); @@ -328,8 +328,18 @@ protected void bindCommandOptions() { * @param exitException exception */ void noteException(ExitUtil.ExitException exitException) { - LOG.debug("Exception raised", exitException); - serviceExitCode = exitException.getExitCode(); + int exitCode = exitException.getExitCode(); + if (exitCode != 0) { + LOG.debug("Exception raised with exit code {}", + exitCode, + exitException); + Throwable cause = exitException.getCause(); + if (cause != null) { + // log the nested exception in more detail + LOG.warn("{}", cause.toString(), cause); + } + } + serviceExitCode = exitCode; serviceException = exitException; } @@ -451,17 +461,38 @@ public int loadConfigurationClasses() { * @param execute execute/wait for the service to stop. * @return an exit exception, which will have a status code of 0 if it worked */ - @VisibleForTesting public ExitUtil.ExitException launchService(Configuration conf, List processedArgs, boolean addShutdownHook, boolean execute) { - + return launchService(conf, null, processedArgs, addShutdownHook, execute); + } + + /** + * Launch a service catching all exceptions and downgrading them to exit codes + * after logging. + * + * Sets {@link #serviceException} to this value. + * @param conf configuration to use + * @param instance optional instance of the service. + * @param processedArgs command line after the launcher-specific arguments + * have been stripped out. + * @param addShutdownHook should a shutdown hook be added to terminate + * this service on shutdown. Tests should set this to false. + * @param execute execute/wait for the service to stop. + * @return an exit exception, which will have a status code of 0 if it worked + */ + public ExitUtil.ExitException launchService(Configuration conf, + S instance, + List processedArgs, + boolean addShutdownHook, + boolean execute) { + ExitUtil.ExitException exitException; - + try { - int exitCode = coreServiceLaunch(conf, processedArgs, addShutdownHook, - execute); + int exitCode = coreServiceLaunch(conf, instance, processedArgs, + addShutdownHook, execute); if (service != null) { // check to see if the service failed Throwable failure = service.getFailureCause(); @@ -495,6 +526,12 @@ public ExitUtil.ExitException launchService(Configuration conf, // exit exceptions are passed through unchanged exitException = ee; } catch (Throwable thrown) { + // other errors need a full log. + LOG.error("Exception raised {}", + service != null + ? (service.toString() + " in state " + service.getServiceState()) + : "during service instantiation", + thrown); exitException = convertToExitException(thrown); } noteException(exitException); @@ -514,6 +551,7 @@ public ExitUtil.ExitException launchService(Configuration conf, * {@link #getService()}. * * @param conf configuration + * @param instance optional instance of the service. * @param processedArgs arguments after the configuration parameters * have been stripped out. * @param addShutdownHook should a shutdown hook be added to terminate @@ -530,12 +568,19 @@ public ExitUtil.ExitException launchService(Configuration conf, */ protected int coreServiceLaunch(Configuration conf, + S instance, List processedArgs, boolean addShutdownHook, boolean execute) throws Exception { // create the service instance - instantiateService(conf); + if (instance == null) { + instantiateService(conf); + } else { + // service already exists, so instantiate + configuration = conf; + service = instance; + } ServiceShutdownHook shutdownHook = null; // and the shutdown hook if requested @@ -685,8 +730,7 @@ protected static ExitUtil.ExitException convertToExitException( } // construct the new exception with the original message and // an exit code - exitException = new ServiceLaunchException(exitCode, message); - exitException.initCause(thrown); + exitException = new ServiceLaunchException(exitCode, thrown, message); return exitException; } @@ -917,7 +961,7 @@ protected List parseCommandArgs(Configuration conf, throw new ServiceLaunchException(EXIT_COMMAND_ARGUMENT_ERROR, e); } catch (RuntimeException e) { // lower level issue such as XML parse failure - throw new ServiceLaunchException(EXIT_COMMAND_ARGUMENT_ERROR, + throw new ServiceLaunchException(EXIT_COMMAND_ARGUMENT_ERROR, e, E_PARSE_FAILED + " %s : %s", argString, e); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java new file mode 100644 index 0000000000000..48767a8852eb7 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java @@ -0,0 +1,406 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.s3guard; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.io.PrintWriter; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Comparator; +import java.util.Date; +import java.util.LinkedList; +import java.util.List; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.service.Service; +import org.apache.hadoop.service.launcher.AbstractLaunchableService; +import org.apache.hadoop.service.launcher.LauncherExitCodes; +import org.apache.hadoop.service.launcher.ServiceLaunchException; +import org.apache.hadoop.service.launcher.ServiceLauncher; +import org.apache.hadoop.util.ExitUtil; + +import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_USAGE; + +/** + * This is a new diagnostics entry point which does a TSV dump of + * the DDB treewalk. + */ +public class DumpS3GuardTable extends AbstractLaunchableService { + + private static final Logger LOG = + LoggerFactory.getLogger(DumpS3GuardTable.class); + + public static final String NAME = "DumpS3GuardTable"; + + private static String USAGE_MESSAGE = NAME + " "; + + private List arguments; + + private DynamoDBMetadataStore store; + + public DumpS3GuardTable(final String name) { + super(name); + } + + public DumpS3GuardTable() { + this("DumpS3GuardTable"); + } + + @Override + public Configuration bindArgs(final Configuration config, + final List args) + throws Exception { + this.arguments = args; + return super.bindArgs(config, args); + } + + public List getArguments() { + return arguments; + } + + /** + * {@inheritDoc} + */ + @Override + public int execute() throws ServiceLaunchException, IOException { + + Preconditions.checkNotNull(arguments, "No arguments"); + Preconditions.checkState(arguments.size() == 2, + "Wrong number of arguments: %s", arguments.size()); + + String fsURI = arguments.get(0); + String destFilename = arguments.get(1); + Configuration conf = getConfig(); + URI uri = null; + try { + uri = new URI(fsURI); + } catch (URISyntaxException e) { + throw fail("Bad URI " + fsURI, e); + } + try { + FileSystem fileSystem = FileSystem.get(uri, conf); + require(fileSystem instanceof S3AFileSystem, + "Not an S3A Filesystem: " + fsURI); + S3AFileSystem fs = (S3AFileSystem) fileSystem; + require(fs.hasMetadataStore(), + "Filesystem has no metadata store: " + fsURI); + MetadataStore ms = fs.getMetadataStore(); + require(ms instanceof DynamoDBMetadataStore, + "Filesystem " + fsURI + + "does not have a DynamoDB metadata store: " + ms); + store = (DynamoDBMetadataStore) ms; +// Table table = store.getTable(); + + File destFile = new File(destFilename).getCanonicalFile(); + LOG.info("Writing Store details to {}", destFile); + try (CsvFile csv = new CsvFile(null, + new PrintWriter(destFile), + "\t", "\n", "\"")) { + csv.row(CsvFile.ALL_QUOTES, + "path", "type", "is_auth_dir", "deleted", + "is_empty_dir", "len", "updated", "updated_s", + "last_modified", "last_modified_s", "etag", "version"); + + Path basePath = fs.qualify(new Path(uri)); + LOG.info("Base path: {}", basePath); + DirListingMetadata children = store.listChildren(basePath); + dumpRecursively(csv, children); + } + return LauncherExitCodes.EXIT_SUCCESS; + } catch (Exception e) { + LOG.error("failure", e); + throw e; + } + } + + + private Pair dumpRecursively( + CsvFile csv, DirListingMetadata dir) throws IOException { + int files = 0, dirs = 1; + List childDirs = new ArrayList<>(); + Collection listing = dir.getListing(); + // sort by name + List sorted = new ArrayList<>(listing); + sorted.sort(new PathOrderComparators.PathMetadataComparator( + (l, r) -> l.compareTo(r))); + + for (PathMetadata pmd : sorted) { + DDBPathMetadata ddbMd = (DDBPathMetadata) pmd; + dumpEntry(csv, ddbMd); + if (ddbMd.getFileStatus().isDirectory()) { + childDirs.add(ddbMd); + } else { + files++; + } + } + for (DDBPathMetadata childDir : childDirs) { + DirListingMetadata children = store.listChildren( + childDir.getFileStatus().getPath()); + Pair pair = dumpRecursively(csv, + children); + dirs += pair.getLeft(); + files += pair.getRight(); + } + + return Pair.of(dirs, files); + } + + /** least significant bit is used for first column; 1 mean 'quote'. */ + public static final int QUOTE_MAP = 0b1110_1001_1111; + + private void dumpEntry(CsvFile csv, DDBPathMetadata md) { + S3AFileStatus fileStatus = md.getFileStatus(); + LOG.info("{}", md.prettyPrint()); + csv.row(QUOTE_MAP, + fileStatus.getPath().toString(), + fileStatus.isDirectory() ? "dir" : "file", + md.isAuthoritativeDir(), + md.isDeleted(), + md.isEmptyDirectory().name(), + fileStatus.getLen(), + md.getLastUpdated(), + stringify(md.getLastUpdated()), + fileStatus.getModificationTime(), + stringify(fileStatus.getModificationTime()), + fileStatus.getETag(), + fileStatus.getVersionId()); + } + + private String stringify(long millis) { + return new Date(millis).toString(); + } + + private void require(boolean condition, String error) { + if (!condition) { + throw fail(error); + } + } + + private ServiceLaunchException fail(String message, Throwable ex) { + return new ServiceLaunchException(LauncherExitCodes.EXIT_FAIL, message, ex); + } + + private ServiceLaunchException fail(String message) { + return new ServiceLaunchException(LauncherExitCodes.EXIT_FAIL, message); + } + + /** + * This is the JVM entry point for the service launcher. + * + * Converts the arguments to a list, then invokes {@link #serviceMain(List)} + * @param args command line arguments. + */ + public static void main(String[] args) { + try { + LinkedList argsList = new LinkedList<>(Arrays.asList(args)); + serviceMain(argsList); + } catch (ExitUtil.ExitException e) { + ExitUtil.terminate(e); + } + } + + /** + * The real main function, which takes the arguments as a list. + * Argument 0 MUST be the service classname + * @param argsList the list of arguments + */ + public static void serviceMain(List argsList) { + if (argsList.size() != 2) { + // no arguments: usage message + ExitUtil.terminate(new ServiceLaunchException(EXIT_USAGE, USAGE_MESSAGE)); + + } else { + ServiceLauncher serviceLauncher = + new ServiceLauncher<>(NAME); + + ExitUtil.ExitException ex = serviceLauncher.launchService( + new Configuration(), + new DumpS3GuardTable(), + argsList, + false, + true); + if (ex != null) { + throw ex; + } + } + } + + /** + * + * @param filesystem filesystem path + * @param conf configuration to instantiate with. + * @param destFile output file. + * @throws ExitUtil.ExitException failure. + */ + public static void dumpS3GuardStore(String filesystem, + Configuration conf, + File destFile) throws ExitUtil.ExitException { + ServiceLauncher serviceLauncher = + new ServiceLauncher<>(""); + if (!filesystem.endsWith("/")) { + filesystem += "/"; + } + + ExitUtil.ExitException ex = serviceLauncher.launchService( + conf, + new DumpS3GuardTable(), + Lists.newArrayList(filesystem, destFile.getAbsolutePath()), + false, + true); + if (ex != null && ex.getExitCode() != 0) { + throw ex; + } + } + + /** + * Writer for generating test CSV files. + * + * Quotes are manged by passing in a long whose specific bits control + * whether or not a row is quoted, bit 0 for column 0, etc. + */ + private static class CsvFile implements Closeable { + + + /** constant to quote all columns. */ + public static final long ALL_QUOTES = 0x7fffffff; + + /** quote nothing: {@value}. */ + public static final long NO_QUOTES = 0; + + private final Path path; + + private final PrintWriter out; + + private final String separator; + + private final String eol; + + private final String quote; + + CsvFile(final FileSystem fs, + final Path path, + final boolean overwrite, + final String separator, + final String eol, + final String quote) throws IOException { + this(path, + new PrintWriter(fs.create(path, overwrite)), + separator, eol, quote); + } + + public CsvFile( + final Path path, + final PrintWriter out, + final String separator, + final String eol, + final String quote) throws IOException { + this.separator = Preconditions.checkNotNull(separator); + this.eol = Preconditions.checkNotNull(eol); + this.quote = Preconditions.checkNotNull(quote); + this.path = path; + this.out = Preconditions.checkNotNull(out); + } + + /** + * Close the file, if not already done. + * @throws IOException on a failure. + */ + @Override + public synchronized void close() throws IOException { + if (out != null) { + out.close(); + } + } + + public Path getPath() { + return path; + } + + public String getSeparator() { + return separator; + } + + public String getEol() { + return eol; + } + + /** + * Write a row. + * Entries are quoted if the bit for that column is true. + * @param quotes quote policy: every bit defines the rule for that element + * @param columns columns to write + * @return self for ease of chaining. + */ + public CsvFile row(long quotes, Object... columns) { + Preconditions.checkNotNull(out); + for (int i = 0; i < columns.length; i++) { + if (i != 0) { + out.write(separator); + } + boolean toQuote = (quotes & 1) == 1; + // unsigned right shift to make next column flag @ position 0 + quotes = quotes >>> 1; + if (toQuote) { + out.write(quote); + } + Object column = columns[i]; + out.write(column != null ? column.toString() : ""); + if (toQuote) { + out.write(quote); + } + } + out.write(eol); + return this; + } + + /** + * Write a line. + * @param line line to print + * @return self for ease of chaining. + */ + public CsvFile line(String line) { + out.write(line); + out.write(eol); + return this; + } + + /** + * Get the output stream. + * @return the stream. + */ + public PrintWriter getOut() { + return out; + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java index a3a7967caf79b..ddbe2f419ee0a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java @@ -115,12 +115,12 @@ public int compare(final Path pathL, final Path pathR) { /** * Compare on path status. */ - private static final class PathMetadataComparator implements + public static final class PathMetadataComparator implements Comparator, Serializable { private final Comparator inner; - private PathMetadataComparator(final Comparator inner) { + public PathMetadataComparator(final Comparator inner) { this.inner = inner; } diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md index 7a591d99b60eb..83feac299a97b 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md @@ -845,7 +845,7 @@ it can be manually done: hadoop s3guard uploads -abort -force s3a://test-bucket/ * If you don't need it, destroy the S3Guard DDB table. - hadoop s3guard destroy s3a://hwdev-steve-ireland-new/ + hadoop s3guard destroy s3a://bucket-x/ The S3Guard tests will automatically create the Dynamo DB table in runs with `-Ds3guard -Ddynamo` set; default capacity of these buckets @@ -881,7 +881,7 @@ using an absolute XInclude reference to it. ``` -# Failure Injection +## Failure Injection **Warning do not enable any type of failure injection in production. The following settings are for testing only.** @@ -1014,7 +1014,7 @@ The inconsistent client is shipped in the `hadoop-aws` JAR, so it can be used in applications which work with S3 to see how they handle inconsistent directory listings. -## Testing S3Guard +## Testing S3Guard [S3Guard](./s3guard.html) is an extension to S3A which adds consistent metadata listings to the S3A client. As it is part of S3A, it also needs to be tested. @@ -1052,7 +1052,7 @@ The basic strategy for testing S3Guard correctness consists of: No charges are incurred for using this store, and its consistency guarantees are that of the underlying object store instance. -## Testing S3A with S3Guard Enabled +### Testing S3A with S3Guard Enabled All the S3A tests which work with a private repository can be configured to run with S3Guard by using the `s3guard` profile. When set, this will run @@ -1100,6 +1100,51 @@ property should be configured, and the name of that table should be different incurring AWS charges. +### How to dump the table to a CSV file + +There's an unstable, unsupported command to list the contents of a table +to a CSV, or more specifically a TSV file, on the local system + +``` +hadoop org.apache.hadoop.fs.s3a.s3guard.DumpS3GuardTable s3a://bucket-x/ out.csv +``` +This generates a file which can then be viewed on the command line or editor: + +``` +"path" "type" "is_auth_dir" "deleted" "is_empty_dir" "len" "updated" "updated_s" "last_modified" "last_modified_s" "etag" "version" +"s3a://bucket-x/FileSystemContractBaseTest" "file" "false" "true" "UNKNOWN" 0 1561484415455 "Tue Jun 25 18:40:15 BST 2019" 1561483826881 "Tue Jun 25 18:30:26 BST 2019" "" "" +"s3a://bucket-x/Users" "file" "false" "true" "UNKNOWN" 0 1561484415455 "Tue Jun 25 18:40:15 BST 2019" 1561484376835 "Tue Jun 25 18:39:36 BST 2019" "" "" +"s3a://bucket-x/dest-6f578c72-eb40-4767-a89d-66a6a5b89578" "file" "false" "true" "UNKNOWN" 0 1561484415455 "Tue Jun 25 18:40:15 BST 2019" 1561483757615 "Tue Jun 25 18:29:17 BST 2019" "" "" +"s3a://bucket-x/file.txt" "file" "false" "true" "UNKNOWN" 0 1561484415455 "Tue Jun 25 18:40:15 BST 2019" 1561484382603 "Tue Jun 25 18:39:42 BST 2019" "" "" +"s3a://bucket-x/fork-0001" "file" "false" "true" "UNKNOWN" 0 1561484415455 "Tue Jun 25 18:40:15 BST 2019" 1561484378086 "Tue Jun 25 18:39:38 BST 2019" "" "" +"s3a://bucket-x/fork-0002" "file" "false" "true" "UNKNOWN" 0 1561484415455 "Tue Jun 25 18:40:15 BST 2019" 1561484380177 "Tue Jun 25 18:39:40 BST 2019" "" "" +"s3a://bucket-x/fork-0003" "file" "false" "true" "UNKNOWN" 0 1561484415455 "Tue Jun 25 18:40:15 BST 2019" 1561484379690 "Tue Jun 25 18:39:39 BST 2019" "" "" +``` + +This is unstable: the output format may change without warning. +To understand the meaning of the fields, consult the documentation. +They are, currently: + +| field | meaning | source | +|-------|---------| -------| +| `path` | path of an entry | filestatus | +| `type` | type | filestatus | +| `is_auth_dir` | directory entry authoritative status | metadata | +| `deleted` | tombstone marker | metadata | +| `is_empty_dir` | does the entry represent an empty directory | metadata | +| `len` | | +| `len` | file length | filestatus | +| `last_modified` | file status last modified | filestatus | +| `last_modified_s` | file status last modified as string | filestatus | +| `updated` | time (millis) metadata was updated | metadata | +| `updated_s` | updated time as a string | metadata | +| `updated` | | metadata | +| `etag` | any etag | filestatus | +| `version` | any version| filestatus | + +As noted: this is unstable; entry list and meaning may change, sorting of output, +the listing algorithm, representation of types, etc. Use at your own risk. + ### Scale Testing MetadataStore Directly There are some scale tests that exercise Metadata Store implementations 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 f774bbb58caf1..7514df7244570 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,7 +18,13 @@ package org.apache.hadoop.fs.s3a.s3guard; +import java.io.BufferedReader; +import java.io.File; +import java.io.FileInputStream; import java.io.IOException; +import java.io.InputStreamReader; +import java.io.Reader; +import java.nio.charset.Charset; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -36,6 +42,7 @@ import org.junit.Assume; import org.junit.Test; +import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileSystem; @@ -280,4 +287,20 @@ public void testDestroyUnknownTable() throws Throwable { "-meta", "dynamodb://" + getTestTableName(DYNAMODB_TABLE)); } + @Test + public void testDumpTable() throws Throwable { + File destFile = File.createTempFile("dump", ".csv"); + describe("Dumping metastore to {}", destFile); + S3AFileSystem fs = getFileSystem(); + DumpS3GuardTable.dumpS3GuardStore( + fs.getUri().toString(), + fs.getConf(), + destFile); + try (BufferedReader in = new BufferedReader(new InputStreamReader( + new FileInputStream(destFile), Charset.forName("UTF-8")))) { + for (String line : IOUtils.readLines(in)) { + LOG.info(line); + } + } + } } From 9b179b9d40ec43b19135c40e0bc7e6c5bdac0709 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 26 Jun 2019 18:11:00 +0100 Subject: [PATCH 03/23] HADOOP-16384: inconsisent DB after test runs. * lots of work on dumping the metastore and S3: you get (metastore treewalk, S3A treewalk, s3A flat list, raw s3) all in adjacent files * prune in DDB no longer mark deleted ancestors as non-auth. I believe this was the cause of the prune confict failures I've been seeing * HADOOP-16397: prune command has -tombstone option with tests and docs. Change-Id: Id838990833fe52184d902a40a5591febd9f339d8 --- .../launcher/ServiceLaunchException.java | 3 +- .../org/apache/hadoop/fs/s3a/Listing.java | 11 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 2 +- .../fs/s3a/s3guard/DumpS3GuardTable.java | 282 ++++++++++++++---- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 14 +- .../hadoop/fs/s3a/s3guard/MetadataStore.java | 4 +- .../PathMetadataDynamoDBTranslation.java | 2 +- .../fs/s3a/s3guard/PathOrderComparators.java | 4 +- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 13 +- .../site/markdown/tools/hadoop-aws/s3guard.md | 19 +- .../s3guard/AbstractS3GuardToolTestBase.java | 13 + .../s3guard/ITestS3GuardRootOperations.java | 20 +- .../s3a/s3guard/ITestS3GuardToolDynamoDB.java | 9 +- 13 files changed, 317 insertions(+), 79 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLaunchException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLaunchException.java index d62496547daea..fdb60236fe645 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLaunchException.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLaunchException.java @@ -88,7 +88,8 @@ public ServiceLaunchException(int exitCode, String format, Object... args) { * @param format format for message to use in exception * @param args list of arguments */ - public ServiceLaunchException(int exitCode, Throwable cause, String format, Object... args) { + public ServiceLaunchException(int exitCode, Throwable cause, + String format, Object... args) { super(exitCode, String.format(Locale.ENGLISH, format, args), cause); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java index b62c4569b6e62..54f8fc618c2d0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java @@ -21,6 +21,8 @@ import com.amazonaws.AmazonClientException; import com.amazonaws.services.s3.model.S3ObjectSummary; import com.google.common.annotations.VisibleForTesting; + +import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; @@ -50,6 +52,7 @@ /** * Place for the S3A listing classes; keeps all the small classes under control. */ +@InterfaceAudience.Private public class Listing { private final S3AFileSystem owner; @@ -87,7 +90,7 @@ ProvidedFileStatusIterator createProvidedFileStatusIterator( * @return the iterator * @throws IOException IO Problems */ - FileStatusListingIterator createFileStatusListingIterator( + public FileStatusListingIterator createFileStatusListingIterator( Path listPath, S3ListRequest request, PathFilter filter, @@ -110,7 +113,7 @@ FileStatusListingIterator createFileStatusListingIterator( * @throws IOException IO Problems */ @Retries.RetryRaw - FileStatusListingIterator createFileStatusListingIterator( + public FileStatusListingIterator createFileStatusListingIterator( Path listPath, S3ListRequest request, PathFilter filter, @@ -129,7 +132,7 @@ FileStatusListingIterator createFileStatusListingIterator( * @return a new remote iterator */ @VisibleForTesting - LocatedFileStatusIterator createLocatedFileStatusIterator( + public LocatedFileStatusIterator createLocatedFileStatusIterator( RemoteIterator statusIterator) { return new LocatedFileStatusIterator(statusIterator); } @@ -789,7 +792,7 @@ public boolean accept(FileStatus status) { * Accept all entries except the base path and those which map to S3N * pseudo directory markers. */ - static class AcceptAllButSelfAndS3nDirs implements FileStatusAcceptor { + public static class AcceptAllButSelfAndS3nDirs implements FileStatusAcceptor { /** Base path. */ private final Path qualifiedPath; 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 3ae4f1fe1139e..4b5cadec85836 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 @@ -2439,7 +2439,7 @@ public FileStatus[] innerListStatus(Path f) throws FileNotFoundException, * @return the request */ @VisibleForTesting - S3ListRequest createListObjectsRequest(String key, + public S3ListRequest createListObjectsRequest(String key, String delimiter) { return createListObjectsRequest(key, delimiter, null); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java index 48767a8852eb7..5ce929b30a8e4 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java @@ -27,7 +27,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Comparator; import java.util.Date; import java.util.LinkedList; import java.util.List; @@ -39,17 +38,24 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.s3a.Listing; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; +import org.apache.hadoop.fs.s3a.S3ListRequest; import org.apache.hadoop.service.Service; import org.apache.hadoop.service.launcher.AbstractLaunchableService; import org.apache.hadoop.service.launcher.LauncherExitCodes; import org.apache.hadoop.service.launcher.ServiceLaunchException; import org.apache.hadoop.service.launcher.ServiceLauncher; +import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.ExitUtil; +import static org.apache.hadoop.fs.s3a.S3AUtils.ACCEPT_ALL; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_USAGE; /** @@ -63,7 +69,8 @@ public class DumpS3GuardTable extends AbstractLaunchableService { public static final String NAME = "DumpS3GuardTable"; - private static String USAGE_MESSAGE = NAME + " "; + private static final String USAGE_MESSAGE = NAME + + " "; private List arguments; @@ -90,7 +97,10 @@ public List getArguments() { } /** - * {@inheritDoc} + * Dump the filesystem and the metastore. + * @return the exit code. + * @throws ServiceLaunchException on failure. + * @throws IOException IO failure. */ @Override public int execute() throws ServiceLaunchException, IOException { @@ -102,7 +112,7 @@ public int execute() throws ServiceLaunchException, IOException { String fsURI = arguments.get(0); String destFilename = arguments.get(1); Configuration conf = getConfig(); - URI uri = null; + URI uri; try { uri = new URI(fsURI); } catch (URISyntaxException e) { @@ -120,31 +130,156 @@ public int execute() throws ServiceLaunchException, IOException { "Filesystem " + fsURI + "does not have a DynamoDB metadata store: " + ms); store = (DynamoDBMetadataStore) ms; -// Table table = store.getTable(); + Path basePath = fs.qualify(new Path(uri)); - File destFile = new File(destFilename).getCanonicalFile(); + final File destFile = new File(destFilename + "-store.csv") + .getCanonicalFile(); LOG.info("Writing Store details to {}", destFile); - try (CsvFile csv = new CsvFile(null, - new PrintWriter(destFile), - "\t", "\n", "\"")) { - csv.row(CsvFile.ALL_QUOTES, - "path", "type", "is_auth_dir", "deleted", - "is_empty_dir", "len", "updated", "updated_s", - "last_modified", "last_modified_s", "etag", "version"); - - Path basePath = fs.qualify(new Path(uri)); + try (CsvFile csv = new CsvFile(destFile); + DurationInfo ignored = new DurationInfo(LOG, "List metastore")) { + csv.header(); + LOG.info("Base path: {}", basePath); - DirListingMetadata children = store.listChildren(basePath); - dumpRecursively(csv, children); + dumpMetastore(csv, basePath); + } + final File treewalkFile = new File(destFilename + "-tree.csv") + .getCanonicalFile(); + + try (CsvFile csv = new CsvFile(treewalkFile); + DurationInfo ignored = new DurationInfo(LOG, + "Treewalk to %s", treewalkFile)) { + csv.header(); + treewalkFilesystem(csv, fs, basePath); + } + final File flatlistFile = new File( + destFilename + "-flat.csv").getCanonicalFile(); + + try (CsvFile csv = new CsvFile(flatlistFile); + DurationInfo ignored = new DurationInfo(LOG, + "Flat list to %s", flatlistFile)) { + csv.header(); + listStatusFilesystem(csv, fs, basePath); + } + final File rawFile = new File( + destFilename + "-raw.csv").getCanonicalFile(); + + try (CsvFile csv = new CsvFile(rawFile); + DurationInfo ignored = new DurationInfo(LOG, + "Raw dump to %s", rawFile)) { + csv.header(); + rawDump(csv, fs); } + return LauncherExitCodes.EXIT_SUCCESS; - } catch (Exception e) { + } catch (IOException | RuntimeException e) { LOG.error("failure", e); throw e; } } + /** + * Dump the filesystem via a recursive treewalk. + * If metastore entries mark directories as deleted, this + * walk will not explore them. + * @param csv destination. + * @param fs filesystem. + * @return number of entries found. + * @throws IOException IO failure. + */ + + protected int treewalkFilesystem( + final CsvFile csv, + final S3AFileSystem fs, + final Path path) throws IOException { + int count = 1; + FileStatus[] fileStatuses = fs.listStatus(path); + // entries + for (FileStatus fileStatus : fileStatuses) { + csv.entry((S3AFileStatus) fileStatus); + } + for (FileStatus fileStatus : fileStatuses) { + if (fileStatus.isDirectory() + && !(fileStatus.getPath().equals(path))) { + count += treewalkFilesystem(csv, fs, fileStatus.getPath()); + } else { + count++; + } + } + return count; + } + + /** + * Dump the filesystem via a recursive listStatus call. + * @param csv destination. + * @param fs filesystem. + * @return number of entries found. + * @throws IOException IO failure. + */ + protected int listStatusFilesystem( + final CsvFile csv, + final S3AFileSystem fs, + final Path path) throws IOException { + int count = 0; + RemoteIterator iterator = fs + .listFilesAndEmptyDirectories(path, true); + while (iterator.hasNext()) { + S3ALocatedFileStatus status = iterator.next(); + csv.entry(toS3AStatus(status)); + } + return count; + } + + private S3AFileStatus toS3AStatus(S3ALocatedFileStatus located) { + return new S3AFileStatus( + located.getLen(), + located.getModificationTime(), + located.getPath(), + located.getBlockSize(), + located.getOwner(), + located.getETag(), + located.getVersionId()); + } + + /** + * Dump the raw S3 Object Store. + * @param csv destination. + * @param fs filesystem. + * @return number of entries found. + * @throws IOException IO failure. + */ + protected int rawDump( + final CsvFile csv, + final S3AFileSystem fs) throws IOException { + Path rootPath = fs.qualify(new Path("/")); + Listing listing = new Listing(fs); + S3ListRequest request = fs.createListObjectsRequest("", null); + int count = 0; + RemoteIterator st = + listing.createFileStatusListingIterator(rootPath, request, + ACCEPT_ALL, + new Listing.AcceptAllButSelfAndS3nDirs(rootPath)); + while (st.hasNext()) { + count++; + S3AFileStatus next = st.next(); + LOG.debug("[{}] {}", count, next); + csv.entry(next); + } + LOG.info("entry count: {}", count); + return count; + } + + protected void dumpMetastore(final CsvFile csv, + final Path basePath) throws IOException { + dumpRecursively(csv, store.listChildren(basePath)); + } + /** + * Recursive Store Dump. + * @param csv open CSV file. + * @param dir directory listing + * @return (directories, files) + * @throws IOException failure + */ private Pair dumpRecursively( CsvFile csv, DirListingMetadata dir) throws IOException { int files = 0, dirs = 1; @@ -176,42 +311,27 @@ private Pair dumpRecursively( return Pair.of(dirs, files); } - /** least significant bit is used for first column; 1 mean 'quote'. */ - public static final int QUOTE_MAP = 0b1110_1001_1111; private void dumpEntry(CsvFile csv, DDBPathMetadata md) { - S3AFileStatus fileStatus = md.getFileStatus(); LOG.info("{}", md.prettyPrint()); - csv.row(QUOTE_MAP, - fileStatus.getPath().toString(), - fileStatus.isDirectory() ? "dir" : "file", - md.isAuthoritativeDir(), - md.isDeleted(), - md.isEmptyDirectory().name(), - fileStatus.getLen(), - md.getLastUpdated(), - stringify(md.getLastUpdated()), - fileStatus.getModificationTime(), - stringify(fileStatus.getModificationTime()), - fileStatus.getETag(), - fileStatus.getVersionId()); + csv.entry(md); } - private String stringify(long millis) { + private static String stringify(long millis) { return new Date(millis).toString(); } - private void require(boolean condition, String error) { + private static void require(boolean condition, String error) { if (!condition) { throw fail(error); } } - private ServiceLaunchException fail(String message, Throwable ex) { + private static ServiceLaunchException fail(String message, Throwable ex) { return new ServiceLaunchException(LauncherExitCodes.EXIT_FAIL, message, ex); } - private ServiceLaunchException fail(String message) { + private static ServiceLaunchException fail(String message) { return new ServiceLaunchException(LauncherExitCodes.EXIT_FAIL, message); } @@ -257,10 +377,10 @@ public static void serviceMain(List argsList) { } /** - * + * Entry point to dump. * @param filesystem filesystem path * @param conf configuration to instantiate with. - * @param destFile output file. + * @param destFile base name of the output files. * @throws ExitUtil.ExitException failure. */ public static void dumpS3GuardStore(String filesystem, @@ -295,6 +415,9 @@ private static class CsvFile implements Closeable { /** constant to quote all columns. */ public static final long ALL_QUOTES = 0x7fffffff; + /** least significant bit is used for first column; 1 mean 'quote'. */ + public static final int ROW_QUOTE_MAP = 0b1110_1001_1111; + /** quote nothing: {@value}. */ public static final long NO_QUOTES = 0; @@ -308,18 +431,7 @@ private static class CsvFile implements Closeable { private final String quote; - CsvFile(final FileSystem fs, - final Path path, - final boolean overwrite, - final String separator, - final String eol, - final String quote) throws IOException { - this(path, - new PrintWriter(fs.create(path, overwrite)), - separator, eol, quote); - } - - public CsvFile( + private CsvFile( final Path path, final PrintWriter out, final String separator, @@ -332,6 +444,11 @@ public CsvFile( this.out = Preconditions.checkNotNull(out); } + private CsvFile(File file) throws IOException { + this(null, + new PrintWriter(file, "UTF-8"), "\t", "\n", "\""); + } + /** * Close the file, if not already done. * @throws IOException on a failure. @@ -402,5 +519,66 @@ public CsvFile line(String line) { public PrintWriter getOut() { return out; } + + /** + * Print the header. + */ + void header() { + row(CsvFile.ALL_QUOTES, + "path", + "type", + "deleted", + "is_auth_dir", + "is_empty_dir", + "len", + "updated", + "updated_s", + "last_modified", + "last_modified_s", + "etag", + "version"); + } + + /** + * Add a metadata entry. + * @param md metadata. + */ + void entry(DDBPathMetadata md) { + S3AFileStatus fileStatus = md.getFileStatus(); + row(ROW_QUOTE_MAP, + fileStatus.getPath().toString(), + fileStatus.isDirectory() ? "dir" : "file", + md.isDeleted(), + md.isAuthoritativeDir(), + md.isEmptyDirectory().name(), + fileStatus.getLen(), + md.getLastUpdated(), + stringify(md.getLastUpdated()), + fileStatus.getModificationTime(), + stringify(fileStatus.getModificationTime()), + fileStatus.getETag(), + fileStatus.getVersionId()); + } + + /** + * filesystem entry: no metadata. + * @param fileStatus file status + */ + void entry(S3AFileStatus fileStatus) { + row(ROW_QUOTE_MAP, + fileStatus.getPath().toString(), + fileStatus.isDirectory() ? "dir" : "file", + "false", + "", + fileStatus.isEmptyDirectory().name(), + fileStatus.getLen(), + "", + "", + fileStatus.getModificationTime(), + stringify(fileStatus.getModificationTime()), + fileStatus.getETag(), + fileStatus.getVersionId()); + } } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java index 9f0631309fd4d..a8f8c9c6b02ae 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java @@ -107,6 +107,7 @@ import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.BlockingThreadPoolExecutorService; +import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.ReflectionUtils; import static org.apache.hadoop.fs.s3a.Constants.*; @@ -1455,7 +1456,10 @@ public void prune(PruneMode pruneMode, long cutoff) throws IOException { @Retries.RetryTranslated public void prune(PruneMode pruneMode, long cutoff, String keyPrefix) throws IOException { - LOG.debug("Prune files under {} with age {}", keyPrefix, cutoff); + LOG.debug("Prune {} under {} with age {}", + pruneMode == PruneMode.ALL_BY_MODTIME + ? "files and tombstones" : "tombstones", + keyPrefix, cutoff); final ItemCollection items = expiredFiles(pruneMode, cutoff, keyPrefix); innerPrune(keyPrefix, items); @@ -1465,7 +1469,9 @@ private void innerPrune(String keyPrefix, ItemCollection items) throws IOException { int itemCount = 0; try (AncestorState state = initiateBulkWrite( - BulkOperationState.OperationType.Prune, null)) { + BulkOperationState.OperationType.Prune, null); + DurationInfo ignored = + new DurationInfo(LOG, "Pruning DynamoDB Store")) { ArrayList deletionBatch = new ArrayList<>(S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT); long delay = conf.getTimeDuration( @@ -1478,6 +1484,7 @@ private void innerPrune(String keyPrefix, ItemCollection items) DDBPathMetadata md = PathMetadataDynamoDBTranslation .itemToPathMetadata(item, username); Path path = md.getFileStatus().getPath(); + LOG.debug("Prune entry {}", path); deletionBatch.add(path); // add parent path of what we remove if it has not @@ -1548,7 +1555,8 @@ private void removeAuthoritativeDirFlag( return null; } DDBPathMetadata ddbPathMetadata = get(path); - if(ddbPathMetadata == null) { + if (ddbPathMetadata == null || ddbPathMetadata.isDeleted()) { + // there is no entry, or it has actually expired return null; } LOG.debug("Setting false isAuthoritativeDir on {}", ddbPathMetadata); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java index 397d23aa34323..3767edcbced39 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java @@ -301,7 +301,7 @@ void put(DirListingMetadata meta, * * * - * @param pruneMode + * @param pruneMode Prune Mode * @param cutoff Oldest time to allow (UTC) * @throws IOException if there is an error * @throws UnsupportedOperationException if not implemented @@ -313,7 +313,7 @@ void prune(PruneMode pruneMode, long cutoff) throws IOException, * Same as {@link MetadataStore#prune(PruneMode, long)}, but with an * additional keyPrefix parameter to filter the pruned keys with a prefix. * - * @param pruneMode + * @param pruneMode Prune Mode * @param cutoff Oldest time to allow (UTC) * @param keyPrefix The prefix for the keys that should be removed * @throws IOException if there is an error diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java index ba691d63479bb..c067ec56410bd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java @@ -298,7 +298,7 @@ public static String pathToParentKey(Path path) { URI uri = path.toUri(); String bucket = uri.getHost(); Preconditions.checkArgument(!StringUtils.isEmpty(bucket), - "Path missing bucket"); + "Path missing bucket %s", path); String pKey = "/" + bucket + uri.getPath(); // Strip trailing slash diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java index ddbe2f419ee0a..a3a7967caf79b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java @@ -115,12 +115,12 @@ public int compare(final Path pathL, final Path pathR) { /** * Compare on path status. */ - public static final class PathMetadataComparator implements + private static final class PathMetadataComparator implements Comparator, Serializable { private final Comparator inner; - public PathMetadataComparator(final Comparator inner) { + private PathMetadataComparator(final Comparator inner) { this.inner = inner; } 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 c9fd6731a0f40..260fb7978049a 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 @@ -1026,12 +1026,16 @@ static class Prune extends S3GuardTool { public static final String PURPOSE = "truncate older metadata from " + "repository " + DATA_IN_S3_IS_PRESERVED;; + + public static final String TOMBSTONE = "tombstone"; + private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" + "\t" + PURPOSE + "\n\n" + "Common options:\n" + " -" + META_FLAG + " URL - Metadata repository details " + "(implementation-specific)\n" + "Age options. Any combination of these integer-valued options:\n" + + "[" + TOMBSTONE + "]\n" + AGE_OPTIONS_USAGE + "\n" + "Amazon DynamoDB-specific options:\n" + " -" + REGION_FLAG + " REGION - Service region for connections\n" + @@ -1041,7 +1045,7 @@ static class Prune extends S3GuardTool { " is not supported."; Prune(Configuration conf) { - super(conf); + super(conf, TOMBSTONE); addAgeOptions(); } @@ -1098,8 +1102,13 @@ public int run(String[] args, PrintStream out) throws keyPrefix = PathMetadataDynamoDBTranslation.pathToParentKey(path); } + MetadataStore.PruneMode mode + = MetadataStore.PruneMode.ALL_BY_MODTIME; + if (getCommandFormat().getOpt(TOMBSTONE)) { + mode = MetadataStore.PruneMode.TOMBSTONES_BY_LASTUPDATED; + } try { - getStore().prune(MetadataStore.PruneMode.ALL_BY_MODTIME, divide, + getStore().prune(mode, divide, keyPrefix); } catch (UnsupportedOperationException e){ errorln("Prune operation not supported in metadata store."); diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md index 1e83e5435d352..cb0fd139262df 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md @@ -796,7 +796,7 @@ time" is older than the specified age. ```bash hadoop s3guard prune [-days DAYS] [-hours HOURS] [-minutes MINUTES] - [-seconds SECONDS] [-meta URI] ( -region REGION | s3a://BUCKET ) + [-seconds SECONDS] [-tombstone] [-meta URI] ( -region REGION | s3a://BUCKET ) ``` A time value of hours, minutes and/or seconds must be supplied. @@ -807,6 +807,13 @@ in the S3 Bucket. 1. If an S3A URI is supplied, only the entries in the table specified by the URI and older than a specific age are deleted. + +The `-tombstone` option instructs the operation to only purge "tombstones", +markers of deleted files. These tombstone markers are only used briefly, +to indicate that a recently deleted file should not be found in listings. +As a result, there is no adverse consequences in regularly pruning old +tombstones. + Example ```bash @@ -817,18 +824,18 @@ Deletes all entries in the S3Guard table for files older than seven days from the table associated with `s3a://ireland-1`. ```bash -hadoop s3guard prune -days 7 s3a://ireland-1/path_prefix/ +hadoop s3guard prune -tombstone -days 7 s3a://ireland-1/path_prefix/ ``` -Deletes all entries in the S3Guard table for files older than seven days from -the table associated with `s3a://ireland-1` and with the prefix "path_prefix" +Deletes all entries in the S3Guard table for tombstones older than seven days from +the table associated with `s3a://ireland-1` and with the prefix `path_prefix` ```bash hadoop s3guard prune -hours 1 -minutes 30 -meta dynamodb://ireland-team -region eu-west-1 ``` -Delete all entries more than 90 minutes old from the table "ireland-team" in -the region "eu-west-1". +Delete all entries more than 90 minutes old from the table "`ireland-team"` in +the region `eu-west-1`. ### Tune the I/O capacity of the DynamoDB Table, `s3guard set-capacity` diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java index 5315474013541..b918500ef98d6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java @@ -322,6 +322,19 @@ public void testPruneCommandCLI() throws Exception { testPath.toString()); } + @Test + public void testPruneCommandTombstones() throws Exception { + Path testPath = path("testPruneCommandTombstones"); + getFileSystem().mkdirs(testPath); + getFileSystem().delete(testPath, true); + S3GuardTool.Prune cmd = new S3GuardTool.Prune(getFileSystem().getConf()); + cmd.setMetadataStore(ms); + exec(cmd, + "prune", "-" + S3GuardTool.Prune.TOMBSTONE, + "-seconds", "0", + testPath.toString()); + } + @Test public void testPruneCommandConf() throws Exception { getConfiguration().setLong(Constants.S3GUARD_CLI_PRUNE_AGE, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java index 70f9623a2816b..9b40343b1dcbd 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java @@ -27,6 +27,7 @@ import org.junit.runners.MethodSorters; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; @@ -121,11 +122,26 @@ public void test_100_FilesystemPrune() throws Throwable { @Test - public void test_200_MetastorePrune() throws Throwable { + public void test_200_MetastorePruneTombstones() throws Throwable { + describe("Execute prune against a dynamo URL"); + S3AFileSystem fs = getFileSystem(); + Configuration conf = fs.getConf(); + int result = S3GuardTool.run(conf, + S3GuardTool.Prune.NAME, + "-tombstone", + "-meta", checkNotNull(metastoreUriStr), + "-seconds", "1", + fs.qualify(new Path("/")).toString()); + Assertions.assertThat(result) + .describedAs("Result of prune %s", fsUriStr) + .isEqualTo(0); + } + + @Test + public void test_300_MetastorePrune() throws Throwable { describe("Execute prune against a dynamo URL"); S3AFileSystem fs = getFileSystem(); Configuration conf = fs.getConf(); - S3GuardTool.Prune cmd = new S3GuardTool.Prune(conf); int result = S3GuardTool.run(conf, S3GuardTool.Prune.NAME, "-meta", checkNotNull(metastoreUriStr), 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 7514df7244570..e85612cdceca2 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 @@ -23,7 +23,6 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.InputStreamReader; -import java.io.Reader; import java.nio.charset.Charset; import java.util.HashMap; import java.util.List; @@ -289,15 +288,19 @@ public void testDestroyUnknownTable() throws Throwable { @Test public void testDumpTable() throws Throwable { - File destFile = File.createTempFile("dump", ".csv"); + String target = System.getProperty("test.build.dir", "target"); + File buildDir = new File(target).getAbsoluteFile(); + String name = "dump-table"; + File destFile = new File(buildDir, name); describe("Dumping metastore to {}", destFile); S3AFileSystem fs = getFileSystem(); DumpS3GuardTable.dumpS3GuardStore( fs.getUri().toString(), fs.getConf(), destFile); + File storeFile = new File(buildDir, name + "-store.csv"); try (BufferedReader in = new BufferedReader(new InputStreamReader( - new FileInputStream(destFile), Charset.forName("UTF-8")))) { + new FileInputStream(storeFile), Charset.forName("UTF-8")))) { for (String line : IOUtils.readLines(in)) { LOG.info(line); } From e83eb0e852578ed65d63f7212fe4a3bf24af66cb Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 26 Jun 2019 18:13:40 +0100 Subject: [PATCH 04/23] HADOOP-16384: fix compilation error; checkstyle was mistaken Change-Id: I7b67576fb46b37b4f5948be1229d1b4a12dc62ba --- .../apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java index a3a7967caf79b..cbf41b4bab099 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java @@ -115,12 +115,12 @@ public int compare(final Path pathL, final Path pathR) { /** * Compare on path status. */ - private static final class PathMetadataComparator implements + static final class PathMetadataComparator implements Comparator, Serializable { private final Comparator inner; - private PathMetadataComparator(final Comparator inner) { + PathMetadataComparator(final Comparator inner) { this.inner = inner; } From 6751629fe8b3a86e1c1a693713315954066ab82e Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 1 Jul 2019 13:24:59 +0100 Subject: [PATCH 05/23] HADOOP-16384 prune resilience: ongoing testing Logic to create an S3ALocatedFileStatus into a S3AFileStatus pulled into S3ALocatedFileStatus and used in both prod and dev code More testing in ITestDynamoDBMetadataStore Still seeing problems with full suite test runs. Change-Id: I093c789d63b4362765cf4b721268c417ec4657da Plan: add scan for listing all items in a bucket, add checks that no orphans are there after various bulk operations --- .../hadoop/fs/s3a/S3ALocatedFileStatus.java | 15 ++ .../fs/s3a/s3guard/DumpS3GuardTable.java | 13 +- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 51 +++++-- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 8 +- .../s3guard/ITestDynamoDBMetadataStore.java | 133 ++++++++++++++---- .../fs/s3a/s3guard/MetadataStoreTestBase.java | 38 +++++ 6 files changed, 207 insertions(+), 51 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ALocatedFileStatus.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ALocatedFileStatus.java index d3ca2610e225b..f78e11c89924a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ALocatedFileStatus.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ALocatedFileStatus.java @@ -60,4 +60,19 @@ public boolean equals(Object o) { public int hashCode() { return super.hashCode(); } + + /** + * Generate an S3AFileStatus instance, including etag and + * version ID, if present. + */ + public S3AFileStatus toS3AFileStatus() { + return new S3AFileStatus( + getLen(), + getModificationTime(), + getPath(), + getBlockSize(), + getOwner(), + getETag(), + getVersionId()); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java index 5ce929b30a8e4..17afcee5e5539 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java @@ -224,22 +224,11 @@ protected int listStatusFilesystem( .listFilesAndEmptyDirectories(path, true); while (iterator.hasNext()) { S3ALocatedFileStatus status = iterator.next(); - csv.entry(toS3AStatus(status)); + csv.entry(status.toS3AFileStatus()); } return count; } - private S3AFileStatus toS3AStatus(S3ALocatedFileStatus located) { - return new S3AFileStatus( - located.getLen(), - located.getModificationTime(), - located.getPath(), - located.getBlockSize(), - located.getOwner(), - located.getETag(), - located.getVersionId()); - } - /** * Dump the raw S3 Object Store. * @param csv destination. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java index a8f8c9c6b02ae..ccaafdb4b6039 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java @@ -833,7 +833,7 @@ private Collection completeAncestry( if (!oldEntry.getFileStatus().isDirectory() || !entry.getFileStatus().isDirectory()) { // check for and warn if the existing bulk operation overwrote it. - // this should never occur outside tests explicitly crating it + // this should never occur outside tests explicitly creating it LOG.warn("Overwriting a S3Guard file created in the operation: {}", oldEntry); LOG.warn("With new entry: {}", entry); @@ -1321,7 +1321,11 @@ static S3AFileStatus makeDirStatus(Path f, String owner) { public void put( final DirListingMetadata meta, @Nullable final BulkOperationState operationState) throws IOException { - LOG.debug("Saving to table {} in region {}: {}", tableName, region, meta); + LOG.debug("Saving {} dir meta for {} to table {} in region {}: {}", + tableName, + meta.isAuthoritative() ? "auth" : "nonauth", + meta.getPath(), + tableName, region, meta); // directory path Path path = meta.getPath(); @@ -1484,13 +1488,19 @@ private void innerPrune(String keyPrefix, ItemCollection items) DDBPathMetadata md = PathMetadataDynamoDBTranslation .itemToPathMetadata(item, username); Path path = md.getFileStatus().getPath(); + boolean tombstone = md.isDeleted(); LOG.debug("Prune entry {}", path); deletionBatch.add(path); - // add parent path of what we remove if it has not - // already been processed + // add parent path of item so it can be marked as non-auth. + // this is only done if + // * it has not already been processed + // * the entry pruned is not a tombstone (no need to update + // * the file is not in the root dir Path parentPath = path.getParent(); - if (parentPath != null && !clearedParentPathSet.contains(parentPath)) { + if (!tombstone + && parentPath != null + && !clearedParentPathSet.contains(parentPath)) { parentPathSet.add(parentPath); } @@ -1534,6 +1544,20 @@ private void innerPrune(String keyPrefix, ItemCollection items) /** * Remove the Authoritative Directory Marker from a set of paths, if * those paths are in the store. + *

+ * This operation is onlyfor pruning; it does not raise an error + * if, during the prune phase, the table appears inconsistent. + * This is not unusual as it can happen in a number of ways + *

    + *
  1. The state of the table changes during a slow prune operation which + * deliberately inserts pauses to avoid overloading prepaid IO capacity. + *
  2. + *
  3. Tombstone markers have been left in the table after many other + * operations have taken place, including deleting/replacing + * parents.
  4. + *
+ *

+ * * If an exception is raised in the get/update process, then the exception * is caught and only rethrown after all the other paths are processed. * This is to ensure a best-effort attempt to update the store. @@ -1555,11 +1579,22 @@ private void removeAuthoritativeDirFlag( return null; } DDBPathMetadata ddbPathMetadata = get(path); - if (ddbPathMetadata == null || ddbPathMetadata.isDeleted()) { - // there is no entry, or it has actually expired + if (ddbPathMetadata == null) { + // there is no entry. + LOG.debug("No parent {}; skipping", path); + return null; + } + if (ddbPathMetadata.isDeleted()) { + // the parent itself is deleted + LOG.debug("Parent has been deleted {}; skipping", path); + return null; + } + if (!ddbPathMetadata.getFileStatus().isDirectory()) { + // the parent itself is deleted + LOG.debug("Parent is not a directory {}; skipping", path); return null; } - LOG.debug("Setting false isAuthoritativeDir on {}", ddbPathMetadata); + LOG.debug("Setting isAuthoritativeDir==false on {}", ddbPathMetadata); ddbPathMetadata.setAuthoritativeDir(false); return ddbPathMetadata; } catch (IOException e) { 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 260fb7978049a..defcfe19ca07f 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 @@ -755,13 +755,7 @@ private long importDir(FileStatus status) throws IOException { located.getOwner()); dirCache.add(child.getPath()); } else { - child = new S3AFileStatus(located.getLen(), - located.getModificationTime(), - located.getPath(), - located.getBlockSize(), - located.getOwner(), - located.getETag(), - located.getVersionId()); + child = located.toS3AFileStatus(); } putParentsIfNotPresent(child, operationState); S3Guard.putWithTtl(getStore(), diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java index 03ebe1ee76a5b..7194d352ba129 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java @@ -94,6 +94,8 @@ */ public class ITestDynamoDBMetadataStore extends MetadataStoreTestBase { + public static final int MINUTE = 60_000; + public ITestDynamoDBMetadataStore() { super(); } @@ -1125,13 +1127,9 @@ public void testPruneAgainstInvalidTable() throws Throwable { // over the subdirectory long now = getTime(); - long oldTime = now - 60_000; + long oldTime = now - MINUTE; putFile(subdir, oldTime, null); - final DDBPathMetadata subDirAsFile = ms.get(subDirPath); - - Assertions.assertThat(subDirAsFile.getFileStatus().isFile()) - .describedAs("Subdirectory entry %s is now file", subDirMetadataOrig) - .isTrue(); + getFile(subdir); Path basePath = strToPath(base); DirListingMetadata listing = ms.listChildren(basePath); @@ -1147,13 +1145,13 @@ public void testPruneAgainstInvalidTable() throws Throwable { Assertions.assertThat(status.isFile()) .as("Entry %s", (Object)pm) .isTrue(); - DDBPathMetadata subFilePm = checkNotNull(ms.get(subFilePath)); - LOG.info("Pruning"); + getNonNull(subFile); + LOG.info("Pruning"); // now prune ms.prune(PruneMode.ALL_BY_MODTIME, - now + 60_000, subdir); - DDBPathMetadata prunedFile = ms.get(subFilePath); + now + MINUTE, subdir); + ms.get(subFilePath); final PathMetadata subDirMetadataFinal = getNonNull(subdir); @@ -1165,7 +1163,7 @@ public void testPruneAgainstInvalidTable() throws Throwable { @Test public void testPutFileDirectlyUnderTombstone() throws Throwable { - describe("Put a file under a tombstone"); + describe("Put a file under a tombstone; verify the tombstone"); String base = "/testPutFileDirectlyUnderTombstone"; long now = getTime(); putTombstone(base, now, null); @@ -1175,35 +1173,102 @@ public void testPutFileDirectlyUnderTombstone() throws Throwable { .isTrue(); String child = base + "/file"; putFile(child, now, null); - PathMetadata baseMeta2 = get(base); - Assertions.assertThat(baseMeta2.isDeleted()) - .as("Metadata %s", baseMeta2) - .isFalse(); + getDirectory(base); + } + + @Test + public void testPruneTombstoneUnderTombstone() throws Throwable { + describe("Put a tombsteone under a tombstone, prune the pair"); + String base = "/testPruneTombstoneUnderTombstone"; + long now = getTime(); + String dir = base + "/dir"; + putTombstone(dir, now, null); + assertIsTombstone(dir); + // parent dir is created + assertCached(base); + String child = dir + "/file"; + String child2 = dir + "/file2"; + + // this will actually mark the parent as a dir, + // so that lists of that dir will pick up the tombstone + putTombstone(child, now, null); + getDirectory(dir); + // put a tombstone + putTombstone(dir, now, null); + putFile(child2, now, null); + getDynamoMetadataStore().prune(PruneMode.TOMBSTONES_BY_LASTUPDATED, + now + MINUTE); + // the child is gone + assertNotFound(child); + // *AND* the parent dir has not been created + assertNotFound(dir); + // the child2 entry is still there, though it's now orphan (the store isn't + // meeting the rule "all entries must have a parent which exists" + getFile(child2); + // a full prune will still find and delete it, as this + // doesn't walk the tree + getDynamoMetadataStore().prune(PruneMode.ALL_BY_MODTIME, + now + MINUTE); + assertNotFound(child2); + assertNotFound(dir); + } + + @Test + public void testPruneFileUnderTombstone() throws Throwable { + describe("Put a file under a tombstone, prune the pair"); + String base = "/testPruneFileUnderTombstone"; + long now = getTime(); + String dir = base + "/dir"; + putTombstone(dir, now, null); + assertIsTombstone(dir); + // parent dir is created + assertCached(base); + String child = dir + "/file"; + + // this will actually mark the parent as a dir, + // so that lists of that dir will pick up the tombstone + putFile(child, now, null); + // dir is reinstated + getDirectory(dir); + + // put a tombstone + putTombstone(dir, now, null); + // prune all entries + getDynamoMetadataStore().prune(PruneMode.ALL_BY_MODTIME, + now + MINUTE); + // the child is gone + assertNotFound(child); + + // *AND* the parent dir has not been created + assertNotFound(dir); } + /** + * Keep in sync with code changes in S3AFileSystem.finishedWrite() so that + * code can be tested here. + * @throws Throwable + */ @Test public void testPutFileDeepUnderTombstone() throws Throwable { describe("Put a file two levels under a tombstone"); String base = "/testPutFileDeepUnderTombstone"; - String subdir = base + "/subdir"; + String dir = base + "/dir"; long now = getTime(); // creating a file MUST create its parents - String child = subdir + "/file"; + String child = dir + "/file"; Path childPath = strToPath(child); putFile(child, now, null); getFile(child); - getDirectory(subdir); + getDirectory(dir); getDirectory(base); // now put the tombstone putTombstone(base, now, null); - PathMetadata baseMeta1 = getNonNull(base); - Assertions.assertThat(baseMeta1.isDeleted()) - .as("Metadata %s", baseMeta1) - .isTrue(); - - // this is the same ordering as S3FileSystem.finishedWrite() + assertIsTombstone(base); + /*- --------------------------------------------*/ + /* Begin S3FileSystem.finishedWrite() sequence. */ + /* ---------------------------------------------*/ AncestorState ancestorState = getDynamoMetadataStore() .initiateBulkWrite(BulkOperationState.OperationType.Put, childPath); @@ -1213,8 +1278,28 @@ public void testPutFileDeepUnderTombstone() throws Throwable { ancestorState); // now write the file again. putFile(child, now, ancestorState); + /* -------------------------------------------*/ + /* End S3FileSystem.finishedWrite() sequence. */ + /* -------------------------------------------*/ + + getFile(child); // the ancestor will now exist. + getDirectory(dir); getDirectory(base); } + + /** + * Assert that an entry exists and is a directory. + * @param pathStr path + * @throws IOException IO failure. + */ + protected PathMetadata verifyAuthDirStatus(String pathStr, boolean authDirFlag) + throws IOException { + DDBPathMetadata md = (DDBPathMetadata) getDirectory(pathStr); + assertEquals("isAuthoritativeDir() mismatch in " + md, + authDirFlag, + md.isAuthoritativeDir()); + return md; + } } 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 75bd760e1fcb7..fdf70898b101a 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 @@ -1050,6 +1050,11 @@ protected PathMetadata getNonNull(final String pathStr) throws IOException { return checkNotNull(get(pathStr), "No metastore entry for %s", pathStr); } + /** + * Assert that either a path has no entry or that it is marked as deleted. + * @param pathStr path + * @throws IOException IO failure. + */ protected void assertDeleted(String pathStr) throws IOException { PathMetadata meta = get(pathStr); boolean cached = meta != null && !meta.isDeleted(); @@ -1073,6 +1078,39 @@ protected PathMetadata verifyCached(final String pathStr) throws IOException { return meta; } + /** + * Assert that an entry exists and is a file. + * @param pathStr path + * @throws IOException IO failure. + */ + protected PathMetadata verifyIsFile(String pathStr) throws IOException { + PathMetadata md = verifyCached(pathStr); + assertTrue("Not a file: " + md, + md.getFileStatus().isFile()); + return md; + } + + /** + * Assert that an entry exists and is a tombstone. + * @param pathStr path + * @throws IOException IO failure. + */ + protected void assertIsTombstone(String pathStr) throws IOException { + PathMetadata meta = getNonNull(pathStr); + assertTrue(pathStr + " must be a tombstone: " + meta, meta.isDeleted()); + } + + /** + * Assert that an entry does not exist. + * @param pathStr path + * @throws IOException IO failure. + */ + protected void assertNotFound(String pathStr) throws IOException { + PathMetadata meta = get(pathStr); + assertNull("Unexpectedly found entry at path " + pathStr + ": " + meta, + meta); + } + /** * Get an entry which must be a file. * @param pathStr path From fbfdd4cfa35abdeafc4318c212a004389c94db52 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 2 Jul 2019 13:17:14 +0100 Subject: [PATCH 06/23] HADOOP-16394 Prune Resilience and root dir test failures * S3GuardTableAccess to issue queries against the store * more on dumping entire metastore, with a scan before the rest of the listings take place (they update the store, see) * Better teardown in ITestDynamoDBMetastoreScale, as /test/ references were staying around Change-Id: I197f0946bf85d3a202d2ea137e2f1584761e6657 --- .../AbstractContractRootDirectoryTest.java | 27 +- .../fs/s3a/s3guard/DumpS3GuardTable.java | 267 ++++++++++++------ .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 8 + .../fs/s3a/s3guard/S3GuardTableAccess.java | 171 +++++++++++ .../s3guard/ITestDynamoDBMetadataStore.java | 24 ++ .../ITestDynamoDBMetadataStoreScale.java | 17 ++ .../s3guard/ITestS3GuardRootOperations.java | 74 ++++- .../s3a/s3guard/ITestS3GuardToolDynamoDB.java | 15 +- 8 files changed, 508 insertions(+), 95 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTableAccess.java diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java index 5fba4bfc2786b..b94888a035a17 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.contract; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; @@ -149,14 +150,18 @@ public void testRmRootRecursive() throws Throwable { Path root = new Path("/"); assertIsDirectory(root); Path file = new Path("/testRmRootRecursive"); - ContractTestUtils.touch(getFileSystem(), file); - boolean deleted = getFileSystem().delete(root, true); - assertIsDirectory(root); - LOG.info("rm -rf / result is {}", deleted); - if (deleted) { - assertPathDoesNotExist("expected file to be deleted", file); - } else { - assertPathExists("expected file to be preserved", file);; + try { + ContractTestUtils.touch(getFileSystem(), file); + boolean deleted = getFileSystem().delete(root, true); + assertIsDirectory(root); + LOG.info("rm -rf / result is {}", deleted); + if (deleted) { + assertPathDoesNotExist("expected file to be deleted", file); + } else { + assertPathExists("expected file to be preserved", file);; + } + } finally{ + getFileSystem().delete(file, false); } } @@ -185,8 +190,10 @@ public void testListEmptyRootDirectory() throws IOException { for (FileStatus status : statuses) { ContractTestUtils.assertDeleted(fs, status.getPath(), true); } - assertEquals("listStatus on empty root-directory returned a non-empty list", - 0, fs.listStatus(root).length); + FileStatus[] list1 = fs.listStatus(root); + assertEquals("listStatus on empty root-directory returned found: " + + StringUtils.join(list1, "\n"), + 0, list1.length); assertFalse("listFiles(/, false).hasNext", fs.listFiles(root, false).hasNext()); assertFalse("listFiles(/, true).hasNext", diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java index 17afcee5e5539..6db77b82c95ae 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java @@ -23,20 +23,22 @@ import java.io.IOException; import java.io.PrintWriter; import java.net.URI; -import java.net.URISyntaxException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.Date; import java.util.LinkedList; import java.util.List; +import com.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder; import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -55,13 +57,18 @@ import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.ExitUtil; +import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.hadoop.fs.s3a.S3AUtils.ACCEPT_ALL; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_USAGE; /** - * This is a new diagnostics entry point which does a TSV dump of - * the DDB treewalk. + * This is a low-level diagnostics entry point which does a CVE/TSV dump of + * the DDB state. + * As it also lists the filesystem, it actually changes the state of the store + * during the operation. */ +@InterfaceAudience.Private +@InterfaceStability.Unstable public class DumpS3GuardTable extends AbstractLaunchableService { private static final Logger LOG = @@ -72,10 +79,27 @@ public class DumpS3GuardTable extends AbstractLaunchableService { private static final String USAGE_MESSAGE = NAME + " "; + public static final String FLAT_CSV = "-flat.csv"; + + public static final String RAW_CSV = "-raw.csv"; + + public static final String SCAN_CSV = "-scan.csv"; + public static final String SCAN2_CSV = "-scan-2.csv"; + + public static final String TREE_CSV = "-tree.csv"; + + public static final String STORE_CSV = "-store.csv"; + private List arguments; private DynamoDBMetadataStore store; + private S3AFileSystem fs; + + private URI uri; + + private String destPath; + public DumpS3GuardTable(final String name) { super(name); } @@ -84,6 +108,22 @@ public DumpS3GuardTable() { this("DumpS3GuardTable"); } + /** + * Bind to a specific FS + store. + * @param fs filesystem + * @param store metastore to use + * @param destFile the base filename for output + */ + public DumpS3GuardTable( + final S3AFileSystem fs, + final DynamoDBMetadataStore store, + final File destFile) { + this(); + this.fs = fs; + this.store = checkNotNull(store, "store"); + this.destPath = destFile.getAbsolutePath(); + } + @Override public Configuration bindArgs(final Configuration config, final List args) @@ -96,33 +136,21 @@ public List getArguments() { return arguments; } - /** - * Dump the filesystem and the metastore. - * @return the exit code. - * @throws ServiceLaunchException on failure. - * @throws IOException IO failure. - */ @Override - public int execute() throws ServiceLaunchException, IOException { - - Preconditions.checkNotNull(arguments, "No arguments"); - Preconditions.checkState(arguments.size() == 2, - "Wrong number of arguments: %s", arguments.size()); - - String fsURI = arguments.get(0); - String destFilename = arguments.get(1); - Configuration conf = getConfig(); - URI uri; - try { + protected void serviceStart() throws Exception { + String fsURI = null; + if (store == null) { + checkNotNull(arguments, "No arguments"); + Preconditions.checkState(arguments.size() == 2, + "Wrong number of arguments: %s", arguments.size()); + fsURI = arguments.get(0); + destPath = arguments.get(1); + Configuration conf = getConfig(); uri = new URI(fsURI); - } catch (URISyntaxException e) { - throw fail("Bad URI " + fsURI, e); - } - try { FileSystem fileSystem = FileSystem.get(uri, conf); require(fileSystem instanceof S3AFileSystem, "Not an S3A Filesystem: " + fsURI); - S3AFileSystem fs = (S3AFileSystem) fileSystem; + fs = (S3AFileSystem) fileSystem; require(fs.hasMetadataStore(), "Filesystem has no metadata store: " + fsURI); MetadataStore ms = fs.getMetadataStore(); @@ -130,44 +158,87 @@ public int execute() throws ServiceLaunchException, IOException { "Filesystem " + fsURI + "does not have a DynamoDB metadata store: " + ms); store = (DynamoDBMetadataStore) ms; - Path basePath = fs.qualify(new Path(uri)); + } else { + if (fs != null) { + fsURI = fs.getUri().toString(); + } + } + if (fsURI != null) { + if (!fsURI.endsWith("/")) { + fsURI += "/"; + } + uri = new URI(fsURI); + } + } - final File destFile = new File(destFilename + "-store.csv") - .getCanonicalFile(); - LOG.info("Writing Store details to {}", destFile); - try (CsvFile csv = new CsvFile(destFile); - DurationInfo ignored = new DurationInfo(LOG, "List metastore")) { - csv.header(); + /** + * Dump the filesystem and the metastore. + * @return the exit code. + * @throws ServiceLaunchException on failure. + * @throws IOException IO failure. + */ + @Override + public int execute() throws ServiceLaunchException, IOException { - LOG.info("Base path: {}", basePath); - dumpMetastore(csv, basePath); - } - final File treewalkFile = new File(destFilename + "-tree.csv") - .getCanonicalFile(); + try { + final File scanFile = new File( + destPath + SCAN_CSV).getCanonicalFile(); - try (CsvFile csv = new CsvFile(treewalkFile); + try (CsvFile csv = new CsvFile(scanFile); DurationInfo ignored = new DurationInfo(LOG, - "Treewalk to %s", treewalkFile)) { - csv.header(); - treewalkFilesystem(csv, fs, basePath); + "scanFile dump to %s", scanFile)) { + scanMetastore(csv); } - final File flatlistFile = new File( - destFilename + "-flat.csv").getCanonicalFile(); - try (CsvFile csv = new CsvFile(flatlistFile); - DurationInfo ignored = new DurationInfo(LOG, - "Flat list to %s", flatlistFile)) { - csv.header(); - listStatusFilesystem(csv, fs, basePath); - } - final File rawFile = new File( - destFilename + "-raw.csv").getCanonicalFile(); + if (fs != null) { + + Path basePath = fs.qualify(new Path(uri)); + + final File destFile = new File(destPath + STORE_CSV) + .getCanonicalFile(); + LOG.info("Writing Store details to {}", destFile); + try (CsvFile csv = new CsvFile(destFile); + DurationInfo ignored = new DurationInfo(LOG, "List metastore")) { + + LOG.info("Base path: {}", basePath); + dumpMetastore(csv, basePath); + } + + // these operations all update the metastore as they list, + // that is: they are side-effecting. + final File treewalkFile = new File(destPath + TREE_CSV) + .getCanonicalFile(); + + try (CsvFile csv = new CsvFile(treewalkFile); + DurationInfo ignored = new DurationInfo(LOG, + "Treewalk to %s", treewalkFile)) { + treewalkFilesystem(csv, basePath); + } + final File flatlistFile = new File( + destPath + FLAT_CSV).getCanonicalFile(); + + try (CsvFile csv = new CsvFile(flatlistFile); + DurationInfo ignored = new DurationInfo(LOG, + "Flat list to %s", flatlistFile)) { + listStatusFilesystem(csv, basePath); + } + final File rawFile = new File( + destPath + RAW_CSV).getCanonicalFile(); + + try (CsvFile csv = new CsvFile(rawFile); + DurationInfo ignored = new DurationInfo(LOG, + "Raw dump to %s", rawFile)) { + dumpRawS3ObjectStore(csv); + } + final File scanFile2 = new File( + destPath + SCAN2_CSV).getCanonicalFile(); + + try (CsvFile csv = new CsvFile(scanFile); + DurationInfo ignored = new DurationInfo(LOG, + "scanFile dump to %s", scanFile2)) { + scanMetastore(csv); + } - try (CsvFile csv = new CsvFile(rawFile); - DurationInfo ignored = new DurationInfo(LOG, - "Raw dump to %s", rawFile)) { - csv.header(); - rawDump(csv, fs); } return LauncherExitCodes.EXIT_SUCCESS; @@ -189,7 +260,6 @@ public int execute() throws ServiceLaunchException, IOException { protected int treewalkFilesystem( final CsvFile csv, - final S3AFileSystem fs, final Path path) throws IOException { int count = 1; FileStatus[] fileStatuses = fs.listStatus(path); @@ -200,7 +270,7 @@ protected int treewalkFilesystem( for (FileStatus fileStatus : fileStatuses) { if (fileStatus.isDirectory() && !(fileStatus.getPath().equals(path))) { - count += treewalkFilesystem(csv, fs, fileStatus.getPath()); + count += treewalkFilesystem(csv, fileStatus.getPath()); } else { count++; } @@ -211,13 +281,11 @@ protected int treewalkFilesystem( /** * Dump the filesystem via a recursive listStatus call. * @param csv destination. - * @param fs filesystem. * @return number of entries found. * @throws IOException IO failure. */ protected int listStatusFilesystem( final CsvFile csv, - final S3AFileSystem fs, final Path path) throws IOException { int count = 0; RemoteIterator iterator = fs @@ -232,13 +300,11 @@ protected int listStatusFilesystem( /** * Dump the raw S3 Object Store. * @param csv destination. - * @param fs filesystem. * @return number of entries found. * @throws IOException IO failure. */ - protected int rawDump( - final CsvFile csv, - final S3AFileSystem fs) throws IOException { + protected int dumpRawS3ObjectStore( + final CsvFile csv) throws IOException { Path rootPath = fs.qualify(new Path("/")); Listing listing = new Listing(fs); S3ListRequest request = fs.createListObjectsRequest("", null); @@ -257,6 +323,13 @@ protected int rawDump( return count; } + /** + * list children under the metastore from a base path, through + * a recursive query + walk strategy. + * @param csv dest + * @param basePath base path + * @throws IOException failure. + */ protected void dumpMetastore(final CsvFile csv, final Path basePath) throws IOException { dumpRecursively(csv, store.listChildren(basePath)); @@ -306,6 +379,29 @@ private void dumpEntry(CsvFile csv, DDBPathMetadata md) { csv.entry(md); } + /** + * Scan the metastore for all entries and dump them. + * There's no attempt to sort the output. + * @param csv file + * @return count of the number of entries. + */ + private int scanMetastore(CsvFile csv) { + S3GuardTableAccess tableAccess = new S3GuardTableAccess(store); + ExpressionSpecBuilder builder = new ExpressionSpecBuilder(); + Iterable results = tableAccess.scanMetadata( + builder); + int count = 0; + for (DDBPathMetadata md : results) { + if (!(md instanceof S3GuardTableAccess.VersionMarker)) { + count++; + // print it + csv.entry(md); + } + } + return count; + } + + private static String stringify(long millis) { return new Date(millis).toString(); } @@ -366,25 +462,33 @@ public static void serviceMain(List argsList) { } /** - * Entry point to dump. - * @param filesystem filesystem path - * @param conf configuration to instantiate with. + * Entry point to dump the metastore and s3 store world views + *

+ * Both the FS and the store will be dumped: the store is scanned + * before and after the sequence to show what changes were made to + * the store during the list operation. + * @param fs fs to dump. If null a store must be provided. + * @param store store to dump (fallback to FS) + * @param conf configuration to use (fallback to fs) * @param destFile base name of the output files. * @throws ExitUtil.ExitException failure. */ - public static void dumpS3GuardStore(String filesystem, + public static void dumpS3GuardStore( + final S3AFileSystem fs, + final DynamoDBMetadataStore store, Configuration conf, File destFile) throws ExitUtil.ExitException { ServiceLauncher serviceLauncher = new ServiceLauncher<>(""); - if (!filesystem.endsWith("/")) { - filesystem += "/"; - } ExitUtil.ExitException ex = serviceLauncher.launchService( - conf, - new DumpS3GuardTable(), - Lists.newArrayList(filesystem, destFile.getAbsolutePath()), + conf == null ? fs.getConf() : conf, + new DumpS3GuardTable(fs, + (store == null + ? (DynamoDBMetadataStore) fs.getMetadataStore() + : store), + destFile), + Collections.emptyList(), false, true); if (ex != null && ex.getExitCode() != 0) { @@ -398,7 +502,7 @@ public static void dumpS3GuardStore(String filesystem, * Quotes are manged by passing in a long whose specific bits control * whether or not a row is quoted, bit 0 for column 0, etc. */ - private static class CsvFile implements Closeable { + private static final class CsvFile implements Closeable { /** constant to quote all columns. */ @@ -426,11 +530,12 @@ private CsvFile( final String separator, final String eol, final String quote) throws IOException { - this.separator = Preconditions.checkNotNull(separator); - this.eol = Preconditions.checkNotNull(eol); - this.quote = Preconditions.checkNotNull(quote); + this.separator = checkNotNull(separator); + this.eol = checkNotNull(eol); + this.quote = checkNotNull(quote); this.path = path; - this.out = Preconditions.checkNotNull(out); + this.out = checkNotNull(out); + header(); } private CsvFile(File file) throws IOException { @@ -469,7 +574,7 @@ public String getEol() { * @return self for ease of chaining. */ public CsvFile row(long quotes, Object... columns) { - Preconditions.checkNotNull(out); + checkNotNull(out); for (int i = 0; i < columns.length; i++) { if (i != 0) { out.write(separator); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java index ccaafdb4b6039..90bbe57df3efa 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java @@ -2296,6 +2296,14 @@ private ITtlTimeProvider extractTimeProvider( return ttlTimeProvider != null ? ttlTimeProvider : timeProvider; } + /** + * Username. + * @return the current username + */ + String getUsername() { + return username; + } + /** * Take an {@code IllegalArgumentException} raised by a DDB operation * and if it contains an inner SDK exception, unwrap it. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTableAccess.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTableAccess.java new file mode 100644 index 0000000000000..af2dab781dc03 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTableAccess.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.s3guard; + +import java.util.Collection; +import java.util.Iterator; + +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.internal.IteratorSupport; +import com.amazonaws.services.dynamodbv2.document.spec.QuerySpec; +import com.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder; +import com.amazonaws.services.dynamodbv2.xspec.ScanExpressionSpec; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileStatus; + +import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.VERSION_MARKER; +import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.CHILD; +import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.PARENT; +import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.TABLE_VERSION; +import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.itemToPathMetadata; + +/** + * Package-scoped accessor to table state in S3Guard. + * This is for maintenance, diagnostics and testing: it is not to + * be used otherwise. + *

    + *
  1. + * Some of the operations here may dramatically alter the state of + * a table, so use carefully. + *
  2. + *
  3. + * Operations to assess consistency of a store are best executed + * against a table which is otherwise inactive. + *
  4. + *
  5. + * No retry/throttling or AWS to IOE logic here. + *
  6. + *
  7. + * If a scan or query includes the version marker in the result, it + * is converted to a VersionMarker instance. + *
  8. + *
+ * + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +class S3GuardTableAccess { + + private static final Logger LOG = + LoggerFactory.getLogger(S3GuardTableAccess.class); + + private final DynamoDBMetadataStore store; + + private final Table table; + + S3GuardTableAccess(final DynamoDBMetadataStore store) { + this.store = checkNotNull(store); + this.table = store.getTable(); + } + + private String getUsername() { + return store.getUsername(); + } + + + ItemCollection query(QuerySpec spec) { + return table.query(spec); + } + + Iterable queryMetadata(QuerySpec spec) { + return new DDBPathMetadataCollection<>(query(spec)); + } + + ItemCollection scan(ExpressionSpecBuilder spec) { + return table.scan(spec.buildForScan()); + } + + Iterable scanMetadata(ExpressionSpecBuilder spec) { + return new DDBPathMetadataCollection<>(scan(spec)); + } + + void delete(Collection paths) { + paths.stream() + .map(PathMetadataDynamoDBTranslation::pathToKey) + .forEach(table::deleteItem); + } + + private final class DDBPathMetadataCollection + implements Iterable { + + private final ItemCollection outcome; + + private DDBPathMetadataCollection(final ItemCollection outcome) { + this.outcome = outcome; + } + + @Override + public Iterator iterator() { + return new DDBPathMetadataIterator<>(outcome.iterator()); + } + + } + + private final class DDBPathMetadataIterator implements + Iterator { + + private final IteratorSupport it; + + private DDBPathMetadataIterator(final IteratorSupport it) { + this.it = it; + } + + @Override + public boolean hasNext() { + return it.hasNext(); + } + + @Override + public DDBPathMetadata next() { + Item item = it.next(); + Pair key = primaryKey(item); + if (VERSION_MARKER.equals(key.getLeft()) && + VERSION_MARKER.equals(key.getRight())) { + // a version marker is found, return the special type + return new VersionMarker(item); + } else { + return itemToPathMetadata(item, getUsername()); + } + } + + } + + static final class VersionMarker extends DDBPathMetadata { + + VersionMarker(Item versionMarker) { + super(new S3AFileStatus(true, new Path("/VERSION"), + "" + versionMarker.getString(TABLE_VERSION))); + } + } + + private static Pair primaryKey(Item it) { + return Pair.of(it.getString(PARENT), it.getString(CHILD)); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java index 7194d352ba129..e1b170bafe9ee 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java @@ -18,9 +18,14 @@ package org.apache.hadoop.fs.s3a.s3guard; +import java.io.BufferedReader; +import java.io.File; +import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; +import java.io.InputStreamReader; import java.net.URI; +import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -1288,6 +1293,25 @@ public void testPutFileDeepUnderTombstone() throws Throwable { getDirectory(base); } + @Test + public void testDumpTable() throws Throwable { + String target = System.getProperty("test.build.dir", "target"); + File buildDir = new File(target).getAbsoluteFile(); + String name = "ITestDynamoDBMetadataStore"; + File destFile = new File(buildDir, name); + DumpS3GuardTable.dumpS3GuardStore( + null, + ddbmsStatic, + null, + destFile); + File storeFile = new File(buildDir, name + "-scan.csv"); + try (BufferedReader in = new BufferedReader(new InputStreamReader( + new FileInputStream(storeFile), Charset.forName("UTF-8")))) { + for (String line : org.apache.commons.io.IOUtils.readLines(in)) { + LOG.info(line); + } + } + } /** * Assert that an entry exists and is a directory. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java index 72a4bb468c6bd..3371b34c5abf0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java @@ -32,6 +32,7 @@ import com.amazonaws.services.dynamodbv2.document.DynamoDB; import com.amazonaws.services.dynamodbv2.document.Table; import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription; +import com.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder; import org.junit.Assume; import org.junit.FixMethodOrder; import org.junit.Test; @@ -59,6 +60,7 @@ import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.s3guard.MetadataStoreTestBase.basicFileStatus; +import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.PARENT; import static org.junit.Assume.*; /** @@ -172,6 +174,21 @@ public void setup() throws Exception { @Override public void teardown() throws Exception { + S3GuardTableAccess tableAccess = new S3GuardTableAccess(ddbms); + ExpressionSpecBuilder builder = new ExpressionSpecBuilder(); + builder.withKeyCondition( + ExpressionSpecBuilder.S(PARENT).beginsWith("/test/")); + + Iterable entries = tableAccess.scanMetadata(builder); + List list = new ArrayList<>(); + entries.iterator().forEachRemaining(e -> { + if (!(e instanceof S3GuardTableAccess.VersionMarker)) { + Path p = e.getFileStatus().getPath(); + LOG.info("Deleting {}", p); + list.add(p); + } + }); + tableAccess.delete(list); IOUtils.cleanupWithLogger(LOG, ddbms); super.teardown(); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java index 9b40343b1dcbd..83995366674f8 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java @@ -18,8 +18,10 @@ package org.apache.hadoop.fs.s3a.s3guard; +import java.io.File; import java.net.URI; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import org.assertj.core.api.Assertions; import org.junit.FixMethodOrder; @@ -28,6 +30,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; @@ -39,10 +42,11 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles; /** * This test run against the root of the FS, and operations which span the DDB - * table. + * table and the filesystem. * For this reason, these tests are executed in the sequential phase of the * integration tests. *

@@ -107,6 +111,21 @@ public void teardown() throws Exception { super.teardown(); } + @Test + public void test_050_dump_metastore() throws Throwable { + String target = System.getProperty("test.build.dir", "target"); + File buildDir = new File(target, + this.getClass().getSimpleName()).getAbsoluteFile(); + buildDir.mkdirs(); + File destFile = new File(buildDir, getMethodName()); + describe("Dumping S3Guard store under %s", destFile); + DumpS3GuardTable.dumpS3GuardStore( + null, + metastore, + getConfiguration(), + destFile); + } + @Test public void test_100_FilesystemPrune() throws Throwable { describe("Execute prune against a filesystem URI"); @@ -151,4 +170,57 @@ public void test_300_MetastorePrune() throws Throwable { .isEqualTo(0); } + @Test + public void test_400_rm_root_recursive() throws Throwable { + describe("Remove the root directory"); + //extra sanity checks here to avoid support calls about complete loss of data + S3AFileSystem fs = getFileSystem(); + Path root = new Path("/"); + Path file = new Path("/test_400_rm_root_recursive-01"); + Path file2 = new Path("/test_400_rm_root_recursive-02"); + // recursive treewalk to delete all files + // does not delete directories. + applyLocatedFiles(fs.listFilesAndEmptyDirectories(root, true), + f -> { + Path p = f.getPath(); + fs.delete(p, true); + assertPathDoesNotExist("expected file to be deleted", p); + }); + ContractTestUtils.deleteChildren(fs, root, true); + // everything must be done by now + StringBuffer sb = new StringBuffer(); + AtomicInteger foundFile = new AtomicInteger(0); + applyLocatedFiles(fs.listFilesAndEmptyDirectories(root, true), + f -> { + foundFile.addAndGet(1); + Path p = f.getPath(); + sb.append(f.isDirectory() + ? "Dir " + : "File ") + .append(p); + if (!f.isDirectory()) { + sb.append("[").append(f.getLen()).append("]"); + } + + fs.delete(p, true); + }); + + assertEquals("Remaining files " + sb, + 0, foundFile.get()); + try { + ContractTestUtils.touch(fs, file); + assertDeleted(file, false); + + + assertTrue("Root directory delete failed", + fs.delete(root, true)); + + ContractTestUtils.touch(fs, file2); + assertFalse("Root directory delete should have failed", + fs.delete(root, true)); + } finally { + fs.delete(file, false); + fs.delete(file2, false); + } + } } 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 e85612cdceca2..279da8a95a36b 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 @@ -72,6 +72,12 @@ public void setup() throws Exception { ms instanceof DynamoDBMetadataStore); } + + @Override + protected DynamoDBMetadataStore getMetadataStore() { + return (DynamoDBMetadataStore) super.getMetadataStore(); + } + // Check the existence of a given DynamoDB table. private static boolean exist(DynamoDB dynamoDB, String tableName) { assertNotNull(dynamoDB); @@ -292,11 +298,14 @@ public void testDumpTable() throws Throwable { File buildDir = new File(target).getAbsoluteFile(); String name = "dump-table"; File destFile = new File(buildDir, name); - describe("Dumping metastore to {}", destFile); S3AFileSystem fs = getFileSystem(); + describe("Dumping metastore %s to %s", + fs.getMetadataStore(), + destFile); DumpS3GuardTable.dumpS3GuardStore( - fs.getUri().toString(), - fs.getConf(), + fs, + null, + null, destFile); File storeFile = new File(buildDir, name + "-store.csv"); try (BufferedReader in = new BufferedReader(new InputStreamReader( From a3cfb6195997d9d638eb9711d1f61ba65e555e27 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 2 Jul 2019 13:21:01 +0100 Subject: [PATCH 07/23] HADOOP-16384 moving some of the sequential tests around so that the root dir test comes near the end Also: move all the commit MR tests to the sequential phase, so that they don't spawn so many processes in parallel that the local system overloads. This will inevitably slow scale tests down, but stop the system becoming unusable in the process Change-Id: I3d98f5a130895c25c9b18299b649a87e8369f8c5 --- hadoop-tools/hadoop-aws/pom.xml | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index f79136d1416cf..5bc278358003b 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -186,9 +186,11 @@ **/ITestS3AHuge*.java **/ITestDynamoDBMetadataStoreScale.java + **/ITestTerasort*.java - **/ITestS3GuardRootOperations*.java + **/ITest*CommitMRJob.java + **/ITestS3GuardRootOperations*.java @@ -217,16 +219,20 @@ - **/ITestS3AContractRootDir.java **/ITestS3AFileContextStatistics.java + **/ITestS3AHuge*.java + **/ITestS3AEncryptionSSEC*.java **/ITestDynamoDBMetadataStoreScale.java **/ITestTerasort*.java + + **/ITest*CommitMRJob.java + **/ITestS3AContractRootDir.java **/ITestS3GuardRootOperations*.java From ccce1ba09a4430fe74ca89da14164e30e19dc5cb Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 2 Jul 2019 18:21:50 +0100 Subject: [PATCH 08/23] HADOOP-16406. ITestDynamoDBMetadataStore.testProvisionTable times out intermittently Change-Id: I9803c9b8b4b4c1150c664bc47a256a3c7be4d8d5 --- .../s3guard/ITestDynamoDBMetadataStore.java | 37 ------------------- 1 file changed, 37 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java index e1b170bafe9ee..1633f04dca307 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java @@ -928,43 +928,6 @@ private DDBPathMetadata verifyInAncestor(AncestorState state, return md; } - @Test - public void testProvisionTable() throws Exception { - final String tableName - = getTestTableName("testProvisionTable-" + UUID.randomUUID()); - final Configuration conf = getTableCreationConfig(); - conf.set(S3GUARD_DDB_TABLE_NAME_KEY, tableName); - conf.setInt(S3GUARD_DDB_TABLE_CAPACITY_WRITE_KEY, 2); - conf.setInt(S3GUARD_DDB_TABLE_CAPACITY_READ_KEY, 2); - DynamoDBMetadataStore ddbms = new DynamoDBMetadataStore(); - try { - ddbms.initialize(conf); - DynamoDB dynamoDB = ddbms.getDynamoDB(); - final DDBCapacities oldProvision = DDBCapacities.extractCapacities( - dynamoDB.getTable(tableName).describe().getProvisionedThroughput()); - Assume.assumeFalse("Table is on-demand", oldProvision.isOnDemandTable()); - long desiredReadCapacity = oldProvision.getRead() - 1; - long desiredWriteCapacity = oldProvision.getWrite() - 1; - ddbms.provisionTable(desiredReadCapacity, - desiredWriteCapacity); - ddbms.initTable(); - // we have to wait until the provisioning settings are applied, - // so until the table is ACTIVE again and not in UPDATING - ddbms.getTable().waitForActive(); - final DDBCapacities newProvision = DDBCapacities.extractCapacities( - dynamoDB.getTable(tableName).describe().getProvisionedThroughput()); - assertEquals("Check newly provisioned table read capacity units.", - desiredReadCapacity, - newProvision.getRead()); - assertEquals("Check newly provisioned table write capacity units.", - desiredWriteCapacity, - newProvision.getWrite()); - } finally { - ddbms.destroy(); - ddbms.close(); - } - } - @Test public void testDeleteTable() throws Exception { final String tableName = getTestTableName("testDeleteTable"); From 4d297e3540ef0395ef1543db1f1e663e1330e820 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 3 Jul 2019 16:41:44 +0100 Subject: [PATCH 09/23] HADOOP-16384. More work on debugging and diagnostics Status: still getting an inconsistent table. * DynamoDB createAncestors now reverts closer to the original * REVISIT: Ancestor tracking is disabled (to see if this is the cause. It isn't) * REVISIT: Delete is no longer parallelised (to see if this is the cause. it isn't) * Audit state at the end of a bulk operation. I'd tried throwing an IOE but it was raising on renames where the src was going away presumably delete had taken it. * Explict "Commit" as bulk operation (for those messages/exceptions), used in WriteOperationHelper.initiateCommitOperation() * Alongside the dump operation, there's a (harsh) purge call. Needs tests, obviously * ITestS3AContractRootDir doesn't retry for eventual consistency if there's a metastore * ITestDynamoDBMetastoreScale does more prune on teardown Change-Id: I9edc81d68d11dbfeecfc41576b77bafeaa3a472e --- .../AbstractContractRootDirectoryTest.java | 48 ++++-- .../hadoop/fs/s3a/WriteOperationHelper.java | 2 +- .../s3guard/AbstractS3GuardDiagnostic.java | 158 ++++++++++++++++++ .../fs/s3a/s3guard/BulkOperationState.java | 2 + .../fs/s3a/s3guard/DumpS3GuardTable.java | 123 ++++---------- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 108 +++++++++--- .../fs/s3a/s3guard/PurgeS3GuardTable.java | 105 ++++++++++++ .../hadoop/fs/s3a/s3guard/RenameTracker.java | 6 +- .../contract/s3a/ITestS3AContractRootDir.java | 18 +- .../s3guard/ITestDynamoDBMetadataStore.java | 29 +++- .../ITestDynamoDBMetadataStoreScale.java | 32 ++-- .../s3guard/ITestS3GuardRootOperations.java | 40 ++++- .../s3a/s3guard/ITestS3GuardToolDynamoDB.java | 3 +- .../fs/s3a/s3guard/MetadataStoreTestBase.java | 2 +- .../s3guard/TestDynamoDBMiscOperations.java | 4 +- 15 files changed, 526 insertions(+), 154 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDiagnostic.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java index b94888a035a17..55ce0a5e46586 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java @@ -18,7 +18,6 @@ package org.apache.hadoop.fs.contract; -import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; @@ -32,8 +31,10 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.test.LambdaTestUtils; +import static org.apache.commons.lang3.StringUtils.join; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.contract.ContractTestUtils.deleteChildren; @@ -192,28 +193,55 @@ public void testListEmptyRootDirectory() throws IOException { } FileStatus[] list1 = fs.listStatus(root); assertEquals("listStatus on empty root-directory returned found: " - + StringUtils.join(list1, "\n"), + + join("\n", list1), 0, list1.length); - assertFalse("listFiles(/, false).hasNext", - fs.listFiles(root, false).hasNext()); - assertFalse("listFiles(/, true).hasNext", - fs.listFiles(root, true).hasNext()); - assertFalse("listLocatedStatus(/).hasNext", - fs.listLocatedStatus(root).hasNext()); + assertNoElements("listFiles(/, false)", + fs.listFiles(root, false)); + assertNoElements("listFiles(/, true)", + fs.listFiles(root, true)); + assertNoElements("listLocatedStatus(/)", + fs.listLocatedStatus(root)); assertIsDirectory(root); } + /** + * Assert that an iterator has no elements; the raised exception + * will include the element list. + * @param operation operation for assertion text. + * @param iter iterator + * @throws IOException failure retrieving the values. + */ + protected void assertNoElements(String operation, + RemoteIterator iter) throws IOException { + List resultList = toList(iter); + if (!resultList.isEmpty()) { + fail("Expected no results from " + operation + ", but got " + + resultList.size() + " elements:\n" + + join(resultList, "\n")); + } + } + @Test public void testSimpleRootListing() throws IOException { describe("test the nonrecursive root listing calls"); FileSystem fs = getFileSystem(); Path root = new Path("/"); FileStatus[] statuses = fs.listStatus(root); + String listStatusResult = join(statuses, "\n"); List locatedStatusList = toList( fs.listLocatedStatus(root)); - assertEquals(statuses.length, locatedStatusList.size()); + String locatedStatusResult = join(locatedStatusList, "\n"); + + assertEquals("listStatus(/) vs listLocatedStatus(/) with \n" + + "listStatus =" + listStatusResult + +" listLocatedStatus = " + locatedStatusResult, + statuses.length, locatedStatusList.size()); List fileList = toList(fs.listFiles(root, false)); - assertTrue(fileList.size() <= statuses.length); + String listFilesResult = join(fileList, "\n"); + assertTrue("listStatus(/) vs listFiles(/, false) with \n" + + "listStatus = " + listStatusResult + + "listFiles = " + listFilesResult, + fileList.size() <= statuses.length); } @Test diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java index 8cdce7b71d928..5b6dae7cdc963 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java @@ -538,7 +538,7 @@ public CompleteMultipartUploadResult commitUpload( public BulkOperationState initiateCommitOperation( Path path) throws IOException { return S3Guard.initiateBulkWrite(owner.getMetadataStore(), - BulkOperationState.OperationType.Put, path); + BulkOperationState.OperationType.Commit, path); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDiagnostic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDiagnostic.java new file mode 100644 index 0000000000000..c418ff9b44cf4 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDiagnostic.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.s3guard; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.List; + +import com.google.common.base.Preconditions; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.service.launcher.AbstractLaunchableService; +import org.apache.hadoop.service.launcher.LauncherExitCodes; +import org.apache.hadoop.service.launcher.ServiceLaunchException; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +public class AbstractS3GuardDiagnostic extends AbstractLaunchableService { + + private S3AFileSystem filesystem; + + private DynamoDBMetadataStore store; + + private URI uri; + + private List arguments; + + public AbstractS3GuardDiagnostic(final String name) { + super(name); + } + + public AbstractS3GuardDiagnostic(final String name, + final S3AFileSystem filesystem, + final DynamoDBMetadataStore store, + final URI uri) { + super(name); + this.store = store; + this.filesystem = filesystem; + if (uri == null) { + checkArgument(filesystem != null, "No filesystem or URI"); + // URI always gets a trailing / + setUri(filesystem.getUri().toString()); + } else { + setUri(uri); + } + if (store == null) { + bindStore(filesystem); + } + } + + private static void require(boolean condition, String error) { + if (!condition) { + throw fail(error); + } + } + + private static ServiceLaunchException fail(String message, Throwable ex) { + return new ServiceLaunchException(LauncherExitCodes.EXIT_FAIL, message, ex); + } + + private static ServiceLaunchException fail(String message) { + return new ServiceLaunchException(LauncherExitCodes.EXIT_FAIL, message); + } + + @Override + public Configuration bindArgs(final Configuration config, + final List args) + throws Exception { + this.arguments = args; + return super.bindArgs(config, args); + } + + public List getArguments() { + return arguments; + } + + protected void bindFromCLI(String fsURI ) + throws IOException, URISyntaxException { + Configuration conf = getConfig(); + setUri(fsURI); + FileSystem fs = FileSystem.get(getUri(), conf); + require(fs instanceof S3AFileSystem, + "Not an S3A Filesystem: " + fsURI); + filesystem = (S3AFileSystem) fs; + bindStore(filesystem); + + } + + private void bindStore(final S3AFileSystem fs) { + require(fs.hasMetadataStore(), + "Filesystem has no metadata store: " + fs.getUri()); + MetadataStore ms = fs.getMetadataStore(); + require(ms instanceof DynamoDBMetadataStore, + "Filesystem " + fs.getUri() + + " does not have a DynamoDB metadata store: " + ms); + store = (DynamoDBMetadataStore) ms; + } + + protected DynamoDBMetadataStore getStore() { + return store; + } + + public S3AFileSystem getFilesystem() { + return filesystem; + } + + public URI getUri() { + return uri; + } + + public void setUri(final URI uri) { + String fsURI = uri.toString(); + if (!fsURI.endsWith("/")) { + setUri(fsURI); + } else { + this.uri = uri; + } + } + + /** + * Set the URI from a string; will add a "/" if needed. + * @param fsURI filesystem URI. + * @throws RuntimeException if the fsURI parameter is not a valid URI. + */ + public void setUri(String fsURI) { + if (fsURI != null) { + if (!fsURI.endsWith("/")) { + fsURI += "/"; + } + try { + setUri(new URI(fsURI)); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java index 0fe05db833552..c3f48e3063227 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java @@ -78,5 +78,7 @@ public enum OperationType { Rename, /** Pruning: deleting entries and updating parents. */ Prune, + /** Commit operation. */ + Commit, } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java index 6db77b82c95ae..e07cd5c67f481 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java @@ -41,7 +41,6 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.s3a.Listing; @@ -50,7 +49,6 @@ import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; import org.apache.hadoop.fs.s3a.S3ListRequest; import org.apache.hadoop.service.Service; -import org.apache.hadoop.service.launcher.AbstractLaunchableService; import org.apache.hadoop.service.launcher.LauncherExitCodes; import org.apache.hadoop.service.launcher.ServiceLaunchException; import org.apache.hadoop.service.launcher.ServiceLauncher; @@ -69,7 +67,7 @@ */ @InterfaceAudience.Private @InterfaceStability.Unstable -public class DumpS3GuardTable extends AbstractLaunchableService { +public class DumpS3GuardTable extends AbstractS3GuardDiagnostic { private static final Logger LOG = LoggerFactory.getLogger(DumpS3GuardTable.class); @@ -81,7 +79,7 @@ public class DumpS3GuardTable extends AbstractLaunchableService { public static final String FLAT_CSV = "-flat.csv"; - public static final String RAW_CSV = "-raw.csv"; + public static final String RAW_CSV = "-s3.csv"; public static final String SCAN_CSV = "-scan.csv"; public static final String SCAN2_CSV = "-scan-2.csv"; @@ -90,22 +88,14 @@ public class DumpS3GuardTable extends AbstractLaunchableService { public static final String STORE_CSV = "-store.csv"; - private List arguments; - - private DynamoDBMetadataStore store; - - private S3AFileSystem fs; - - private URI uri; - - private String destPath; + protected String destPath; public DumpS3GuardTable(final String name) { super(name); } public DumpS3GuardTable() { - this("DumpS3GuardTable"); + this(NAME); } /** @@ -113,61 +103,27 @@ public DumpS3GuardTable() { * @param fs filesystem * @param store metastore to use * @param destFile the base filename for output + * @param uri URI of store -only needed if FS is null. */ public DumpS3GuardTable( final S3AFileSystem fs, final DynamoDBMetadataStore store, - final File destFile) { - this(); - this.fs = fs; - this.store = checkNotNull(store, "store"); + final File destFile, + final URI uri) { + super(NAME, fs, store, uri); this.destPath = destFile.getAbsolutePath(); } - @Override - public Configuration bindArgs(final Configuration config, - final List args) - throws Exception { - this.arguments = args; - return super.bindArgs(config, args); - } - - public List getArguments() { - return arguments; - } @Override protected void serviceStart() throws Exception { - String fsURI = null; - if (store == null) { + if (getStore()== null) { + List arguments = getArguments(); checkNotNull(arguments, "No arguments"); Preconditions.checkState(arguments.size() == 2, "Wrong number of arguments: %s", arguments.size()); - fsURI = arguments.get(0); + bindFromCLI(arguments.get(0)); destPath = arguments.get(1); - Configuration conf = getConfig(); - uri = new URI(fsURI); - FileSystem fileSystem = FileSystem.get(uri, conf); - require(fileSystem instanceof S3AFileSystem, - "Not an S3A Filesystem: " + fsURI); - fs = (S3AFileSystem) fileSystem; - require(fs.hasMetadataStore(), - "Filesystem has no metadata store: " + fsURI); - MetadataStore ms = fs.getMetadataStore(); - require(ms instanceof DynamoDBMetadataStore, - "Filesystem " + fsURI - + "does not have a DynamoDB metadata store: " + ms); - store = (DynamoDBMetadataStore) ms; - } else { - if (fs != null) { - fsURI = fs.getUri().toString(); - } - } - if (fsURI != null) { - if (!fsURI.endsWith("/")) { - fsURI += "/"; - } - uri = new URI(fsURI); } } @@ -190,9 +146,9 @@ public int execute() throws ServiceLaunchException, IOException { scanMetastore(csv); } - if (fs != null) { + if (getFilesystem() != null) { - Path basePath = fs.qualify(new Path(uri)); + Path basePath = getFilesystem().qualify(new Path(getUri())); final File destFile = new File(destPath + STORE_CSV) .getCanonicalFile(); @@ -253,16 +209,14 @@ public int execute() throws ServiceLaunchException, IOException { * If metastore entries mark directories as deleted, this * walk will not explore them. * @param csv destination. - * @param fs filesystem. * @return number of entries found. * @throws IOException IO failure. */ - protected int treewalkFilesystem( final CsvFile csv, final Path path) throws IOException { int count = 1; - FileStatus[] fileStatuses = fs.listStatus(path); + FileStatus[] fileStatuses = getFilesystem().listStatus(path); // entries for (FileStatus fileStatus : fileStatuses) { csv.entry((S3AFileStatus) fileStatus); @@ -288,7 +242,7 @@ protected int listStatusFilesystem( final CsvFile csv, final Path path) throws IOException { int count = 0; - RemoteIterator iterator = fs + RemoteIterator iterator = getFilesystem() .listFilesAndEmptyDirectories(path, true); while (iterator.hasNext()) { S3ALocatedFileStatus status = iterator.next(); @@ -305,6 +259,7 @@ protected int listStatusFilesystem( */ protected int dumpRawS3ObjectStore( final CsvFile csv) throws IOException { + S3AFileSystem fs = getFilesystem(); Path rootPath = fs.qualify(new Path("/")); Listing listing = new Listing(fs); S3ListRequest request = fs.createListObjectsRequest("", null); @@ -332,7 +287,7 @@ protected int dumpRawS3ObjectStore( */ protected void dumpMetastore(final CsvFile csv, final Path basePath) throws IOException { - dumpRecursively(csv, store.listChildren(basePath)); + dumpRecursively(csv, getStore().listChildren(basePath)); } /** @@ -362,7 +317,7 @@ private Pair dumpRecursively( } } for (DDBPathMetadata childDir : childDirs) { - DirListingMetadata children = store.listChildren( + DirListingMetadata children = getStore().listChildren( childDir.getFileStatus().getPath()); Pair pair = dumpRecursively(csv, children); @@ -386,7 +341,7 @@ private void dumpEntry(CsvFile csv, DDBPathMetadata md) { * @return count of the number of entries. */ private int scanMetastore(CsvFile csv) { - S3GuardTableAccess tableAccess = new S3GuardTableAccess(store); + S3GuardTableAccess tableAccess = new S3GuardTableAccess(getStore()); ExpressionSpecBuilder builder = new ExpressionSpecBuilder(); Iterable results = tableAccess.scanMetadata( builder); @@ -406,20 +361,6 @@ private static String stringify(long millis) { return new Date(millis).toString(); } - private static void require(boolean condition, String error) { - if (!condition) { - throw fail(error); - } - } - - private static ServiceLaunchException fail(String message, Throwable ex) { - return new ServiceLaunchException(LauncherExitCodes.EXIT_FAIL, message, ex); - } - - private static ServiceLaunchException fail(String message) { - return new ServiceLaunchException(LauncherExitCodes.EXIT_FAIL, message); - } - /** * This is the JVM entry point for the service launcher. * @@ -471,23 +412,31 @@ public static void serviceMain(List argsList) { * @param store store to dump (fallback to FS) * @param conf configuration to use (fallback to fs) * @param destFile base name of the output files. + * @param uri URI of store -only needed if FS is null. * @throws ExitUtil.ExitException failure. */ public static void dumpS3GuardStore( final S3AFileSystem fs, - final DynamoDBMetadataStore store, + DynamoDBMetadataStore store, Configuration conf, - File destFile) throws ExitUtil.ExitException { + final File destFile, + URI uri) throws ExitUtil.ExitException { ServiceLauncher serviceLauncher = new ServiceLauncher<>(""); + if (conf == null) { + conf = checkNotNull(fs, "No filesystem").getConf(); + } + if (store == null) { + store = (DynamoDBMetadataStore) checkNotNull(fs, "No filesystem") + .getMetadataStore(); + } ExitUtil.ExitException ex = serviceLauncher.launchService( - conf == null ? fs.getConf() : conf, + conf, new DumpS3GuardTable(fs, - (store == null - ? (DynamoDBMetadataStore) fs.getMetadataStore() - : store), - destFile), + store, + destFile, + uri), Collections.emptyList(), false, true); @@ -619,9 +568,9 @@ public PrintWriter getOut() { */ void header() { row(CsvFile.ALL_QUOTES, - "path", "type", "deleted", + "path", "is_auth_dir", "is_empty_dir", "len", @@ -640,9 +589,9 @@ void header() { void entry(DDBPathMetadata md) { S3AFileStatus fileStatus = md.getFileStatus(); row(ROW_QUOTE_MAP, - fileStatus.getPath().toString(), fileStatus.isDirectory() ? "dir" : "file", md.isDeleted(), + fileStatus.getPath().toString(), md.isAuthoritativeDir(), md.isEmptyDirectory().name(), fileStatus.getLen(), diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java index 90bbe57df3efa..102ea17b3b856 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java @@ -606,11 +606,18 @@ public void deleteSubtree(Path path, ITtlTimeProvider ttlTimeProvider) tableName, region, path); final PathMetadata meta = get(path); - if (meta == null || meta.isDeleted()) { + // REVISIT + if (meta == null /*|| meta.isDeleted()*/) { LOG.debug("Subtree path {} does not exist; this will be a no-op", path); return; } + // REVISIT + for (DescendantsIterator desc = new DescendantsIterator(this, meta); + desc.hasNext(); ) { + innerDelete(desc.next().getPath(), true, ttlTimeProvider); + } + /* // Execute via the bounded threadpool. final List> futures = new ArrayList<>(); for (DescendantsIterator desc = new DescendantsIterator(this, meta); @@ -628,6 +635,7 @@ public void deleteSubtree(Path path, ITtlTimeProvider ttlTimeProvider) } // now wait for the final set. waitForCompletion(futures); + */ } /** @@ -807,7 +815,8 @@ private Collection completeAncestry( final Collection pathsToCreate, final AncestorState ancestorState, final ITtlTimeProvider ttlTimeProvider) throws PathIOException { - List ancestorsToAdd = new ArrayList<>(0); + // Key on path to allow fast lookup + Map ancestry = new HashMap<>(); LOG.debug("Completing ancestry for {} paths", pathsToCreate.size()); // we sort the inputs to guarantee that the topmost entries come first. // that way if the put request contains both parents and children @@ -847,9 +856,9 @@ private Collection completeAncestry( path, entry); } } - ancestorsToAdd.add(entry); + ancestry.put(path, entry); Path parent = path.getParent(); - while (!parent.isRoot()) { + while (!parent.isRoot() && !ancestry.containsKey(parent)) { if (!ancestorState.findEntry(parent, true)) { // don't add this entry, but carry on with the parents LOG.debug("auto-create ancestor path {} for child path {}", @@ -858,12 +867,12 @@ private Collection completeAncestry( DDBPathMetadata md = new DDBPathMetadata(status, Tristate.FALSE, false, false, ttlTimeProvider.getNow()); ancestorState.put(parent, md); - ancestorsToAdd.add(md); + ancestry.put(parent, md); } parent = parent.getParent(); } } - return ancestorsToAdd; + return ancestry.values(); } /** @@ -937,7 +946,7 @@ public void addAncestors( entryFound = true; if (directory.getFileStatus().isFile()) { throw new PathIOException(parent.toString(), - "Cannot overwrite parent file: metadatstore is" + "Cannot overwrite parent file: metastore is" + " in an inconsistent state"); } // the directory exists. Add it to the ancestor state for next time. @@ -1040,7 +1049,7 @@ public void move( *

  • No attempt is made to sort the input: the caller must do that
  • * * As well as retrying on the operation invocation, incomplete - * batches are retried until all have been processed.. + * batches are retried until all have been processed. * @param keysToDelete primary keys to be deleted; can be null * @param itemsToPut new items to be put; can be null * @return the number of iterations needed to complete the call. @@ -1063,8 +1072,10 @@ private int processBatchWriteRequest(PrimaryKey[] keysToDelete, && count < totalToDelete) { numToDelete = Math.min(S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT, totalToDelete - count); - writeItems.withPrimaryKeysToDelete( - Arrays.copyOfRange(keysToDelete, count, count + numToDelete)); + PrimaryKey[] toDelete = Arrays.copyOfRange(keysToDelete, + count, count + numToDelete); + LOG.debug("Deleting {} entries: {}", toDelete.length, toDelete); + writeItems.withPrimaryKeysToDelete(toDelete); count += numToDelete; } @@ -1075,8 +1086,10 @@ private int processBatchWriteRequest(PrimaryKey[] keysToDelete, S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT - numToDelete, totalToDelete + totalToPut - count); final int index = count - totalToDelete; - writeItems.withItemsToPut( - Arrays.copyOfRange(itemsToPut, index, index + numToPut)); + Item[] toPut = Arrays.copyOfRange(itemsToPut, index, + index + numToPut); + LOG.debug("Putting {} entries: {}", toPut.length, toPut); + writeItems.withItemsToPut(toPut); count += numToPut; } @@ -1495,7 +1508,7 @@ private void innerPrune(String keyPrefix, ItemCollection items) // add parent path of item so it can be marked as non-auth. // this is only done if // * it has not already been processed - // * the entry pruned is not a tombstone (no need to update + // * the entry pruned is not a tombstone (no need to update) // * the file is not in the root dir Path parentPath = path.getParent(); if (!tombstone @@ -2275,14 +2288,14 @@ public RenameTracker initiateRenameOperation( final S3AFileStatus sourceStatus, final Path dest) { return new ProgressiveRenameTracker(storeContext, this, source, dest, - new AncestorState(BulkOperationState.OperationType.Rename, dest)); + new AncestorState(this, BulkOperationState.OperationType.Rename, dest)); } @Override public AncestorState initiateBulkWrite( final BulkOperationState.OperationType operation, final Path dest) { - return new AncestorState(operation, dest); + return new AncestorState(this, operation, dest); } /** @@ -2352,13 +2365,12 @@ static IOException translateTableWaitFailure( * @param operation the type of the operation to use if the state is created. * @return the cast or created state. */ - @VisibleForTesting - static AncestorState extractOrCreate(@Nullable BulkOperationState state, + private AncestorState extractOrCreate(@Nullable BulkOperationState state, BulkOperationState.OperationType operation) { if (state != null) { return (AncestorState) state; } else { - return new AncestorState(operation, null); + return new AncestorState(this, operation, null); } } @@ -2371,17 +2383,25 @@ static AncestorState extractOrCreate(@Nullable BulkOperationState state, @VisibleForTesting static final class AncestorState extends BulkOperationState { + private final DynamoDBMetadataStore store; + private final Map ancestry = new HashMap<>(); private final Path dest; /** * Create the state. + * @param store the store, for use in validation. + * If null: no validation (test only operation) * @param operation the type of the operation. * @param dest destination path. */ - AncestorState(final OperationType operation, @Nullable final Path dest) { + AncestorState( + @Nullable final DynamoDBMetadataStore store, + final OperationType operation, + @Nullable final Path dest) { super(operation); + this.store = store; this.dest = dest; } @@ -2413,7 +2433,7 @@ public String toString() { * @return true if the state has an entry */ boolean contains(Path p) { - return ancestry.containsKey(p); + return get(p) != null; } DDBPathMetadata put(Path p, DDBPathMetadata md) { @@ -2425,7 +2445,9 @@ DDBPathMetadata put(DDBPathMetadata md) { } DDBPathMetadata get(Path p) { - return ancestry.get(p); + // REVISIT: reinstate + return null; +// return ancestry.get(p); } /** @@ -2457,5 +2479,49 @@ boolean findEntry( return false; } } + + /** + * If debug logging is enabled, this does an audit of the store state. + * it only logs this; the error messages are created so as they could + * be turned into exception messages. + * There reason the audit failures aren't being turned into IOEs is that + * rename operations delete the source entry and that ends up in the + * ancestor state as present + * @throws IOException + */ + @Override + public void close() throws IOException { + if (LOG.isDebugEnabled() && store != null) { + for (Map.Entry entry : ancestry + .entrySet()) { + Path path = entry.getKey(); + DDBPathMetadata expected = entry.getValue(); + if (expected.isDeleted()) { + // file was deleted in bulk op; we don't care about it + // any more + continue; + } + DDBPathMetadata actual = store.get(path); + if (actual == null || actual.isDeleted()) { + String message = "Metastore entry for path " + + path + " deleted during bulk " + + getOperation() + " operation"; + LOG.debug(message); + } + if (actual.getFileStatus().isDirectory() != expected.getFileStatus() + .isDirectory()) { + // the type of the entry has changed + String message = "Metastore entry for path " + + path + " changed during bulk " + + getOperation() + " operation" + + " from " + expected + + " to " + actual; + LOG.debug(message); + } + + } + } + } } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java new file mode 100644 index 0000000000000..1ac58281f3a1e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.s3guard; + +import java.io.IOException; +import java.net.URI; +import java.util.ArrayList; +import java.util.List; + +import com.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder; +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.service.launcher.LauncherExitCodes; +import org.apache.hadoop.service.launcher.ServiceLaunchException; + +import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.PARENT; + +/** + * Purge the S3Guard table of a FileSystem from all entries related to + * that table. + * Will fail if there is no table, or the store is in auth mode + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class PurgeS3GuardTable extends AbstractS3GuardDiagnostic { + + private static final Logger LOG = + LoggerFactory.getLogger(PurgeS3GuardTable.class); + + public static final String NAME = "PurgeS3GuardTable"; + + public PurgeS3GuardTable(final String name) { + super(name); + } + + public PurgeS3GuardTable() { + this(NAME); + } + + @Override + protected void serviceStart() throws Exception { + if (getStore() == null) { + List arguments = getArguments(); + checkNotNull(arguments, "No arguments"); + Preconditions.checkState(arguments.size() == 1, + "Wrong number of arguments: %s", arguments.size()); + bindFromCLI(arguments.get(0)); + } + } + + /** + * Extract the host from the FS URI, then scan and + * delete all entries from thtat bucket + * @return the exit code. + * @throws ServiceLaunchException on failure. + * @throws IOException IO failure. + */ + @Override + public int execute() throws ServiceLaunchException, IOException { + + URI uri = getUri(); + String host = uri.getHost(); + String prefix = "/" + host + "/"; + DynamoDBMetadataStore ddbms = getStore(); + S3GuardTableAccess tableAccess = new S3GuardTableAccess(ddbms); + ExpressionSpecBuilder builder = new ExpressionSpecBuilder(); + builder.withKeyCondition( + ExpressionSpecBuilder.S(PARENT).beginsWith(prefix)); + + Iterable entries = tableAccess.scanMetadata(builder); + List list = new ArrayList<>(); + entries.iterator().forEachRemaining(e -> { + if (!(e instanceof S3GuardTableAccess.VersionMarker)) { + Path p = e.getFileStatus().getPath(); + LOG.info("Deleting {}", p); + list.add(p); + } + }); + LOG.info("Deleting {} entries", list.size()); + tableAccess.delete(list); + return LauncherExitCodes.EXIT_SUCCESS; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java index 76e269e1e3f6e..bfb5c186dd719 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java @@ -94,7 +94,7 @@ public abstract class RenameTracker extends AbstractStoreOperation { * Constructor. * @param name tracker name for logs. * @param storeContext store context. - * @param metadataStore the stopre + * @param metadataStore the store * @param sourceRoot source path. * @param dest destination path. * @param operationState ongoing move state. @@ -220,7 +220,9 @@ public void sourceObjectsDeleted( * @throws IOException failure. */ public void completeRename() throws IOException { - IOUtils.cleanupWithLogger(LOG, operationState); + // if the state is doing any final actions, it should be + // completed now. + operationState.close(); noteRenameFinished(); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java index 5c2e2cdf3677a..9ef3763bfc39a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java @@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest; import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,9 +57,24 @@ protected AbstractFSContract createContract(Configuration conf) { return new S3AContract(conf); } + @Override + public S3AFileSystem getFileSystem() { + return (S3AFileSystem) super.getFileSystem(); + } + + /** + * This is overridden to allow for eventual consistency on listings, + * but only if the store does not have S3Guard protecting it. + */ @Override public void testListEmptyRootDirectory() throws IOException { - for (int attempt = 1, maxAttempts = 10; attempt <= maxAttempts; ++attempt) { + int maxAttempts = 10; + if (getFileSystem().hasMetadataStore()) { + maxAttempts = 1; + } + describe("Listing root directory; for consistency allowing " + + maxAttempts + " attempts"); + for (int attempt = 1; attempt <= maxAttempts; ++attempt) { try { super.testListEmptyRootDirectory(); break; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java index 1633f04dca307..27ec6e221c3da 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java @@ -1161,15 +1161,28 @@ public void testPruneTombstoneUnderTombstone() throws Throwable { // so that lists of that dir will pick up the tombstone putTombstone(child, now, null); getDirectory(dir); - // put a tombstone + // tombstone the dir putTombstone(dir, now, null); + // add another child entry; this will update the dir entry from being + // tombstone to dir putFile(child2, now, null); + getDirectory(dir); + + // put a tombstone over the directory again + putTombstone(dir, now, null); + // verify + assertIsTombstone(dir); + + //prune all tombstones getDynamoMetadataStore().prune(PruneMode.TOMBSTONES_BY_LASTUPDATED, now + MINUTE); + // the child is gone assertNotFound(child); + // *AND* the parent dir has not been created assertNotFound(dir); + // the child2 entry is still there, though it's now orphan (the store isn't // meeting the rule "all entries must have a parent which exists" getFile(child2); @@ -1213,8 +1226,7 @@ public void testPruneFileUnderTombstone() throws Throwable { /** * Keep in sync with code changes in S3AFileSystem.finishedWrite() so that - * code can be tested here. - * @throws Throwable + * the production code can be tested here. */ @Test public void testPutFileDeepUnderTombstone() throws Throwable { @@ -1258,6 +1270,7 @@ public void testPutFileDeepUnderTombstone() throws Throwable { @Test public void testDumpTable() throws Throwable { + describe("Dump the table contents, but not the S3 Store"); String target = System.getProperty("test.build.dir", "target"); File buildDir = new File(target).getAbsoluteFile(); String name = "ITestDynamoDBMetadataStore"; @@ -1265,9 +1278,10 @@ public void testDumpTable() throws Throwable { DumpS3GuardTable.dumpS3GuardStore( null, ddbmsStatic, - null, - destFile); - File storeFile = new File(buildDir, name + "-scan.csv"); + getFileSystem().getConf(), + destFile, + fsUri); + File storeFile = new File(buildDir, name + DumpS3GuardTable.SCAN_CSV); try (BufferedReader in = new BufferedReader(new InputStreamReader( new FileInputStream(storeFile), Charset.forName("UTF-8")))) { for (String line : org.apache.commons.io.IOUtils.readLines(in)) { @@ -1281,7 +1295,8 @@ public void testDumpTable() throws Throwable { * @param pathStr path * @throws IOException IO failure. */ - protected PathMetadata verifyAuthDirStatus(String pathStr, boolean authDirFlag) + protected DDBPathMetadata verifyAuthDirStatus(String pathStr, + boolean authDirFlag) throws IOException { DDBPathMetadata md = (DDBPathMetadata) getDirectory(pathStr); assertEquals("isAuthoritativeDir() mismatch in " + md, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java index 3371b34c5abf0..a4763c9dd6d65 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java @@ -174,21 +174,23 @@ public void setup() throws Exception { @Override public void teardown() throws Exception { - S3GuardTableAccess tableAccess = new S3GuardTableAccess(ddbms); - ExpressionSpecBuilder builder = new ExpressionSpecBuilder(); - builder.withKeyCondition( - ExpressionSpecBuilder.S(PARENT).beginsWith("/test/")); - - Iterable entries = tableAccess.scanMetadata(builder); - List list = new ArrayList<>(); - entries.iterator().forEachRemaining(e -> { - if (!(e instanceof S3GuardTableAccess.VersionMarker)) { - Path p = e.getFileStatus().getPath(); - LOG.info("Deleting {}", p); - list.add(p); - } - }); - tableAccess.delete(list); + if (ddbms != null) { + S3GuardTableAccess tableAccess = new S3GuardTableAccess(ddbms); + ExpressionSpecBuilder builder = new ExpressionSpecBuilder(); + builder.withKeyCondition( + ExpressionSpecBuilder.S(PARENT).beginsWith("/test/")); + + Iterable entries = tableAccess.scanMetadata(builder); + List list = new ArrayList<>(); + entries.iterator().forEachRemaining(e -> { + if (!(e instanceof S3GuardTableAccess.VersionMarker)) { + Path p = e.getFileStatus().getPath(); + LOG.info("Deleting {}", p); + list.add(p); + } + }); + tableAccess.delete(list); + } IOUtils.cleanupWithLogger(LOG, ddbms); super.teardown(); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java index 83995366674f8..7f893abe3287b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java @@ -25,6 +25,7 @@ import org.assertj.core.api.Assertions; import org.junit.FixMethodOrder; +import org.junit.Ignore; import org.junit.Test; import org.junit.runners.MethodSorters; @@ -63,6 +64,8 @@ public class ITestS3GuardRootOperations extends AbstractS3ATestBase { private String metastoreUriStr; + private boolean cleaning = true; + /** * The test timeout is increased in case previous tests have created * many tombstone markers which now need to be purged. @@ -111,24 +114,26 @@ public void teardown() throws Exception { super.teardown(); } + private void assumeCleaningOperation() { + assume("Cleaning operation skipped", cleaning); + } + @Test public void test_050_dump_metastore() throws Throwable { - String target = System.getProperty("test.build.dir", "target"); - File buildDir = new File(target, - this.getClass().getSimpleName()).getAbsoluteFile(); - buildDir.mkdirs(); - File destFile = new File(buildDir, getMethodName()); + File destFile = calculateDumpFileBase(); describe("Dumping S3Guard store under %s", destFile); DumpS3GuardTable.dumpS3GuardStore( null, metastore, getConfiguration(), - destFile); + destFile, + getFileSystem().getUri()); } @Test public void test_100_FilesystemPrune() throws Throwable { describe("Execute prune against a filesystem URI"); + assumeCleaningOperation(); S3AFileSystem fs = getFileSystem(); Configuration conf = fs.getConf(); int result = S3GuardTool.run(conf, @@ -143,6 +148,7 @@ public void test_100_FilesystemPrune() throws Throwable { @Test public void test_200_MetastorePruneTombstones() throws Throwable { describe("Execute prune against a dynamo URL"); + assumeCleaningOperation(); S3AFileSystem fs = getFileSystem(); Configuration conf = fs.getConf(); int result = S3GuardTool.run(conf, @@ -159,6 +165,7 @@ public void test_200_MetastorePruneTombstones() throws Throwable { @Test public void test_300_MetastorePrune() throws Throwable { describe("Execute prune against a dynamo URL"); + assumeCleaningOperation(); S3AFileSystem fs = getFileSystem(); Configuration conf = fs.getConf(); int result = S3GuardTool.run(conf, @@ -173,6 +180,7 @@ public void test_300_MetastorePrune() throws Throwable { @Test public void test_400_rm_root_recursive() throws Throwable { describe("Remove the root directory"); + assumeCleaningOperation(); //extra sanity checks here to avoid support calls about complete loss of data S3AFileSystem fs = getFileSystem(); Path root = new Path("/"); @@ -223,4 +231,24 @@ public void test_400_rm_root_recursive() throws Throwable { fs.delete(file2, false); } } + + @Test + public void test_600_dump_metastore() throws Throwable { + File destFile = calculateDumpFileBase(); + describe("Dumping S3Guard store under %s", destFile); + DumpS3GuardTable.dumpS3GuardStore( + getFileSystem(), + metastore, + getConfiguration(), + destFile, + getFileSystem().getUri()); + } + + protected File calculateDumpFileBase() { + String target = System.getProperty("test.build.dir", "target"); + File buildDir = new File(target, + this.getClass().getSimpleName()).getAbsoluteFile(); + buildDir.mkdirs(); + return new File(buildDir, getMethodName()); + } } 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 279da8a95a36b..026247e3fbb8b 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 @@ -306,7 +306,8 @@ public void testDumpTable() throws Throwable { fs, null, null, - destFile); + destFile, + getFileSystem().getUri()); File storeFile = new File(buildDir, name + "-store.csv"); try (BufferedReader in = new BufferedReader(new InputStreamReader( new FileInputStream(storeFile), Charset.forName("UTF-8")))) { 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 fdf70898b101a..fc3c3fd0788cf 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 @@ -1107,7 +1107,7 @@ protected void assertIsTombstone(String pathStr) throws IOException { */ protected void assertNotFound(String pathStr) throws IOException { PathMetadata meta = get(pathStr); - assertNull("Unexpectedly found entry at path " + pathStr + ": " + meta, + assertNull("Unexpectedly found entry at path " + pathStr, meta); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDynamoDBMiscOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDynamoDBMiscOperations.java index a6d20fd966af2..1e2f7be8c0f42 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDynamoDBMiscOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDynamoDBMiscOperations.java @@ -115,7 +115,7 @@ public void testInnerListChildrenDirectoryNpe() throws Exception { public void testAncestorStateForDir() throws Throwable { final DynamoDBMetadataStore.AncestorState ancestorState = new DynamoDBMetadataStore.AncestorState( - BulkOperationState.OperationType.Rename, null); + null, BulkOperationState.OperationType.Rename, null); // path 1 is a directory final Path path1 = new Path("s3a://bucket/1"); @@ -143,7 +143,7 @@ public void testAncestorStateForDir() throws Throwable { public void testAncestorStateForFile() throws Throwable { final DynamoDBMetadataStore.AncestorState ancestorState = new DynamoDBMetadataStore.AncestorState( - BulkOperationState.OperationType.Rename, null); + null, BulkOperationState.OperationType.Rename, null); // path 1 is a file final Path path1 = new Path("s3a://bucket/1"); From c87415abfe31417f75603913b462289c65c6b0f1 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 3 Jul 2019 19:00:35 +0100 Subject: [PATCH 10/23] HADOOP-16384 ITestS3GuardRootOperations does a dump of the full state before any pruning/cleaning Change-Id: Ib4a946117d158eb9d50436cf90db49a14197ca17 --- .../s3a/s3guard/ITestS3GuardRootOperations.java | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java index 7f893abe3287b..21adf1406e775 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java @@ -64,7 +64,7 @@ public class ITestS3GuardRootOperations extends AbstractS3ATestBase { private String metastoreUriStr; - private boolean cleaning = true; + private boolean cleaning = false; /** * The test timeout is increased in case previous tests have created @@ -130,6 +130,18 @@ public void test_050_dump_metastore() throws Throwable { getFileSystem().getUri()); } + @Test + public void test_060_dump_metastore_and_s3() throws Throwable { + File destFile = calculateDumpFileBase(); + describe("Dumping S3Guard store under %s", destFile); + DumpS3GuardTable.dumpS3GuardStore( + getFileSystem(), + metastore, + getConfiguration(), + destFile, + getFileSystem().getUri()); + } + @Test public void test_100_FilesystemPrune() throws Throwable { describe("Execute prune against a filesystem URI"); @@ -181,7 +193,6 @@ public void test_300_MetastorePrune() throws Throwable { public void test_400_rm_root_recursive() throws Throwable { describe("Remove the root directory"); assumeCleaningOperation(); - //extra sanity checks here to avoid support calls about complete loss of data S3AFileSystem fs = getFileSystem(); Path root = new Path("/"); Path file = new Path("/test_400_rm_root_recursive-01"); From cabe6b419add84594cb7493f82b3ceca992f664a Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 4 Jul 2019 20:03:11 +0100 Subject: [PATCH 11/23] HADOOP-16384 new Purge entry point for erasing all entries of a filesystem in a DDB table * Needs a -force argument, otherwise it's just a dry run * With tests * And coverage in the testing docs Change-Id: I9b0e5edf8dd728edc2b007535140bc09a566d9a4 --- .../s3guard/AbstractS3GuardDiagnostic.java | 102 ++++++++++--- .../fs/s3a/s3guard/DumpS3GuardTable.java | 64 ++++---- .../fs/s3a/s3guard/PurgeS3GuardTable.java | 141 ++++++++++++++++-- .../site/markdown/tools/hadoop-aws/testing.md | 74 ++++++--- .../s3guard/ITestDynamoDBMetadataStore.java | 66 +++++++- .../s3guard/ITestS3GuardRootOperations.java | 11 +- .../s3a/s3guard/ITestS3GuardToolDynamoDB.java | 2 +- .../fs/s3a/s3guard/MetadataStoreTestBase.java | 2 + 8 files changed, 360 insertions(+), 102 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDiagnostic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDiagnostic.java index c418ff9b44cf4..921451e679157 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDiagnostic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDiagnostic.java @@ -18,23 +18,20 @@ package org.apache.hadoop.fs.s3a.s3guard; +import javax.annotation.Nullable; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; import java.util.List; -import com.google.common.base.Preconditions; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.service.launcher.AbstractLaunchableService; import org.apache.hadoop.service.launcher.LauncherExitCodes; import org.apache.hadoop.service.launcher.ServiceLaunchException; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_USAGE; public class AbstractS3GuardDiagnostic extends AbstractLaunchableService { @@ -50,36 +47,62 @@ public AbstractS3GuardDiagnostic(final String name) { super(name); } - public AbstractS3GuardDiagnostic(final String name, - final S3AFileSystem filesystem, - final DynamoDBMetadataStore store, - final URI uri) { + /** + * Constructor. If the store is set then that is the store for the operation, + * otherwise the filesystem's binding is used instead. + * @param name entry point name. + * @param filesystem filesystem + * @param store optional metastore. + * @param uri URI. Must be set if filesystem == null. + */ + public AbstractS3GuardDiagnostic( + final String name, + @Nullable final S3AFileSystem filesystem, + @Nullable final DynamoDBMetadataStore store, + @Nullable final URI uri) { super(name); this.store = store; this.filesystem = filesystem; + if (store == null) { + require(filesystem != null, "No filesystem or URI"); + bindStore(filesystem); + } if (uri == null) { - checkArgument(filesystem != null, "No filesystem or URI"); - // URI always gets a trailing / - setUri(filesystem.getUri().toString()); + require(filesystem != null, "No filesystem or URI"); + setUri(filesystem.getUri()); } else { setUri(uri); } - if (store == null) { - bindStore(filesystem); - } } - private static void require(boolean condition, String error) { + /** + * Require a condition to hold, otherwise an exception is thrown. + * @param condition condition to be true + * @param error text on failure. + * @throws ServiceLaunchException if the condition is not met + */ + protected static void require(boolean condition, String error) { if (!condition) { - throw fail(error); + throw failure(error); } } - private static ServiceLaunchException fail(String message, Throwable ex) { + /** + * Generate a failure exception for throwing. + * @param message message + * @param ex optional nested exception. + * @return an exception to throw + */ + protected static ServiceLaunchException failure(String message, Throwable ex) { return new ServiceLaunchException(LauncherExitCodes.EXIT_FAIL, message, ex); } - private static ServiceLaunchException fail(String message) { + /** + * Generate a failure exception for throwing. + * @param message message + * @return an exception to throw + */ + protected static ServiceLaunchException failure(String message) { return new ServiceLaunchException(LauncherExitCodes.EXIT_FAIL, message); } @@ -91,12 +114,21 @@ public Configuration bindArgs(final Configuration config, return super.bindArgs(config, args); } - public List getArguments() { + /** + * Get the argument list. + * @return the argument list. + */ + protected List getArguments() { return arguments; } - protected void bindFromCLI(String fsURI ) - throws IOException, URISyntaxException { + /** + * Bind to the store from a CLI argument. + * @param fsURI filesystem URI + * @throws IOException failure + */ + protected void bindFromCLI(String fsURI) + throws IOException { Configuration conf = getConfig(); setUri(fsURI); FileSystem fs = FileSystem.get(getUri(), conf); @@ -104,9 +136,14 @@ protected void bindFromCLI(String fsURI ) "Not an S3A Filesystem: " + fsURI); filesystem = (S3AFileSystem) fs; bindStore(filesystem); - + setUri(fs.getUri()); } + /** + * Binds the {@link #store} field to the metastore of + * the filesystem -which must have a DDB metastore. + * @param fs filesystem to bind the store to. + */ private void bindStore(final S3AFileSystem fs) { require(fs.hasMetadataStore(), "Filesystem has no metadata store: " + fs.getUri()); @@ -155,4 +192,23 @@ public void setUri(String fsURI) { } } } + + /** + * Get the list of arguments, after validating the list size. + * @param argMin minimum number of entries. + * @param argMax maximum number of entries. + * @param usage Usage message. + * @return the argument list, which will be in the range. + * @throws ServiceLaunchException if the argument list is not valid. + */ + protected List getArgumentList(final int argMin, + final int argMax, + final String usage) { + List arg = getArguments(); + if (arg == null || arg.size() < argMin || arg.size() > argMax) { + // no arguments: usage message + throw new ServiceLaunchException(EXIT_USAGE, usage); + } + return arg; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java index e07cd5c67f481..7e8b22c9fb440 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.s3guard; +import javax.annotation.Nullable; import java.io.Closeable; import java.io.File; import java.io.IOException; @@ -28,11 +29,9 @@ import java.util.Collection; import java.util.Collections; import java.util.Date; -import java.util.LinkedList; import java.util.List; import com.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder; -import com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,7 +56,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.hadoop.fs.s3a.S3AUtils.ACCEPT_ALL; -import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_USAGE; /** * This is a low-level diagnostics entry point which does a CVE/TSV dump of @@ -117,13 +115,10 @@ public DumpS3GuardTable( @Override protected void serviceStart() throws Exception { - if (getStore()== null) { - List arguments = getArguments(); - checkNotNull(arguments, "No arguments"); - Preconditions.checkState(arguments.size() == 2, - "Wrong number of arguments: %s", arguments.size()); - bindFromCLI(arguments.get(0)); - destPath = arguments.get(1); + if (getStore() == null) { + List arg = getArgumentList(2, 2, USAGE_MESSAGE); + bindFromCLI(arg.get(0)); + destPath = arg.get(1); } } @@ -369,8 +364,8 @@ private static String stringify(long millis) { */ public static void main(String[] args) { try { - LinkedList argsList = new LinkedList<>(Arrays.asList(args)); - serviceMain(argsList); + serviceMain(Arrays.asList(args), + new DumpS3GuardTable()); } catch (ExitUtil.ExitException e) { ExitUtil.terminate(e); } @@ -380,25 +375,22 @@ public static void main(String[] args) { * The real main function, which takes the arguments as a list. * Argument 0 MUST be the service classname * @param argsList the list of arguments + * @param service service to launch. */ - public static void serviceMain(List argsList) { - if (argsList.size() != 2) { - // no arguments: usage message - ExitUtil.terminate(new ServiceLaunchException(EXIT_USAGE, USAGE_MESSAGE)); - - } else { - ServiceLauncher serviceLauncher = - new ServiceLauncher<>(NAME); - - ExitUtil.ExitException ex = serviceLauncher.launchService( - new Configuration(), - new DumpS3GuardTable(), - argsList, - false, - true); - if (ex != null) { - throw ex; - } + static void serviceMain( + final List argsList, + final AbstractS3GuardDiagnostic service) { + ServiceLauncher serviceLauncher = + new ServiceLauncher<>(service.getName()); + + ExitUtil.ExitException ex = serviceLauncher.launchService( + new Configuration(), + service, + argsList, + false, + true); + if (ex != null) { + throw ex; } } @@ -415,14 +407,14 @@ public static void serviceMain(List argsList) { * @param uri URI of store -only needed if FS is null. * @throws ExitUtil.ExitException failure. */ - public static void dumpS3GuardStore( - final S3AFileSystem fs, - DynamoDBMetadataStore store, - Configuration conf, + public static void dumpStore( + @Nullable final S3AFileSystem fs, + @Nullable DynamoDBMetadataStore store, + @Nullable Configuration conf, final File destFile, - URI uri) throws ExitUtil.ExitException { + @Nullable URI uri) throws ExitUtil.ExitException { ServiceLauncher serviceLauncher = - new ServiceLauncher<>(""); + new ServiceLauncher<>(NAME); if (conf == null) { conf = checkNotNull(fs, "No filesystem").getConf(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java index 1ac58281f3a1e..4c6f6db99470c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java @@ -18,29 +18,43 @@ package org.apache.hadoop.fs.s3a.s3guard; -import java.io.IOException; +import javax.annotation.Nullable; import java.net.URI; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.List; import com.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder; -import com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.service.Service; import org.apache.hadoop.service.launcher.LauncherExitCodes; import org.apache.hadoop.service.launcher.ServiceLaunchException; +import org.apache.hadoop.service.launcher.ServiceLauncher; +import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.ExitUtil; import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.hadoop.fs.s3a.s3guard.DumpS3GuardTable.serviceMain; import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.PARENT; /** * Purge the S3Guard table of a FileSystem from all entries related to * that table. - * Will fail if there is no table, or the store is in auth mode + * Will fail if there is no table, or the store is in auth mode. + *
    + *   hadoop org.apache.hadoop.fs.s3a.s3guard.PurgeS3GuardTable \
    + *   -force s3a://example-bucket/
    + * 
    + * */ @InterfaceAudience.Private @InterfaceStability.Unstable @@ -51,6 +65,16 @@ public class PurgeS3GuardTable extends AbstractS3GuardDiagnostic { public static final String NAME = "PurgeS3GuardTable"; + public static final String FORCE = "-force"; + + private static final String USAGE_MESSAGE = NAME + + " [-force] "; + + private boolean force; + + private long filesFound; + private long filesDeleted; + public PurgeS3GuardTable(final String name) { super(name); } @@ -59,14 +83,29 @@ public PurgeS3GuardTable() { this(NAME); } + public PurgeS3GuardTable( + final S3AFileSystem filesystem, + final DynamoDBMetadataStore store, + final URI uri, + final boolean force) { + super(NAME, filesystem, store, uri); + this.force = force; + } + @Override protected void serviceStart() throws Exception { if (getStore() == null) { - List arguments = getArguments(); - checkNotNull(arguments, "No arguments"); - Preconditions.checkState(arguments.size() == 1, - "Wrong number of arguments: %s", arguments.size()); - bindFromCLI(arguments.get(0)); + List arg = getArgumentList(1, 2, USAGE_MESSAGE); + String fsURI = arg.get(0); + if (arg.size() == 2) { + if (!arg.get(0).equals(FORCE)) { + throw new ServiceLaunchException(LauncherExitCodes.EXIT_USAGE, + USAGE_MESSAGE); + } + force = true; + fsURI = arg.get(1); + } + bindFromCLI(fsURI); } } @@ -75,10 +114,9 @@ protected void serviceStart() throws Exception { * delete all entries from thtat bucket * @return the exit code. * @throws ServiceLaunchException on failure. - * @throws IOException IO failure. */ @Override - public int execute() throws ServiceLaunchException, IOException { + public int execute() throws ServiceLaunchException { URI uri = getUri(); String host = uri.getHost(); @@ -98,8 +136,87 @@ public int execute() throws ServiceLaunchException, IOException { list.add(p); } }); - LOG.info("Deleting {} entries", list.size()); - tableAccess.delete(list); + int count = list.size(); + filesFound = count; + LOG.info("Found {} entries", count); + if (count > 0) { + if (force) { + DurationInfo duration = + new DurationInfo(LOG, + "deleting %s entries from %s", + count, ddbms.toString()); + tableAccess.delete(list); + duration.close(); + long durationMillis = duration.value(); + long timePerEntry = durationMillis / count; + LOG.info("Time per entry: {} ms", timePerEntry); + filesDeleted = count; + } else { + LOG.info("Delete process will only be executed when " + + FORCE + " is set"); + } + } return LauncherExitCodes.EXIT_SUCCESS; } + + /** + * This is the JVM entry point for the service launcher. + * + * Converts the arguments to a list, instantiates a instance of the class + * then executes it. + * @param args command line arguments. + */ + public static void main(String[] args) { + try { + serviceMain(Arrays.asList(args), new PurgeS3GuardTable()); + } catch (ExitUtil.ExitException e) { + ExitUtil.terminate(e); + } + } + + /** + * Entry point to dump the metastore and s3 store world views + *

    + * Both the FS and the store will be dumped: the store is scanned + * before and after the sequence to show what changes were made to + * the store during the list operation. + * @param fs fs to dump. If null a store must be provided. + * @param store store to dump (fallback to FS) + * @param conf configuration to use (fallback to fs) + * @param uri URI of store -only needed if FS is null. + * @param force force the actual delete + * @return (filesFound, filesDeleted) + * @throws ExitUtil.ExitException failure. + */ + public static Pair purgeStore( + @Nullable final S3AFileSystem fs, + @Nullable DynamoDBMetadataStore store, + @Nullable Configuration conf, + @Nullable URI uri, + boolean force) throws ExitUtil.ExitException { + ServiceLauncher serviceLauncher = + new ServiceLauncher<>(NAME); + + if (conf == null) { + conf = checkNotNull(fs, "No filesystem").getConf(); + } + if (store == null) { + store = (DynamoDBMetadataStore) checkNotNull(fs, "No filesystem") + .getMetadataStore(); + } + PurgeS3GuardTable purge = new PurgeS3GuardTable(fs, + store, + uri, + force); + ExitUtil.ExitException ex = serviceLauncher.launchService( + conf, + purge, + Collections.emptyList(), + false, + true); + if (ex != null && ex.getExitCode() != 0) { + throw ex; + } + return Pair.of(purge.filesFound, purge.filesDeleted); + } } diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md index 83feac299a97b..764c075a4d969 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md @@ -1100,25 +1100,24 @@ property should be configured, and the name of that table should be different incurring AWS charges. -### How to dump the table to a CSV file +### How to Dump the Table and Metastore State -There's an unstable, unsupported command to list the contents of a table -to a CSV, or more specifically a TSV file, on the local system +There's an unstable entry point to list the contents of a table +and S3 filesystem ot a set of TSV files ``` -hadoop org.apache.hadoop.fs.s3a.s3guard.DumpS3GuardTable s3a://bucket-x/ out.csv +hadoop org.apache.hadoop.fs.s3a.s3guard.DumpS3GuardTable s3a://bucket-x/ dir/out ``` -This generates a file which can then be viewed on the command line or editor: + +This generates a set of files prefixed `dir/out-` with different views of the worl. + which can then be viewed on the command line or editor: ``` -"path" "type" "is_auth_dir" "deleted" "is_empty_dir" "len" "updated" "updated_s" "last_modified" "last_modified_s" "etag" "version" -"s3a://bucket-x/FileSystemContractBaseTest" "file" "false" "true" "UNKNOWN" 0 1561484415455 "Tue Jun 25 18:40:15 BST 2019" 1561483826881 "Tue Jun 25 18:30:26 BST 2019" "" "" -"s3a://bucket-x/Users" "file" "false" "true" "UNKNOWN" 0 1561484415455 "Tue Jun 25 18:40:15 BST 2019" 1561484376835 "Tue Jun 25 18:39:36 BST 2019" "" "" -"s3a://bucket-x/dest-6f578c72-eb40-4767-a89d-66a6a5b89578" "file" "false" "true" "UNKNOWN" 0 1561484415455 "Tue Jun 25 18:40:15 BST 2019" 1561483757615 "Tue Jun 25 18:29:17 BST 2019" "" "" -"s3a://bucket-x/file.txt" "file" "false" "true" "UNKNOWN" 0 1561484415455 "Tue Jun 25 18:40:15 BST 2019" 1561484382603 "Tue Jun 25 18:39:42 BST 2019" "" "" -"s3a://bucket-x/fork-0001" "file" "false" "true" "UNKNOWN" 0 1561484415455 "Tue Jun 25 18:40:15 BST 2019" 1561484378086 "Tue Jun 25 18:39:38 BST 2019" "" "" -"s3a://bucket-x/fork-0002" "file" "false" "true" "UNKNOWN" 0 1561484415455 "Tue Jun 25 18:40:15 BST 2019" 1561484380177 "Tue Jun 25 18:39:40 BST 2019" "" "" -"s3a://bucket-x/fork-0003" "file" "false" "true" "UNKNOWN" 0 1561484415455 "Tue Jun 25 18:40:15 BST 2019" 1561484379690 "Tue Jun 25 18:39:39 BST 2019" "" "" +"type" "deleted" "path" "is_auth_dir" "is_empty_dir" "len" "updated" "updated_s" "last_modified" "last_modified_s" "etag" "version" +"file" "true" "s3a://bucket/fork-0001/test/ITestS3AContractDistCp/testDirectWrite/remote" "false" "UNKNOWN" 0 1562171244451 "Wed Jul 03 17:27:24 BST 2019" 1562171244451 "Wed Jul 03 17:27:24 BST 2019" "" "" +"file" "true" "s3a://bucket/Users/stevel/Projects/hadoop-trunk/hadoop-tools/hadoop-aws/target/test-dir/1/5xlPpalRwv/test/new/newdir/file1" "false" "UNKNOWN" 0 1562171518435 "Wed Jul 03 17:31:58 BST 2019" 1562171518435 "Wed Jul 03 17:31:58 BST 2019" "" "" +"file" "true" "s3a://bucket/Users/stevel/Projects/hadoop-trunk/hadoop-tools/hadoop-aws/target/test-dir/1/5xlPpalRwv/test/new/newdir/subdir" "false" "UNKNOWN" 0 1562171518535 "Wed Jul 03 17:31:58 BST 2019" 1562171518535 "Wed Jul 03 17:31:58 BST 2019" "" "" +"file" "true" "s3a://bucket/test/DELAY_LISTING_ME/testMRJob" "false" "UNKNOWN" 0 1562172036299 "Wed Jul 03 17:40:36 BST 2019" 1562172036299 "Wed Jul 03 17:40:36 BST 2019" "" "" ``` This is unstable: the output format may change without warning. @@ -1127,23 +1126,60 @@ They are, currently: | field | meaning | source | |-------|---------| -------| -| `path` | path of an entry | filestatus | | `type` | type | filestatus | -| `is_auth_dir` | directory entry authoritative status | metadata | | `deleted` | tombstone marker | metadata | +| `path` | path of an entry | filestatus | +| `is_auth_dir` | directory entry authoritative status | metadata | | `is_empty_dir` | does the entry represent an empty directory | metadata | -| `len` | | | `len` | file length | filestatus | | `last_modified` | file status last modified | filestatus | | `last_modified_s` | file status last modified as string | filestatus | | `updated` | time (millis) metadata was updated | metadata | | `updated_s` | updated time as a string | metadata | -| `updated` | | metadata | | `etag` | any etag | filestatus | | `version` | any version| filestatus | -As noted: this is unstable; entry list and meaning may change, sorting of output, -the listing algorithm, representation of types, etc. Use at your own risk. +Files generated + +| suffix | content | +|---------------|---------| +| `-scan.csv` | Full scan/dump of the metastore | +| `-store.csv` | Recursive walk through the metastore | +| `-tree.csv` | Treewalk through filesystem `listStatus("/")` calls | +| `-flat.csv` | Flat listing through filesystem `listFiles("/", recursive)` | +| `-s3.csv` | Dump of the S3 Store *only* | +| `-scan-2.csv` | Scan of the store after the previous operations | + +Why the two scan entries? The S3Guard+S3 listing/treewalk operations +may add new entries to the store. + +Note 1: this is unstable; entry list and meaning may change, sorting of output, +the listing algorithm, representation of types, etc. It's expected +uses are: diagnostics, support calls and helping us developers +work out what we've just broken. + +Note 2: This *is* safe to use against an active store; the tables may be inconsistent +due to changes taking place during the dump sequence. + +### Resetting the Metastore: `PurgeS3GuardTable` + +The `PurgeS3GuardTable` entry point `org.apache.hadoop.fs.s3a.s3guard.PurgeS3GuardTable` can +list all entries in a store for a specific filesystem, and delete them. +It *only* deletes those entries in the store for that specific filesystem, +even if the store is shared. + +```bash +hadoop org.apache.hadoop.fs.s3a.s3guard.PurgeS3GuardTable \ + -force s3a://example-bucket/ +``` + +Without the `-force` option the table is scanned, but no entries deleted; +with it then all entries for that filesystem are deleted. +No attempt is made to order the deletion; while the operation is under way +the store is not fully connected (i.e. there may be entries whose parent has +already been deleted). + +Needless to say: *it is not safe to use this against a table in active use.* ### Scale Testing MetadataStore Directly diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java index 27ec6e221c3da..9a66afc01157e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java @@ -46,6 +46,7 @@ import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.contract.s3a.S3AContract; import org.apache.hadoop.fs.s3a.Constants; @@ -1077,7 +1078,7 @@ public void testPruneAgainstInvalidTable() throws Throwable { describe("Create an Invalid listing and prune it"); DynamoDBMetadataStore ms = ITestDynamoDBMetadataStore.ddbmsStatic; - String base = "/testPruneAgainstInvalidTable"; + String base = "/" + getMethodName(); String subdir = base + "/subdir"; Path subDirPath = strToPath(subdir); createNewDirs(base, subdir); @@ -1132,7 +1133,7 @@ public void testPruneAgainstInvalidTable() throws Throwable { @Test public void testPutFileDirectlyUnderTombstone() throws Throwable { describe("Put a file under a tombstone; verify the tombstone"); - String base = "/testPutFileDirectlyUnderTombstone"; + String base = "/" + getMethodName(); long now = getTime(); putTombstone(base, now, null); PathMetadata baseMeta1 = get(base); @@ -1147,7 +1148,7 @@ public void testPutFileDirectlyUnderTombstone() throws Throwable { @Test public void testPruneTombstoneUnderTombstone() throws Throwable { describe("Put a tombsteone under a tombstone, prune the pair"); - String base = "/testPruneTombstoneUnderTombstone"; + String base = "/" + getMethodName(); long now = getTime(); String dir = base + "/dir"; putTombstone(dir, now, null); @@ -1197,7 +1198,7 @@ public void testPruneTombstoneUnderTombstone() throws Throwable { @Test public void testPruneFileUnderTombstone() throws Throwable { describe("Put a file under a tombstone, prune the pair"); - String base = "/testPruneFileUnderTombstone"; + String base = "/" + getMethodName(); long now = getTime(); String dir = base + "/dir"; putTombstone(dir, now, null); @@ -1231,7 +1232,7 @@ public void testPruneFileUnderTombstone() throws Throwable { @Test public void testPutFileDeepUnderTombstone() throws Throwable { describe("Put a file two levels under a tombstone"); - String base = "/testPutFileDeepUnderTombstone"; + String base = "/" + getMethodName(); String dir = base + "/dir"; long now = getTime(); // creating a file MUST create its parents @@ -1275,7 +1276,7 @@ public void testDumpTable() throws Throwable { File buildDir = new File(target).getAbsoluteFile(); String name = "ITestDynamoDBMetadataStore"; File destFile = new File(buildDir, name); - DumpS3GuardTable.dumpS3GuardStore( + DumpS3GuardTable.dumpStore( null, ddbmsStatic, getFileSystem().getConf(), @@ -1290,6 +1291,59 @@ public void testDumpTable() throws Throwable { } } + @Test + public void testPurgeTableNoForce() throws Throwable { + describe("Purge the table"); + + putTombstone("/" + getMethodName(), getTime(), null); + Pair r = PurgeS3GuardTable.purgeStore( + null, + ddbmsStatic, + getFileSystem().getConf(), + fsUri, + false); + + Assertions.assertThat(r.getLeft()). + describedAs("entries found in %s", r) + .isGreaterThanOrEqualTo(1); + Assertions.assertThat(r.getRight()). + describedAs("entries deleted in %s", r) + .isZero(); + } + + @Test + public void testPurgeTableForce() throws Throwable { + describe("Purge the table -force"); + putTombstone("/" + getMethodName(), getTime(), null); + Pair r = PurgeS3GuardTable.purgeStore( + null, + ddbmsStatic, + getFileSystem().getConf(), + fsUri, + true); + Assertions.assertThat(r.getLeft()). + describedAs("entries found in %s", r) + .isGreaterThanOrEqualTo(1); + Assertions.assertThat(r.getRight()). + describedAs("entries deleted in %s", r) + .isEqualTo(r.getLeft()); + + // second iteration will have zero entries; this ensures that + + r = PurgeS3GuardTable.purgeStore( + null, + ddbmsStatic, + getFileSystem().getConf(), + fsUri, + true); + Assertions.assertThat(r.getLeft()). + describedAs("entries found in %s", r) + .isZero(); + Assertions.assertThat(r.getRight()). + describedAs("entries deleted in %s", r) + .isZero(); + } + /** * Assert that an entry exists and is a directory. * @param pathStr path diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java index 21adf1406e775..029d0158498bd 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java @@ -25,7 +25,6 @@ import org.assertj.core.api.Assertions; import org.junit.FixMethodOrder; -import org.junit.Ignore; import org.junit.Test; import org.junit.runners.MethodSorters; @@ -64,7 +63,9 @@ public class ITestS3GuardRootOperations extends AbstractS3ATestBase { private String metastoreUriStr; - private boolean cleaning = false; + // this is a switch you can change in your IDE to enable + // or disable those tests which clean up the metastore. + private final boolean cleaning = false; /** * The test timeout is increased in case previous tests have created @@ -122,7 +123,7 @@ private void assumeCleaningOperation() { public void test_050_dump_metastore() throws Throwable { File destFile = calculateDumpFileBase(); describe("Dumping S3Guard store under %s", destFile); - DumpS3GuardTable.dumpS3GuardStore( + DumpS3GuardTable.dumpStore( null, metastore, getConfiguration(), @@ -134,7 +135,7 @@ public void test_050_dump_metastore() throws Throwable { public void test_060_dump_metastore_and_s3() throws Throwable { File destFile = calculateDumpFileBase(); describe("Dumping S3Guard store under %s", destFile); - DumpS3GuardTable.dumpS3GuardStore( + DumpS3GuardTable.dumpStore( getFileSystem(), metastore, getConfiguration(), @@ -247,7 +248,7 @@ public void test_400_rm_root_recursive() throws Throwable { public void test_600_dump_metastore() throws Throwable { File destFile = calculateDumpFileBase(); describe("Dumping S3Guard store under %s", destFile); - DumpS3GuardTable.dumpS3GuardStore( + DumpS3GuardTable.dumpStore( getFileSystem(), metastore, getConfiguration(), 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 026247e3fbb8b..3b24af2382225 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 @@ -302,7 +302,7 @@ public void testDumpTable() throws Throwable { describe("Dumping metastore %s to %s", fs.getMetadataStore(), destFile); - DumpS3GuardTable.dumpS3GuardStore( + DumpS3GuardTable.dumpStore( fs, null, null, 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 fc3c3fd0788cf..6c2b7074ee60f 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 @@ -120,6 +120,7 @@ protected AbstractMSContract getContract() { @Before public void setUp() throws Exception { + Thread.currentThread().setName("setup"); LOG.debug("== Setup. =="); contract = createContract(); ms = contract.getMetadataStore(); @@ -132,6 +133,7 @@ public void setUp() throws Exception { @After public void tearDown() throws Exception { + Thread.currentThread().setName("teardown"); LOG.debug("== Tear down. =="); if (ms != null) { try { From 817bfa2b0d37221f873c1902e30c598fcbd75125 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 5 Jul 2019 10:56:04 +0100 Subject: [PATCH 12/23] HADOOP-16384 tune dump and prune commands for better UX Change-Id: Idb3c7406ede62a02123a7d4ea66e77dd107d4f47 --- .../hadoop/fs/s3a/s3guard/DumpS3GuardTable.java | 1 + .../hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java | 14 ++++++++++++-- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java index 7e8b22c9fb440..298b94822b465 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java @@ -134,6 +134,7 @@ public int execute() throws ServiceLaunchException, IOException { try { final File scanFile = new File( destPath + SCAN_CSV).getCanonicalFile(); + scanFile.getParentFile().mkdirs(); try (CsvFile csv = new CsvFile(scanFile); DurationInfo ignored = new DurationInfo(LOG, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java index 4c6f6db99470c..269acdf796bd8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java @@ -127,18 +127,28 @@ public int execute() throws ServiceLaunchException { builder.withKeyCondition( ExpressionSpecBuilder.S(PARENT).beginsWith(prefix)); + LOG.info("Scanning for entries with prefix {} to delete from {}", + prefix, ddbms); + Iterable entries = tableAccess.scanMetadata(builder); List list = new ArrayList<>(); entries.iterator().forEachRemaining(e -> { if (!(e instanceof S3GuardTableAccess.VersionMarker)) { Path p = e.getFileStatus().getPath(); - LOG.info("Deleting {}", p); + String type = e.getFileStatus().isFile() ? "file" : "directory"; + boolean tombstone = e.isDeleted(); + if (tombstone) { + type = "tombstone " + type; + } + LOG.info("{} {}", type, p); list.add(p); } }); int count = list.size(); filesFound = count; - LOG.info("Found {} entries", count); + LOG.info("Found {} entries{}", + count, + (count == 0 ? " -nothing to purge": "")); if (count > 0) { if (force) { DurationInfo duration = From eee02eff3126caba167bd4a0b411c75fa144d8b5 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 5 Jul 2019 23:45:20 +0100 Subject: [PATCH 13/23] HADOOP-16384 CTU provides more details on deletion; move onto a non-deprecated exception Change-Id: Ifc7babd679d157e6db20aa9a0c78b22188100fae --- .../java/org/apache/hadoop/fs/contract/ContractTestUtils.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java index b4db3a5803ad8..ad34fda5d09b3 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java @@ -29,7 +29,7 @@ import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.io.IOUtils; import org.junit.Assert; -import org.junit.internal.AssumptionViolatedException; +import org.junit.AssumptionViolatedException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -457,8 +457,10 @@ public static void rejectRootOperation(Path path) throws IOException { public static FileStatus[] deleteChildren(FileSystem fileSystem, Path path, boolean recursive) throws IOException { + LOG.debug("Deleting children of {} (recursive={}", path, recursive); FileStatus[] children = listChildren(fileSystem, path); for (FileStatus entry : children) { + LOG.debug("Deleting {}", entry); fileSystem.delete(entry.getPath(), recursive); } return children; From e68b46353b526248814ffecfb5ff67dd9ac9d26d Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 5 Jul 2019 23:49:03 +0100 Subject: [PATCH 14/23] HADOOP-16384: diagnostics DDB store now has a special log s3guard.Operations which logs PUT/DELETE/TOMBSTONE calls with the application state; that state now has a counter to help provide context DumpS3GuardTable handles situation where a file isn't there when you look closer Change-Id: I49e0e65d6a65adbdf4ec048484bf323a9332d89e --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 7 +- .../fs/s3a/s3guard/BulkOperationState.java | 2 + .../fs/s3a/s3guard/DumpS3GuardTable.java | 11 +- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 185 ++++++++++++++---- .../PathMetadataDynamoDBTranslation.java | 34 ++++ 5 files changed, 197 insertions(+), 42 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 4b5cadec85836..21ac2380d211f 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 @@ -2231,6 +2231,11 @@ private boolean innerDelete(S3AFileStatus status, boolean recursive) throws IOException, AmazonClientException { Path f = status.getPath(); LOG.debug("Delete path {} - recursive {}", f, recursive); + LOG.debug("Type = {}", + status.isFile() ? "File" + : (status.isEmptyDirectory() == Tristate.TRUE + ? "Empty Directory" + : "Directory")); String key = pathToKey(f); @@ -2290,7 +2295,7 @@ private boolean innerDelete(S3AFileStatus status, boolean recursive) metadataStore.deleteSubtree(f, ttlTimeProvider); } } else { - LOG.debug("delete: Path is a file"); + LOG.debug("delete: Path is a file: {}", key); deleteObjectAtPath(f, key, true); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java index c3f48e3063227..caa2f4318376a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java @@ -80,5 +80,7 @@ public enum OperationType { Prune, /** Commit operation. */ Commit, + /** Deletion operation. */ + Delete, } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java index 298b94822b465..15f06d2fe3452 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java @@ -21,6 +21,7 @@ import javax.annotation.Nullable; import java.io.Closeable; import java.io.File; +import java.io.FileNotFoundException; import java.io.IOException; import java.io.PrintWriter; import java.net.URI; @@ -212,7 +213,13 @@ protected int treewalkFilesystem( final CsvFile csv, final Path path) throws IOException { int count = 1; - FileStatus[] fileStatuses = getFilesystem().listStatus(path); + FileStatus[] fileStatuses; + try { + fileStatuses = getFilesystem().listStatus(path); + } catch (FileNotFoundException e) { + LOG.warn("File {} was not found", path); + return 0; + } // entries for (FileStatus fileStatus : fileStatuses) { csv.entry((S3AFileStatus) fileStatus); @@ -602,9 +609,9 @@ void entry(DDBPathMetadata md) { */ void entry(S3AFileStatus fileStatus) { row(ROW_QUOTE_MAP, - fileStatus.getPath().toString(), fileStatus.isDirectory() ? "dir" : "file", "false", + fileStatus.getPath().toString(), "", fileStatus.isEmptyDirectory().name(), fileStatus.getLen(), diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java index 102ea17b3b856..9c7caf54eb67c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java @@ -218,6 +218,19 @@ public class DynamoDBMetadataStore implements MetadataStore, public static final Logger LOG = LoggerFactory.getLogger( DynamoDBMetadataStore.class); + /** + * Name of the operations log. + */ + public static final String OPERATIONS_LOG_NAME = + "org.apache.hadoop.fs.s3a.s3guard.Operations"; + + /** + * A log of all state changing operations to the store; + * only updated at debug level. + */ + public static final Logger OPERATIONS_LOG = LoggerFactory.getLogger( + OPERATIONS_LOG_NAME); + /** parent/child name to use in the version marker. */ public static final String VERSION_MARKER = "../VERSION"; @@ -529,14 +542,14 @@ private void initDataAccessRetries(Configuration config) { @Retries.RetryTranslated public void delete(Path path, ITtlTimeProvider ttlTimeProvider) throws IOException { - innerDelete(path, true, ttlTimeProvider); + innerDelete(path, true, ttlTimeProvider, null); } @Override @Retries.RetryTranslated public void forgetMetadata(Path path) throws IOException { LOG.debug("Forget metadata for {}", path); - innerDelete(path, false, null); + innerDelete(path, false, null, null); } /** @@ -547,11 +560,14 @@ public void forgetMetadata(Path path) throws IOException { * @param tombstone flag to create a tombstone marker * @param ttlTimeProvider The time provider to set last_updated. Must not * be null if tombstone is true. + * @param ancestorState * @throws IOException I/O error. */ @Retries.RetryTranslated - private void innerDelete(final Path path, boolean tombstone, - ITtlTimeProvider ttlTimeProvider) + private void innerDelete(final Path path, + boolean tombstone, + ITtlTimeProvider ttlTimeProvider, + final AncestorState ancestorState) throws IOException { checkPath(path); LOG.debug("Deleting from table {} in region {}: {}", @@ -573,24 +589,28 @@ private void innerDelete(final Path path, boolean tombstone, pmTombstone.setLastUpdated(ttlTimeProvider.getNow()); Item item = PathMetadataDynamoDBTranslation.pathMetadataToItem( new DDBPathMetadata(pmTombstone)); + Item[] items = new Item[1]; writeOp.retry( "Put tombstone", path.toString(), idempotent, () -> { - LOG.debug("Adding tombstone to {}", path); + logPut(ancestorState, items); recordsWritten(1); table.putItem(item); + items[0] = item; }); } else { PrimaryKey key = pathToKey(path); + PrimaryKey[] keys = new PrimaryKey[1]; writeOp.retry( "Delete key", path.toString(), idempotent, () -> { // record the attempt so even on retry the counter goes up. - LOG.debug("Delete key {}", path); + keys[0] = key; + logDelete(ancestorState, keys); recordsDeleted(1); table.deleteItem(key); }); @@ -611,31 +631,35 @@ public void deleteSubtree(Path path, ITtlTimeProvider ttlTimeProvider) LOG.debug("Subtree path {} does not exist; this will be a no-op", path); return; } + ; - // REVISIT - for (DescendantsIterator desc = new DescendantsIterator(this, meta); - desc.hasNext(); ) { - innerDelete(desc.next().getPath(), true, ttlTimeProvider); - } - /* - // Execute via the bounded threadpool. - final List> futures = new ArrayList<>(); - for (DescendantsIterator desc = new DescendantsIterator(this, meta); - desc.hasNext();) { - final Path pathToDelete = desc.next().getPath(); - futures.add(submit(executor, () -> { - innerDelete(pathToDelete, true, ttlTimeProvider); - return null; - })); - if (futures.size() > S3GUARD_DDB_SUBMITTED_TASK_LIMIT) { - // first batch done; block for completion. - waitForCompletion(futures); - futures.clear(); + try(AncestorState state = new AncestorState(this, + BulkOperationState.OperationType.Delete, path)) { + // REVISIT + for (DescendantsIterator desc = new DescendantsIterator(this, meta); + desc.hasNext(); ) { + innerDelete(desc.next().getPath(), true, ttlTimeProvider, state); + } + /* + // Execute via the bounded threadpool. + final List> futures = new ArrayList<>(); + for (DescendantsIterator desc = new DescendantsIterator(this, meta); + desc.hasNext();) { + final Path pathToDelete = desc.next().getPath(); + futures.add(submit(executor, () -> { + innerDelete(pathToDelete, true, ttlTimeProvider); + return null; + })); + if (futures.size() > S3GUARD_DDB_SUBMITTED_TASK_LIMIT) { + // first batch done; block for completion. + waitForCompletion(futures); + futures.clear(); + } } + // now wait for the final set. + waitForCompletion(futures); + */ } - // now wait for the final set. - waitForCompletion(futures); - */ } /** @@ -859,7 +883,8 @@ private Collection completeAncestry( ancestry.put(path, entry); Path parent = path.getParent(); while (!parent.isRoot() && !ancestry.containsKey(parent)) { - if (!ancestorState.findEntry(parent, true)) { + // REVISIT + if (true /*!ancestorState.findEntry(parent, true)*/) { // don't add this entry, but carry on with the parents LOG.debug("auto-create ancestor path {} for child path {}", parent, path); @@ -1039,7 +1064,8 @@ public void move( newItems.addAll(tombstones); } - processBatchWriteRequest(null, pathMetadataToItem(newItems)); + processBatchWriteRequest(ancestorState, + null, pathMetadataToItem(newItems)); } /** @@ -1050,12 +1076,16 @@ public void move( * * As well as retrying on the operation invocation, incomplete * batches are retried until all have been processed. + * + * @param ancestorState ancestor state for logging * @param keysToDelete primary keys to be deleted; can be null * @param itemsToPut new items to be put; can be null * @return the number of iterations needed to complete the call. */ @Retries.RetryTranslated("Outstanding batch items are updated with backoff") - private int processBatchWriteRequest(PrimaryKey[] keysToDelete, + private int processBatchWriteRequest( + @Nullable AncestorState ancestorState, + PrimaryKey[] keysToDelete, Item[] itemsToPut) throws IOException { final int totalToDelete = (keysToDelete == null ? 0 : keysToDelete.length); final int totalToPut = (itemsToPut == null ? 0 : itemsToPut.length); @@ -1120,9 +1150,12 @@ private int processBatchWriteRequest(PrimaryKey[] keysToDelete, } if (itemsToPut != null) { recordsWritten(itemsToPut.length); + logPut(ancestorState, itemsToPut); } if (keysToDelete != null) { recordsDeleted(keysToDelete.length); + logDelete(ancestorState, keysToDelete); + } return batches; } @@ -1241,7 +1274,7 @@ private void innerPut( } LOG.debug("Saving batch of {} items to table {}, region {}", items.length, tableName, region); - processBatchWriteRequest(null, items); + processBatchWriteRequest(ancestorState, null, items); } /** @@ -1304,7 +1337,7 @@ List fullPathsToPut(DDBPathMetadata meta, * @return true iff the item isn't null and, if there is an is_deleted * column, that its value is false. */ - private boolean itemExists(Item item) { + private static boolean itemExists(Item item) { if (item == null) { return false; } @@ -1323,7 +1356,7 @@ static S3AFileStatus makeDirStatus(Path f, String owner) { /** * {@inheritDoc}. * There is retry around building the list of paths to update, but - * the call to {@link #processBatchWriteRequest(PrimaryKey[], Item[])} + * the call to {@link #processBatchWriteRequest(DynamoDBMetadataStore.AncestorState, PrimaryKey[], Item[])} * is only tried once. * @param meta Directory listing metadata. * @param operationState operational state for a bulk update @@ -1345,8 +1378,7 @@ public void put( DDBPathMetadata ddbPathMeta = new DDBPathMetadata(makeDirStatus(path, username), meta.isEmpty(), false, meta.isAuthoritative(), meta.getLastUpdated()); - // put all its ancestors if not present; as an optimization we return at its - // first existent ancestor + // put all its ancestors if not present final AncestorState ancestorState = extractOrCreate(operationState, BulkOperationState.OperationType.Put); // First add any missing ancestors... @@ -1359,7 +1391,8 @@ public void put( // sort so highest-level entries are written to the store first. // if a sequence fails, no orphan entries will have been written. metasToPut.sort(PathOrderComparators.TOPMOST_PM_FIRST); - processBatchWriteRequest(null, pathMetadataToItem(metasToPut)); + processBatchWriteRequest(ancestorState, + null, pathMetadataToItem(metasToPut)); // and add the ancestors synchronized (ancestorState) { metasToPut.forEach(ancestorState::put); @@ -1521,7 +1554,7 @@ private void innerPrune(String keyPrefix, ItemCollection items) if (deletionBatch.size() == S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT) { // lowest path entries get deleted first. deletionBatch.sort(PathOrderComparators.TOPMOST_PATH_LAST); - processBatchWriteRequest(pathToKey(deletionBatch), null); + processBatchWriteRequest(state, pathToKey(deletionBatch), null); // set authoritative false for each pruned dir listing removeAuthoritativeDirFlag(parentPathSet, state); @@ -1537,7 +1570,7 @@ private void innerPrune(String keyPrefix, ItemCollection items) } // final batch of deletes if (!deletionBatch.isEmpty()) { - processBatchWriteRequest(pathToKey(deletionBatch), null); + processBatchWriteRequest(state, pathToKey(deletionBatch), null); // set authoritative false for each pruned dir listing removeAuthoritativeDirFlag(parentPathSet, state); @@ -2359,6 +2392,41 @@ static IOException translateTableWaitFailure( } } + /** + * Log a PUT into the operations log at debug level. + * @param state optional ancestor state. + * @param items items which have been PUT + */ + private static void logPut(@Nullable AncestorState state, Item[] items) { + if (OPERATIONS_LOG.isDebugEnabled()) { + // log the operations + String stateStr = AncestorState.stateAsString(state); + for (Item item : items) { + boolean tombstone = itemExists(item); + OPERATIONS_LOG.debug("{} {} {}", + stateStr, + tombstone ? "TOMBSTONE" : "PUT", + itemPrimaryKeyToString(item)); + } + } + } + + /** + * Log a DELETE into the operations log at debug level. + * @param state optional ancestor state. + * @param keysDeleted keys which were deleted. + */ + private static void logDelete( + @Nullable AncestorState state, PrimaryKey[] keysDeleted) { + if (OPERATIONS_LOG.isDebugEnabled()) { + // log the operations + String stateStr = AncestorState.stateAsString(state); + for (PrimaryKey key : keysDeleted) { + OPERATIONS_LOG.debug("{} DELETE {}", stateStr, primaryKeyToString(key)); + } + } + } + /** * Get the move state passed in; create a new one if needed. * @param state state. @@ -2383,10 +2451,25 @@ private AncestorState extractOrCreate(@Nullable BulkOperationState state, @VisibleForTesting static final class AncestorState extends BulkOperationState { + /** + * Counter of IDs issued. + */ + private static final AtomicLong ID_COUNTER = new AtomicLong(0); + + /** Owning store. */ private final DynamoDBMetadataStore store; + /** The ID of the state; for logging. */ + private final long id; + + /** + * Map of ancestors. + */ private final Map ancestry = new HashMap<>(); + /** + * Destination path. + */ private final Path dest; /** @@ -2403,6 +2486,7 @@ static final class AncestorState extends BulkOperationState { super(operation); this.store = store; this.dest = dest; + this.id = ID_COUNTER.addAndGet(1); } int size() { @@ -2413,11 +2497,16 @@ public Path getDest() { return dest; } + long getId() { + return id; + } + @Override public String toString() { final StringBuilder sb = new StringBuilder( "AncestorState{"); sb.append("operation=").append(getOperation()); + sb.append("id=").append(id); sb.append("; dest=").append(dest); sb.append("; size=").append(size()); sb.append("; paths={") @@ -2492,6 +2581,7 @@ boolean findEntry( @Override public void close() throws IOException { if (LOG.isDebugEnabled() && store != null) { + LOG.debug("Auditing {}",stateAsString(this)); for (Map.Entry entry : ancestry .entrySet()) { Path path = entry.getKey(); @@ -2522,6 +2612,23 @@ public void close() throws IOException { } } } + + /** + * Create a string from the state including operation and ID. + * @param state state to use -may be null + * @return a string for logging. + */ + private static String stateAsString(@Nullable AncestorState state) { + String stateStr; + if (state != null) { + stateStr = String.format("#(%s-%04d)", + state.getOperation(), + state.getId()); + } else { + stateStr = "#()"; + } + return stateStr; + } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java index c067ec56410bd..348dfbfced448 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java @@ -364,4 +364,38 @@ static List pathMetaToDDBPathMeta( .collect(Collectors.toList()); } + /** + * Convert an item's (parent, child) key to a string value + * for logging. There is no validation of the item. + * @param item item. + * @return an s3a:// prefixed string. + */ + static String itemPrimaryKeyToString(Item item) { + String parent = item.getString(PARENT); + String child = item.getString(CHILD); + return "s3a://" + parent + "/" + child; + } + /** + * Convert an item's (parent, child) key to a string value + * for logging. There is no validation of the item. + * @param item item. + * @return an s3a:// prefixed string. + */ + static String primaryKeyToString(PrimaryKey item) { + Collection c = item.getComponents(); + String parent = ""; + String child = ""; + for (KeyAttribute attr : c) { + switch (attr.getName()) { + case PARENT: + parent = attr.getValue().toString(); + break; + case CHILD: + child = attr.getValue().toString(); + break; + default: + } + } + return "s3a://" + parent + "/" + child; + } } From 49d4586cca25f5ccc2bc6b43e771033d0b8e947c Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 8 Jul 2019 16:41:11 +0100 Subject: [PATCH 15/23] HADOOP-16384: remove the pruning from test teardowns This code is changed based on the hypothesis that it's been the pruning which has been rendering the store inconsistent in parallel test runs. Change-Id: Iab33d7eb57891f6ab92a12e4c75db12bfd794d75 --- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 35 +++++++++++++--- .../s3a/impl/ITestPartialRenamesDeletes.java | 40 ------------------- .../s3guard/ITestDynamoDBMetadataStore.java | 16 -------- .../s3a/s3guard/ITestS3GuardToolDynamoDB.java | 13 ++++-- .../fs/s3a/s3guard/MetadataStoreTestBase.java | 32 +++++++++++++++ .../src/test/resources/log4j.properties | 2 + 6 files changed, 72 insertions(+), 66 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java index 9c7caf54eb67c..e55d0ee1d8f2d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java @@ -589,28 +589,24 @@ private void innerDelete(final Path path, pmTombstone.setLastUpdated(ttlTimeProvider.getNow()); Item item = PathMetadataDynamoDBTranslation.pathMetadataToItem( new DDBPathMetadata(pmTombstone)); - Item[] items = new Item[1]; writeOp.retry( "Put tombstone", path.toString(), idempotent, () -> { - logPut(ancestorState, items); + logPut(ancestorState, item); recordsWritten(1); table.putItem(item); - items[0] = item; }); } else { PrimaryKey key = pathToKey(path); - PrimaryKey[] keys = new PrimaryKey[1]; writeOp.retry( "Delete key", path.toString(), idempotent, () -> { // record the attempt so even on retry the counter goes up. - keys[0] = key; - logDelete(ancestorState, keys); + logDelete(ancestorState, key); recordsDeleted(1); table.deleteItem(key); }); @@ -2411,6 +2407,20 @@ private static void logPut(@Nullable AncestorState state, Item[] items) { } } + /** + * Log a PUT into the operations log at debug level. + * @param state optional ancestor state. + * @param item item PUT. + */ + private static void logPut(@Nullable AncestorState state, Item item) { + if (OPERATIONS_LOG.isDebugEnabled()) { + // log the operations + Item[] items = new Item[1]; + items[0] = item; + logPut(state, items); + } + } + /** * Log a DELETE into the operations log at debug level. * @param state optional ancestor state. @@ -2426,6 +2436,19 @@ private static void logDelete( } } } + /** + * Log a DELETE into the operations log at debug level. + * @param state optional ancestor state. + * @param key Deleted key + */ + private static void logDelete( + @Nullable AncestorState state, PrimaryKey key) { + if (OPERATIONS_LOG.isDebugEnabled()) { + PrimaryKey[] keys = new PrimaryKey[1]; + keys[0] = key; + logDelete(state, keys); + } + } /** * Get the move state passed in; create a new one if needed. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java index 942f0b6658077..45ec48845232a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java @@ -286,23 +286,6 @@ public void teardown() throws Exception { super.teardown(); } - /** - * Directory cleanup includes pruning everything under the path. - * This ensures that any in the tree from failed tests don't fill up - * the store with many, many, deleted entries. - * @throws IOException failure. - */ - @Override - protected void deleteTestDirInTeardown() throws IOException { - super.deleteTestDirInTeardown(); - Path path = getContract().getTestPath(); - try { - prune(path); - } catch (IOException e) { - LOG.warn("When pruning the test directory {}", path, e); - } - } - private void assumeRoleTests() { assume("No ARN for role tests", !getAssumedRoleARN().isEmpty()); } @@ -692,10 +675,6 @@ public void testPartialDirDelete() throws Throwable { Assertions.assertThat(readOnlyListing) .as("ReadOnly directory " + directoryList) .containsAll(readOnlyFiles); - - // do this prune in the test as well as teardown, so that the test - // reporting includes it in the runtime of a successful run. - prune(basePath); } /** @@ -753,25 +732,6 @@ private void pathMustExist(Path p) { eval(() -> assertPathExists("Missing path", p)); } - /** - * Prune the store for everything under the test path. - * @param path path. - * @throws IOException on failure. - */ - private void prune(Path path) throws IOException { - S3AFileSystem fs = getFileSystem(); - if (fs.hasMetadataStore()) { - MetadataStore store = fs.getMetadataStore(); - try (DurationInfo ignored = - new DurationInfo(LOG, true, "prune %s", path)) { - store.prune( - MetadataStore.PruneMode.ALL_BY_MODTIME, - System.currentTimeMillis(), - PathMetadataDynamoDBTranslation.pathToParentKey(fs.qualify(path))); - } - } - } - /** * List all files under a path. * @param path path to list diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java index 9a66afc01157e..ed6e4395c6b19 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java @@ -289,22 +289,6 @@ private void deleteAllMetadata() throws IOException { public static void deleteMetadataUnderPath(final DynamoDBMetadataStore ms, final Path path, final boolean suppressErrors) throws IOException { ThrottleTracker throttleTracker = new ThrottleTracker(ms); - try (DurationInfo ignored = new DurationInfo(LOG, true, "prune")) { - ms.prune(PruneMode.ALL_BY_MODTIME, - System.currentTimeMillis(), - PathMetadataDynamoDBTranslation.pathToParentKey(path)); - LOG.info("Throttle statistics: {}", throttleTracker); - } catch (FileNotFoundException fnfe) { - // there is no table. - return; - } catch (IOException ioe) { - // prune failed. warn and then fall back to forget. - LOG.warn("Failed to prune {}", path, ioe); - if (!suppressErrors) { - throw ioe; - } - } - // and after the pruning, make sure all other metadata is gone int forgotten = 0; try (DurationInfo ignored = new DurationInfo(LOG, true, "forget")) { PathMetadata meta = ms.get(path); 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 3b24af2382225..ddd47e6636908 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 @@ -39,10 +39,12 @@ import com.amazonaws.services.dynamodbv2.model.Tag; import org.junit.Assert; import org.junit.Assume; +import org.junit.AssumptionViolatedException; import org.junit.Test; import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Destroy; @@ -66,10 +68,13 @@ public class ITestS3GuardToolDynamoDB extends AbstractS3GuardToolTestBase { @Override public void setup() throws Exception { super.setup(); - MetadataStore ms = getMetadataStore(); - Assume.assumeTrue("Test only applies when DynamoDB is used for S3Guard;" - + "Store is " + (ms == null ? "none" : ms.toString()), - ms instanceof DynamoDBMetadataStore); + try { + getMetadataStore(); + } catch (ClassCastException e) { + throw new AssumptionViolatedException( + "Test only applies when DynamoDB is used for S3Guard Store", + e); + } } 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 6c2b7074ee60f..27468d3d8a94a 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 @@ -1139,6 +1139,38 @@ protected PathMetadata getDirectory(final String pathStr) throws IOException { return meta; } + /** + * Get an entry which must not be marked as an empty directory: + * its empty directory field must be FALSE or UNKNOWN. + * @param pathStr path + * @return the entry + * @throws IOException IO failure. + */ + protected PathMetadata getNonEmptyDirectory(final String pathStr) throws IOException { + PathMetadata meta = getDirectory(pathStr); + assertNotEquals("Path " + pathStr + + " is considered an empty dir " + meta, + Tristate.TRUE, + meta.isEmptyDirectory()); + return meta; + } + + /** + * Get an entry which must be an empty directory. + * its empty directory field must be TRUE. + * @param pathStr path + * @return the entry + * @throws IOException IO failure. + */ + protected PathMetadata getEmptyDirectory(final String pathStr) throws IOException { + PathMetadata meta = getDirectory(pathStr); + assertEquals("Path " + pathStr + + " is not considered an empty dir " + meta, + Tristate.TRUE, + meta.isEmptyDirectory()); + return meta; + } + /** * Convenience to create a fully qualified Path from string. */ diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties index 383dec82204ee..6e20fbcda7efd 100644 --- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties @@ -57,6 +57,8 @@ log4j.logger.org.apache.hadoop.ipc.Server=WARN #log4j.logger.org.apache.hadoop.fs.s3a.Listing=INFO # Log S3Guard classes #log4j.logger.org.apache.hadoop.fs.s3a.s3guard=DEBUG +# if set to debug, this will log the PUT/DELETE operations on a store +log4j.logger.org.apache.hadoop.fs.s3a.s3guard.Operations=DEBUG # Log Committer classes #log4j.logger.org.apache.hadoop.fs.s3a.commit=DEBUG From b915e688f70ae26fe432454de235d1db6a580bb9 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 8 Jul 2019 22:36:30 +0100 Subject: [PATCH 16/23] HADOOP-16384: track down final cause of the tombstone problem; reenable rename speedups * The final cause of tombstones seems to be in ITestS3GuardListConsistency whose test files were appearing the most. Changes: * turn the FS into consistent mode before test teardown * explicitly delete the individually created files, so that in the rename() test we were guaranteeing that they were being deleted Made the turning back on of the rename speedup features a compile time option; turned back on, tests are happy. Planning more retests of this to see that all has gone quiet. Change-Id: If341687a1cbdc905782862f36b473faa9a0df84c --- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 75 ++++++------ .../fs/s3a/ITestS3GuardListConsistency.java | 115 +++++++++++------- 2 files changed, 112 insertions(+), 78 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java index e55d0ee1d8f2d..72d26e5bce9b6 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java @@ -286,6 +286,8 @@ public class DynamoDBMetadataStore implements MetadataStore, private static ValueMap deleteTrackingValueMap = new ValueMap().withBoolean(":false", false); + private static final boolean NEW_OPERATIONS = true; + /** * The maximum number of outstanding operations to submit * before blocking to await completion of all the executors. @@ -623,38 +625,44 @@ public void deleteSubtree(Path path, ITtlTimeProvider ttlTimeProvider) final PathMetadata meta = get(path); // REVISIT - if (meta == null /*|| meta.isDeleted()*/) { + if (meta == null) { LOG.debug("Subtree path {} does not exist; this will be a no-op", path); return; } - ; + if (meta.isDeleted()) { + LOG.debug("Subtree path {} is deleted; this will be a no-op", path); + return; + } try(AncestorState state = new AncestorState(this, BulkOperationState.OperationType.Delete, path)) { // REVISIT - for (DescendantsIterator desc = new DescendantsIterator(this, meta); - desc.hasNext(); ) { - innerDelete(desc.next().getPath(), true, ttlTimeProvider, state); - } - /* - // Execute via the bounded threadpool. - final List> futures = new ArrayList<>(); - for (DescendantsIterator desc = new DescendantsIterator(this, meta); - desc.hasNext();) { - final Path pathToDelete = desc.next().getPath(); - futures.add(submit(executor, () -> { - innerDelete(pathToDelete, true, ttlTimeProvider); - return null; - })); - if (futures.size() > S3GUARD_DDB_SUBMITTED_TASK_LIMIT) { - // first batch done; block for completion. - waitForCompletion(futures); - futures.clear(); + if (!NEW_OPERATIONS) { + + for (DescendantsIterator desc = new DescendantsIterator(this, meta); + desc.hasNext(); ) { + innerDelete(desc.next().getPath(), true, ttlTimeProvider, state); + } + } else { + + // Execute via the bounded threadpool. + final List> futures = new ArrayList<>(); + for (DescendantsIterator desc = new DescendantsIterator(this, meta); + desc.hasNext(); ) { + final Path pathToDelete = desc.next().getPath(); + futures.add(submit(executor, () -> { + innerDelete(pathToDelete, true, ttlTimeProvider, state); + return null; + })); + if (futures.size() > S3GUARD_DDB_SUBMITTED_TASK_LIMIT) { + // first batch done; block for completion. + waitForCompletion(futures); + futures.clear(); + } } + // now wait for the final set. + waitForCompletion(futures); } - // now wait for the final set. - waitForCompletion(futures); - */ } } @@ -880,7 +888,7 @@ private Collection completeAncestry( Path parent = path.getParent(); while (!parent.isRoot() && !ancestry.containsKey(parent)) { // REVISIT - if (true /*!ancestorState.findEntry(parent, true)*/) { + if (!NEW_OPERATIONS || !ancestorState.findEntry(parent, true)) { // don't add this entry, but carry on with the parents LOG.debug("auto-create ancestor path {} for child path {}", parent, path); @@ -2415,9 +2423,7 @@ private static void logPut(@Nullable AncestorState state, Item[] items) { private static void logPut(@Nullable AncestorState state, Item item) { if (OPERATIONS_LOG.isDebugEnabled()) { // log the operations - Item[] items = new Item[1]; - items[0] = item; - logPut(state, items); + logPut(state, new Item[]{ item }); } } @@ -2444,9 +2450,7 @@ private static void logDelete( private static void logDelete( @Nullable AncestorState state, PrimaryKey key) { if (OPERATIONS_LOG.isDebugEnabled()) { - PrimaryKey[] keys = new PrimaryKey[1]; - keys[0] = key; - logDelete(state, keys); + logDelete(state, new PrimaryKey[]{key}); } } @@ -2558,8 +2562,11 @@ DDBPathMetadata put(DDBPathMetadata md) { DDBPathMetadata get(Path p) { // REVISIT: reinstate - return null; -// return ancestry.get(p); + if (NEW_OPERATIONS) { + return ancestry.get(p); + } else { + return null; + } } /** @@ -2596,10 +2603,10 @@ boolean findEntry( * If debug logging is enabled, this does an audit of the store state. * it only logs this; the error messages are created so as they could * be turned into exception messages. - * There reason the audit failures aren't being turned into IOEs is that + * Audit failures aren't being turned into IOEs is that * rename operations delete the source entry and that ends up in the * ancestor state as present - * @throws IOException + * @throws IOException failure */ @Override public void close() throws IOException { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java index 10ebacdbed815..4f6771a1685af 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java @@ -67,6 +67,14 @@ public void setup() throws Exception { invoker = new Invoker(new S3ARetryPolicy(getConfiguration()), Invoker.NO_OP ); + Assume.assumeTrue("No metadata store in test filesystem", + getFileSystem().hasMetadataStore()); + } + + @Override + public void teardown() throws Exception { + clearInconsistency(getFileSystem()); + super.teardown(); } @Override @@ -79,6 +87,9 @@ protected AbstractFSContract createContract(Configuration conf) { // this is a long value to guarantee that the inconsistency holds // even over long-haul connections, and in the debugger too/ conf.setLong(FAIL_INJECT_INCONSISTENCY_MSEC, (long) (60 * 1000)); + + // ask for a new filesystem instance. + return new S3AContract(conf); } @@ -95,7 +106,7 @@ protected AbstractFSContract createContract(Configuration conf) { private void doTestRenameSequence(Path[] mkdirs, Path[] srcdirs, Path[] dstdirs, Path[] yesdirs, Path[] nodirs) throws Exception { S3AFileSystem fs = getFileSystem(); - Assume.assumeTrue(fs.hasMetadataStore()); + if (mkdirs != null) { for (Path mkdir : mkdirs) { @@ -104,8 +115,8 @@ private void doTestRenameSequence(Path[] mkdirs, Path[] srcdirs, clearInconsistency(fs); } - assertTrue("srcdirs and dstdirs must have equal length", - srcdirs.length == dstdirs.length); + assertEquals("srcdirs and dstdirs must have equal length", + srcdirs.length, dstdirs.length); for (int i = 0; i < srcdirs.length; i++) { assertTrue("Rename returned false: " + srcdirs[i] + " -> " + dstdirs[i], fs.rename(srcdirs[i], dstdirs[i])); @@ -119,6 +130,21 @@ private void doTestRenameSequence(Path[] mkdirs, Path[] srcdirs, } } + /** + * Delete an array of paths; log exceptions. + * @param paths paths to delete + */ + private void deletePathsQuietly(Path...paths) { + for (Path dir : paths) { + try { + getFileSystem().delete(dir, true); + } catch (IOException e) { + LOG.info("Failed to delete {}: {}", dir, e.toString()); + LOG.debug("Delete failure:, e"); + } + } + } + /** * Tests that after renaming a directory, the original directory and its * contents are indeed missing and the corresponding new paths are visible. @@ -126,19 +152,29 @@ private void doTestRenameSequence(Path[] mkdirs, Path[] srcdirs, */ @Test public void testConsistentListAfterRename() throws Exception { + Path d1f = path("d1/f"); + Path d1f2 = path("d1/f" + DEFAULT_DELAY_KEY_SUBSTRING); Path[] mkdirs = { - path("d1/f"), - path("d1/f" + DEFAULT_DELAY_KEY_SUBSTRING) + d1f, + d1f2 + }; + Path d1 = path("d1"); + Path[] srcdirs = {d1}; + Path d2 = path("d2"); + Path[] dstdirs = {d2}; + Path d2f2 = path("d2/f" + DEFAULT_DELAY_KEY_SUBSTRING); + Path[] yesdirs = { + d2, path("d2/f"), + d2f2 }; - Path[] srcdirs = {path("d1")}; - Path[] dstdirs = {path("d2")}; - Path[] yesdirs = {path("d2"), path("d2/f"), - path("d2/f" + DEFAULT_DELAY_KEY_SUBSTRING)}; - Path[] nodirs = {path("d1"), path("d1/f"), - path("d1/f" + DEFAULT_DELAY_KEY_SUBSTRING)}; - doTestRenameSequence(mkdirs, srcdirs, dstdirs, yesdirs, nodirs); - getFileSystem().delete(path("d1"), true); - getFileSystem().delete(path("d2"), true); + Path[] nodirs = { + d1, d1f, d1f2}; + try { + doTestRenameSequence(mkdirs, srcdirs, dstdirs, yesdirs, nodirs); + } finally { + clearInconsistency(getFileSystem()); + deletePathsQuietly(d1, d2, d1f, d1f2, d2f2); + } } /** @@ -157,18 +193,23 @@ public void testRollingRenames() throws Exception { Path[] setB = {dir2[0], dir1[0]}; Path[] setC = {dir0[0], dir2[0]}; - for(int i = 0; i < 2; i++) { - Path[] firstSet = i == 0 ? setA : null; - doTestRenameSequence(firstSet, setA, setB, setB, dir0); - doTestRenameSequence(null, setB, setC, setC, dir1); - doTestRenameSequence(null, setC, setA, setA, dir2); - } + try { + for(int i = 0; i < 2; i++) { + Path[] firstSet = i == 0 ? setA : null; + doTestRenameSequence(firstSet, setA, setB, setB, dir0); + doTestRenameSequence(null, setB, setC, setC, dir1); + doTestRenameSequence(null, setC, setA, setA, dir2); + } - S3AFileSystem fs = getFileSystem(); - assertFalse("Renaming deleted file should have failed", - fs.rename(dir2[0], dir1[0])); - assertTrue("Renaming over existing file should have succeeded", - fs.rename(dir1[0], dir0[0])); + S3AFileSystem fs = getFileSystem(); + assertFalse("Renaming deleted file should have failed", + fs.rename(dir2[0], dir1[0])); + assertTrue("Renaming over existing file should have succeeded", + fs.rename(dir1[0], dir0[0])); + } finally { + clearInconsistency(getFileSystem()); + deletePathsQuietly(dir0[0], dir1[0], dir2[0]); + } } /** @@ -179,9 +220,6 @@ public void testRollingRenames() throws Exception { @Test public void testConsistentListAfterDelete() throws Exception { S3AFileSystem fs = getFileSystem(); - // test will fail if NullMetadataStore (the default) is configured: skip it. - Assume.assumeTrue("FS needs to have a metadatastore.", - fs.hasMetadataStore()); // Any S3 keys that contain DELAY_KEY_SUBSTRING will be delayed // in listObjects() results via InconsistentS3Client @@ -223,8 +261,6 @@ public void testConsistentListAfterDelete() throws Exception { @Test public void testConsistentRenameAfterDelete() throws Exception { S3AFileSystem fs = getFileSystem(); - // test will fail if NullMetadataStore (the default) is configured: skip it. - Assume.assumeTrue(fs.hasMetadataStore()); // Any S3 keys that contain DELAY_KEY_SUBSTRING will be delayed // in listObjects() results via InconsistentS3Client @@ -266,10 +302,6 @@ public void testConsistentListStatusAfterPut() throws Exception { S3AFileSystem fs = getFileSystem(); - // This test will fail if NullMetadataStore (the default) is configured: - // skip it. - Assume.assumeTrue(fs.hasMetadataStore()); - // Any S3 keys that contain DELAY_KEY_SUBSTRING will be delayed // in listObjects() results via InconsistentS3Client Path inconsistentPath = @@ -301,9 +333,6 @@ public void testConsistentListStatusAfterPut() throws Exception { @Test public void testConsistentListLocatedStatusAfterPut() throws Exception { final S3AFileSystem fs = getFileSystem(); - // This test will fail if NullMetadataStore (the default) is configured: - // skip it. - Assume.assumeTrue(fs.hasMetadataStore()); String rootDir = "doTestConsistentListLocatedStatusAfterPut"; fs.mkdirs(path(rootDir)); @@ -368,9 +397,6 @@ private void doTestConsistentListLocatedStatusAfterPut(S3AFileSystem fs, @Test public void testConsistentListFiles() throws Exception { final S3AFileSystem fs = getFileSystem(); - // This test will fail if NullMetadataStore (the default) is configured: - // skip it. - Assume.assumeTrue(fs.hasMetadataStore()); final int[] numOfPaths = {0, 2}; for (int dirNum : numOfPaths) { @@ -480,7 +506,6 @@ private static void verifyFileIsListed(Collection listedFiles, @Test public void testCommitByRenameOperations() throws Throwable { S3AFileSystem fs = getFileSystem(); - Assume.assumeTrue(fs.hasMetadataStore()); Path work = path("test-commit-by-rename-" + DEFAULT_DELAY_KEY_SUBSTRING); Path task00 = new Path(work, "task00"); fs.mkdirs(task00); @@ -564,7 +589,6 @@ public void testInconsistentS3ClientDeletes() throws Throwable { @Test public void testListingReturnsVersionMetadata() throws Throwable { S3AFileSystem fs = getFileSystem(); - Assume.assumeTrue(fs.hasMetadataStore()); // write simple file Path file = path("file1"); @@ -577,6 +601,7 @@ public void testListingReturnsVersionMetadata() throws Throwable { assertEquals(1, fileStatuses.length); S3AFileStatus status = (S3AFileStatus) fileStatuses[0]; String eTag = status.getETag(); + assertNotNull("Etag in " + eTag, eTag); String versionId = status.getVersionId(); // get status through recursive directory listing @@ -591,8 +616,10 @@ public void testListingReturnsVersionMetadata() throws Throwable { // ensure eTag and versionId are preserved in directory listing S3ALocatedFileStatus locatedFileStatus = (S3ALocatedFileStatus) files.get(0); - assertEquals(eTag, locatedFileStatus.getETag()); - assertEquals(versionId, locatedFileStatus.getVersionId()); + assertEquals("etag of " + locatedFileStatus, + eTag, locatedFileStatus.getETag()); + assertEquals("versionID of " + locatedFileStatus, + versionId, locatedFileStatus.getVersionId()); } /** From 3c0dd912fe3c3789f03c79ce2593ffd0287fe3a4 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 9 Jul 2019 19:00:09 +0100 Subject: [PATCH 17/23] HADOOP-16384: code review, one more test * reinstate all the disabled HADOOP-15183 ancestor caching * Add a test to explicitly verify that renaming an empty dir removes the marker from S3 * compare with trunk to reduce difference * Add javadocs on new dump and purge classes Change-Id: I80abc8e022a92bbb163226aa4dfa6ef80fcc2b0e --- .../hadoop/fs/contract/ContractTestUtils.java | 2 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 3 +- .../s3guard/AbstractS3GuardDiagnostic.java | 7 ++ .../fs/s3a/s3guard/BulkOperationState.java | 2 + .../fs/s3a/s3guard/DumpS3GuardTable.java | 70 +++++++++++++++- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 72 ++++++++-------- .../fs/s3a/s3guard/PurgeS3GuardTable.java | 12 ++- .../fs/s3a/s3guard/S3GuardTableAccess.java | 82 +++++++++++++++++-- .../hadoop/fs/s3a/ITestS3GuardEmptyDirs.java | 44 ++++++++++ .../fs/s3a/ITestS3GuardListConsistency.java | 13 +-- .../s3guard/ITestDynamoDBMetadataStore.java | 2 +- .../s3a/s3guard/ITestS3GuardToolDynamoDB.java | 1 - .../s3guard/TestDynamoDBMiscOperations.java | 4 +- 13 files changed, 246 insertions(+), 68 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java index ad34fda5d09b3..b7b67390aa77c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java @@ -457,7 +457,7 @@ public static void rejectRootOperation(Path path) throws IOException { public static FileStatus[] deleteChildren(FileSystem fileSystem, Path path, boolean recursive) throws IOException { - LOG.debug("Deleting children of {} (recursive={}", path, recursive); + LOG.debug("Deleting children of {} (recursive={})", path, recursive); FileStatus[] children = listChildren(fileSystem, path); for (FileStatus entry : children) { LOG.debug("Deleting {}", entry); 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 21ac2380d211f..f9148ef47bbfb 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 @@ -1588,7 +1588,8 @@ public S3AStorageStatistics getStorageStatistics() { * @throws IOException if the retry invocation raises one (it shouldn't). */ @Retries.RetryRaw - protected ObjectMetadata getObjectMetadata(String key) throws IOException { + @VisibleForTesting + ObjectMetadata getObjectMetadata(String key) throws IOException { return getObjectMetadata(key, null, invoker,null); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDiagnostic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDiagnostic.java index 921451e679157..aea000dcd3d4f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDiagnostic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDiagnostic.java @@ -33,6 +33,9 @@ import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_USAGE; +/** + * Entry point for diagnostics operations. + */ public class AbstractS3GuardDiagnostic extends AbstractLaunchableService { private S3AFileSystem filesystem; @@ -43,6 +46,10 @@ public class AbstractS3GuardDiagnostic extends AbstractLaunchableService { private List arguments; + /** + * Constructor. + * @param name entry point name. + */ public AbstractS3GuardDiagnostic(final String name) { super(name); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java index caa2f4318376a..a1185da03165a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java @@ -82,5 +82,7 @@ public enum OperationType { Commit, /** Deletion operation. */ Delete, + /** FSCK operation. */ + Fsck } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java index 15f06d2fe3452..3a51b116d7809 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java @@ -71,28 +71,63 @@ public class DumpS3GuardTable extends AbstractS3GuardDiagnostic { private static final Logger LOG = LoggerFactory.getLogger(DumpS3GuardTable.class); + /** + * Application name. + */ public static final String NAME = "DumpS3GuardTable"; + /** + * Usage. + */ private static final String USAGE_MESSAGE = NAME + " "; + /** + * Suffix for the flat list: {@value}. + */ public static final String FLAT_CSV = "-flat.csv"; + /** + * Suffix for the raw S3 dump: {@value}. + */ public static final String RAW_CSV = "-s3.csv"; + /** + * Suffix for the DDB scan: {@value}. + */ public static final String SCAN_CSV = "-scan.csv"; + + /** + * Suffix for the second DDB scan: : {@value}. + */ public static final String SCAN2_CSV = "-scan-2.csv"; + /** + * Suffix for the treewalk scan of the S3A Filesystem: {@value}. + */ public static final String TREE_CSV = "-tree.csv"; + /** + * Suffix for a recursive treewalk through the metastore: {@value}. + */ public static final String STORE_CSV = "-store.csv"; + /** + * Path in the local filesystem to save the data. + */ protected String destPath; + /** + * Instantiate. + * @param name application name. + */ public DumpS3GuardTable(final String name) { super(name); } + /** + * Instantiate with default name. + */ public DumpS3GuardTable() { this(NAME); } @@ -113,7 +148,10 @@ public DumpS3GuardTable( this.destPath = destFile.getAbsolutePath(); } - + /** + * Bind to the argument list, including validating the CLI + * @throws Exception failure. + */ @Override protected void serviceStart() throws Exception { if (getStore() == null) { @@ -332,6 +370,11 @@ private Pair dumpRecursively( } + /** + * Dump a single entry, and log it. + * @param csv CSV output file. + * @param md metadata to log. + */ private void dumpEntry(CsvFile csv, DDBPathMetadata md) { LOG.info("{}", md.prettyPrint()); csv.entry(md); @@ -360,6 +403,11 @@ private int scanMetastore(CsvFile csv) { } + /** + * Convert a timestamp in milliseconds to a human string. + * @param millis epoch time in millis + * @return a string for the CSV file. + */ private static String stringify(long millis) { return new Date(millis).toString(); } @@ -372,8 +420,7 @@ private static String stringify(long millis) { */ public static void main(String[] args) { try { - serviceMain(Arrays.asList(args), - new DumpS3GuardTable()); + serviceMain(Arrays.asList(args), new DumpS3GuardTable()); } catch (ExitUtil.ExitException e) { ExitUtil.terminate(e); } @@ -450,6 +497,8 @@ public static void dumpStore( * * Quotes are manged by passing in a long whose specific bits control * whether or not a row is quoted, bit 0 for column 0, etc. + * + * There is no escaping of values here. */ private static final class CsvFile implements Closeable { @@ -473,6 +522,15 @@ private static final class CsvFile implements Closeable { private final String quote; + /** + * Create. + * @param path filesystem path. + * @param out output write. + * @param separator separator of entries. + * @param eol EOL marker. + * @param quote quote marker. + * @throws IOException failure. + */ private CsvFile( final Path path, final PrintWriter out, @@ -487,6 +545,12 @@ private CsvFile( header(); } + /** + * Create to a file, with UTF-8 output and the standard + * options of the TSV file. + * @param file destination file. + * @throws IOException failure. + */ private CsvFile(File file) throws IOException { this(null, new PrintWriter(file, "UTF-8"), "\t", "\n", "\""); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java index 72d26e5bce9b6..89e8860d69a81 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java @@ -286,8 +286,6 @@ public class DynamoDBMetadataStore implements MetadataStore, private static ValueMap deleteTrackingValueMap = new ValueMap().withBoolean(":false", false); - private static final boolean NEW_OPERATIONS = true; - /** * The maximum number of outstanding operations to submit * before blocking to await completion of all the executors. @@ -562,13 +560,13 @@ public void forgetMetadata(Path path) throws IOException { * @param tombstone flag to create a tombstone marker * @param ttlTimeProvider The time provider to set last_updated. Must not * be null if tombstone is true. - * @param ancestorState + * @param ancestorState ancestor state for logging * @throws IOException I/O error. */ @Retries.RetryTranslated private void innerDelete(final Path path, - boolean tombstone, - ITtlTimeProvider ttlTimeProvider, + final boolean tombstone, + final ITtlTimeProvider ttlTimeProvider, final AncestorState ancestorState) throws IOException { checkPath(path); @@ -624,7 +622,6 @@ public void deleteSubtree(Path path, ITtlTimeProvider ttlTimeProvider) tableName, region, path); final PathMetadata meta = get(path); - // REVISIT if (meta == null) { LOG.debug("Subtree path {} does not exist; this will be a no-op", path); return; @@ -636,19 +633,10 @@ public void deleteSubtree(Path path, ITtlTimeProvider ttlTimeProvider) try(AncestorState state = new AncestorState(this, BulkOperationState.OperationType.Delete, path)) { - // REVISIT - if (!NEW_OPERATIONS) { - - for (DescendantsIterator desc = new DescendantsIterator(this, meta); - desc.hasNext(); ) { - innerDelete(desc.next().getPath(), true, ttlTimeProvider, state); - } - } else { - // Execute via the bounded threadpool. final List> futures = new ArrayList<>(); for (DescendantsIterator desc = new DescendantsIterator(this, meta); - desc.hasNext(); ) { + desc.hasNext();) { final Path pathToDelete = desc.next().getPath(); futures.add(submit(executor, () -> { innerDelete(pathToDelete, true, ttlTimeProvider, state); @@ -663,7 +651,6 @@ public void deleteSubtree(Path path, ITtlTimeProvider ttlTimeProvider) // now wait for the final set. waitForCompletion(futures); } - } } /** @@ -887,8 +874,7 @@ private Collection completeAncestry( ancestry.put(path, entry); Path parent = path.getParent(); while (!parent.isRoot() && !ancestry.containsKey(parent)) { - // REVISIT - if (!NEW_OPERATIONS || !ancestorState.findEntry(parent, true)) { + if (!ancestorState.findEntry(parent, true)) { // don't add this entry, but carry on with the parents LOG.debug("auto-create ancestor path {} for child path {}", parent, path); @@ -1120,10 +1106,8 @@ private int processBatchWriteRequest( S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT - numToDelete, totalToDelete + totalToPut - count); final int index = count - totalToDelete; - Item[] toPut = Arrays.copyOfRange(itemsToPut, index, - index + numToPut); - LOG.debug("Putting {} entries: {}", toPut.length, toPut); - writeItems.withItemsToPut(toPut); + writeItems.withItemsToPut( + Arrays.copyOfRange(itemsToPut, index, index + numToPut)); count += numToPut; } @@ -1360,7 +1344,8 @@ static S3AFileStatus makeDirStatus(Path f, String owner) { /** * {@inheritDoc}. * There is retry around building the list of paths to update, but - * the call to {@link #processBatchWriteRequest(DynamoDBMetadataStore.AncestorState, PrimaryKey[], Item[])} + * the call to + * {@link #processBatchWriteRequest(DynamoDBMetadataStore.AncestorState, PrimaryKey[], Item[])} * is only tried once. * @param meta Directory listing metadata. * @param operationState operational state for a bulk update @@ -1376,7 +1361,6 @@ public void put( meta.isAuthoritative() ? "auth" : "nonauth", meta.getPath(), tableName, region, meta); - // directory path Path path = meta.getPath(); DDBPathMetadata ddbPathMeta = @@ -1396,7 +1380,8 @@ public void put( // if a sequence fails, no orphan entries will have been written. metasToPut.sort(PathOrderComparators.TOPMOST_PM_FIRST); processBatchWriteRequest(ancestorState, - null, pathMetadataToItem(metasToPut)); + null, + pathMetadataToItem(metasToPut)); // and add the ancestors synchronized (ancestorState) { metasToPut.forEach(ancestorState::put); @@ -2401,7 +2386,9 @@ static IOException translateTableWaitFailure( * @param state optional ancestor state. * @param items items which have been PUT */ - private static void logPut(@Nullable AncestorState state, Item[] items) { + private static void logPut( + @Nullable AncestorState state, + Item[] items) { if (OPERATIONS_LOG.isDebugEnabled()) { // log the operations String stateStr = AncestorState.stateAsString(state); @@ -2420,7 +2407,9 @@ private static void logPut(@Nullable AncestorState state, Item[] items) { * @param state optional ancestor state. * @param item item PUT. */ - private static void logPut(@Nullable AncestorState state, Item item) { + private static void logPut( + @Nullable AncestorState state, + Item item) { if (OPERATIONS_LOG.isDebugEnabled()) { // log the operations logPut(state, new Item[]{ item }); @@ -2433,22 +2422,26 @@ private static void logPut(@Nullable AncestorState state, Item item) { * @param keysDeleted keys which were deleted. */ private static void logDelete( - @Nullable AncestorState state, PrimaryKey[] keysDeleted) { + @Nullable AncestorState state, + PrimaryKey[] keysDeleted) { if (OPERATIONS_LOG.isDebugEnabled()) { // log the operations String stateStr = AncestorState.stateAsString(state); for (PrimaryKey key : keysDeleted) { - OPERATIONS_LOG.debug("{} DELETE {}", stateStr, primaryKeyToString(key)); + OPERATIONS_LOG.debug("{} DELETE {}", + stateStr, primaryKeyToString(key)); } } } + /** * Log a DELETE into the operations log at debug level. * @param state optional ancestor state. * @param key Deleted key */ private static void logDelete( - @Nullable AncestorState state, PrimaryKey key) { + @Nullable AncestorState state, + PrimaryKey key) { if (OPERATIONS_LOG.isDebugEnabled()) { logDelete(state, new PrimaryKey[]{key}); } @@ -2561,12 +2554,7 @@ DDBPathMetadata put(DDBPathMetadata md) { } DDBPathMetadata get(Path p) { - // REVISIT: reinstate - if (NEW_OPERATIONS) { - return ancestry.get(p); - } else { - return null; - } + return ancestry.get(p); } /** @@ -2621,7 +2609,14 @@ public void close() throws IOException { // any more continue; } - DDBPathMetadata actual = store.get(path); + DDBPathMetadata actual; + try { + actual = store.get(path); + } catch (IOException e) { + LOG.debug("Retrieving {}", path, e); + // this is for debug; don't be ambitious + return; + } if (actual == null || actual.isDeleted()) { String message = "Metastore entry for path " + path + " deleted during bulk " @@ -2660,5 +2655,4 @@ private static String stateAsString(@Nullable AncestorState state) { return stateStr; } } - } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java index 269acdf796bd8..f88ac66b9308d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java @@ -92,6 +92,10 @@ public PurgeS3GuardTable( this.force = force; } + /** + * Bind to the argument list, including validating the CLI + * @throws Exception failure. + */ @Override protected void serviceStart() throws Exception { if (getStore() == null) { @@ -111,7 +115,7 @@ protected void serviceStart() throws Exception { /** * Extract the host from the FS URI, then scan and - * delete all entries from thtat bucket + * delete all entries from that bucket * @return the exit code. * @throws ServiceLaunchException on failure. */ @@ -170,7 +174,7 @@ public int execute() throws ServiceLaunchException { } /** - * This is the JVM entry point for the service launcher. + * This is the Main entry point for the service launcher. * * Converts the arguments to a list, instantiates a instance of the class * then executes it. @@ -185,7 +189,7 @@ public static void main(String[] args) { } /** - * Entry point to dump the metastore and s3 store world views + * API Entry point to dump the metastore and S3 store world views *

    * Both the FS and the store will be dumped: the store is scanned * before and after the sequence to show what changes were made to @@ -198,6 +202,8 @@ public static void main(String[] args) { * @return (filesFound, filesDeleted) * @throws ExitUtil.ExitException failure. */ + @InterfaceAudience.Private + @InterfaceStability.Unstable public static Pair purgeStore( @Nullable final S3AFileSystem fs, @Nullable DynamoDBMetadataStore store, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTableAccess.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTableAccess.java index af2dab781dc03..5592faafe3ebd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTableAccess.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTableAccess.java @@ -29,7 +29,6 @@ import com.amazonaws.services.dynamodbv2.document.internal.IteratorSupport; import com.amazonaws.services.dynamodbv2.document.spec.QuerySpec; import com.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder; -import com.amazonaws.services.dynamodbv2.xspec.ScanExpressionSpec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -64,7 +63,7 @@ * *

  • * If a scan or query includes the version marker in the result, it - * is converted to a VersionMarker instance. + * is converted to a {@link VersionMarker} instance. *
  • * * @@ -76,24 +75,49 @@ class S3GuardTableAccess { private static final Logger LOG = LoggerFactory.getLogger(S3GuardTableAccess.class); + /** + * Store instance to work with. + */ private final DynamoDBMetadataStore store; + /** + * Table; retrieved from the store. + */ private final Table table; + /** + * Construct. + * @param store store to work with. + */ S3GuardTableAccess(final DynamoDBMetadataStore store) { this.store = checkNotNull(store); - this.table = store.getTable(); + this.table = checkNotNull(store.getTable()); } + /** + * Username of user in store. + * @return a string. + */ private String getUsername() { return store.getUsername(); } - + /** + * Execute a query. + * @param spec query spec. + * @return the outcome. + */ ItemCollection query(QuerySpec spec) { return table.query(spec); } + /** + * Issue a query where the result is to be an iterator over + * the entries + * of DDBPathMetadata instances. + * @param spec query spec. + * @return an iterator over path entries. + */ Iterable queryMetadata(QuerySpec spec) { return new DDBPathMetadataCollection<>(query(spec)); } @@ -112,15 +136,32 @@ void delete(Collection paths) { .forEach(table::deleteItem); } + /** + * A collection which wraps the result of a query or scan. + * Important: iterate through this only once; the outcome + * of repeating an iteration is "undefined" + * @param type of outcome. + */ private final class DDBPathMetadataCollection implements Iterable { + /** + * Query/scan result. + */ private final ItemCollection outcome; + /** + * Instantiate. + * @param outcome query/scan outcome. + */ private DDBPathMetadataCollection(final ItemCollection outcome) { this.outcome = outcome; } + /** + * Get the iterator. + * @return the iterator. + */ @Override public Iterator iterator() { return new DDBPathMetadataIterator<>(outcome.iterator()); @@ -128,11 +169,23 @@ public Iterator iterator() { } + /** + * An iterator which converts the iterated-over result of + * a query or scan into a {@code DDBPathMetadataIterator} entry. + * @param type of source. + */ private final class DDBPathMetadataIterator implements Iterator { + /** + * Iterator to invoke. + */ private final IteratorSupport it; + /** + * Instantiate. + * @param it Iterator to invoke. + */ private DDBPathMetadataIterator(final IteratorSupport it) { this.it = it; } @@ -157,15 +210,32 @@ public DDBPathMetadata next() { } + /** + * DDBPathMetadata subclass returned when a query returns + * the version marker. + * There is a FileStatus returned where the owner field contains + * the table version; the path is always the unqualified path "/VERSION". + * Because it is unqualified, operations which treat this as a normal + * DDB metadata entry usually fail. + */ static final class VersionMarker extends DDBPathMetadata { + /** + * Instantiate. + * @param versionMarker the version marker. + */ VersionMarker(Item versionMarker) { super(new S3AFileStatus(true, new Path("/VERSION"), "" + versionMarker.getString(TABLE_VERSION))); } } - private static Pair primaryKey(Item it) { - return Pair.of(it.getString(PARENT), it.getString(CHILD)); + /** + * Given an item, split it to the parent and child fields. + * @param item item to split. + * @return (parent, child). + */ + private static Pair primaryKey(Item item) { + return Pair.of(item.getString(PARENT), item.getString(CHILD)); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java index fb6e3701d3227..c086597355f47 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java @@ -18,13 +18,18 @@ package org.apache.hadoop.fs.s3a; +import java.io.FileNotFoundException; +import java.io.IOException; + import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore; import org.junit.Assume; import org.junit.Test; +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test logic around whether or not a directory is empty, with S3Guard enabled. @@ -37,6 +42,45 @@ */ public class ITestS3GuardEmptyDirs extends AbstractS3ATestBase { + @Test + public void testRenameEmptyDir() throws Throwable { + S3AFileSystem fs = getFileSystem(); + Path basePath = path(getMethodName()); + Path sourceDir = new Path(basePath, "AAA-source"); + String sourceDirMarker = fs.pathToKey(sourceDir) + "/"; + Path destDir = new Path(basePath, "BBB-dest"); + String destDirMarker = fs.pathToKey(destDir) + "/"; + // set things up. + mkdirs(sourceDir); + // there'a source directory marker + fs.getObjectMetadata(sourceDirMarker); + S3AFileStatus srcStatus = getEmptyDirStatus(sourceDir); + assertEquals("Must be an empty dir: " + srcStatus, Tristate.TRUE, + srcStatus.isEmptyDirectory()); + // do the rename + assertRenameOutcome(fs, sourceDir, destDir, true); + S3AFileStatus destStatus = getEmptyDirStatus(destDir); + assertEquals("Must be an empty dir: " + destStatus, Tristate.TRUE, + destStatus.isEmptyDirectory()); + // source does not exist. + intercept(FileNotFoundException.class, + () -> getEmptyDirStatus(sourceDir)); + // and verify that there's no dir marker hidden under a tombstone + intercept(FileNotFoundException.class, + () -> Invoker.once("HEAD", sourceDirMarker, + () -> fs.getObjectMetadata(sourceDirMarker))); + // the parent dir mustn't be confused + S3AFileStatus baseStatus = getEmptyDirStatus(basePath); + assertEquals("Must not be an empty dir: " + baseStatus, Tristate.FALSE, + baseStatus.isEmptyDirectory()); + // and verify the dest dir has a marker + fs.getObjectMetadata(destDirMarker); + } + + private S3AFileStatus getEmptyDirStatus(Path dir) throws IOException { + return getFileSystem().innerGetFileStatus(dir, true); + } + @Test public void testEmptyDirs() throws Exception { S3AFileSystem fs = getFileSystem(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java index 4f6771a1685af..799277e0f8498 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java @@ -87,9 +87,6 @@ protected AbstractFSContract createContract(Configuration conf) { // this is a long value to guarantee that the inconsistency holds // even over long-haul connections, and in the debugger too/ conf.setLong(FAIL_INJECT_INCONSISTENCY_MSEC, (long) (60 * 1000)); - - // ask for a new filesystem instance. - return new S3AContract(conf); } @@ -154,19 +151,13 @@ private void deletePathsQuietly(Path...paths) { public void testConsistentListAfterRename() throws Exception { Path d1f = path("d1/f"); Path d1f2 = path("d1/f" + DEFAULT_DELAY_KEY_SUBSTRING); - Path[] mkdirs = { - d1f, - d1f2 - }; + Path[] mkdirs = {d1f, d1f2}; Path d1 = path("d1"); Path[] srcdirs = {d1}; Path d2 = path("d2"); Path[] dstdirs = {d2}; Path d2f2 = path("d2/f" + DEFAULT_DELAY_KEY_SUBSTRING); - Path[] yesdirs = { - d2, path("d2/f"), - d2f2 - }; + Path[] yesdirs = {d2, path("d2/f"), d2f2}; Path[] nodirs = { d1, d1f, d1f2}; try { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java index ed6e4395c6b19..5bcaa57cd6562 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java @@ -1312,7 +1312,7 @@ public void testPurgeTableForce() throws Throwable { describedAs("entries deleted in %s", r) .isEqualTo(r.getLeft()); - // second iteration will have zero entries; this ensures that + // second iteration will have zero entries r = PurgeS3GuardTable.purgeStore( null, 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 ddd47e6636908..28236eccf9177 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 @@ -77,7 +77,6 @@ public void setup() throws Exception { } } - @Override protected DynamoDBMetadataStore getMetadataStore() { return (DynamoDBMetadataStore) super.getMetadataStore(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDynamoDBMiscOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDynamoDBMiscOperations.java index 1e2f7be8c0f42..578aed06bc3ca 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDynamoDBMiscOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDynamoDBMiscOperations.java @@ -115,7 +115,7 @@ public void testInnerListChildrenDirectoryNpe() throws Exception { public void testAncestorStateForDir() throws Throwable { final DynamoDBMetadataStore.AncestorState ancestorState = new DynamoDBMetadataStore.AncestorState( - null, BulkOperationState.OperationType.Rename, null); + null, BulkOperationState.OperationType.Rename, null); // path 1 is a directory final Path path1 = new Path("s3a://bucket/1"); @@ -143,7 +143,7 @@ public void testAncestorStateForDir() throws Throwable { public void testAncestorStateForFile() throws Throwable { final DynamoDBMetadataStore.AncestorState ancestorState = new DynamoDBMetadataStore.AncestorState( - null, BulkOperationState.OperationType.Rename, null); + null, BulkOperationState.OperationType.Rename, null); // path 1 is a file final Path path1 = new Path("s3a://bucket/1"); From 66da306a564647465ff1ceddf0b56d577d6e4d74 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 10 Jul 2019 21:06:37 +0100 Subject: [PATCH 18/23] HADOOP-16384 address Gabor and Yetus feedback Change-Id: I13734049ddabdf3a451c906ca3fd2780d4b49c51 --- .../AbstractContractRootDirectoryTest.java | 8 +-- hadoop-tools/hadoop-aws/pom.xml | 8 ++- ...=> AbstractS3GuardDynamoDBDiagnostic.java} | 17 +++--- ...Table.java => DumpS3GuardDynamoTable.java} | 32 ++++++----- .../fs/s3a/s3guard/DynamoDBMetadataStore.java | 57 ++++++++++--------- ...able.java => PurgeS3GuardDynamoTable.java} | 34 +++++++---- .../hadoop/fs/s3a/s3guard/RenameTracker.java | 4 +- .../site/markdown/tools/hadoop-aws/testing.md | 54 +++++++++--------- .../fs/s3a/ITestS3GuardListConsistency.java | 4 +- .../s3a/impl/ITestPartialRenamesDeletes.java | 2 - .../s3guard/ITestDynamoDBMetadataStore.java | 10 ++-- .../ITestDynamoDBMetadataStoreScale.java | 2 +- ...ava => ITestS3GuardDDBRootOperations.java} | 8 +-- .../s3a/s3guard/ITestS3GuardToolDynamoDB.java | 3 +- .../fs/s3a/s3guard/MetadataStoreTestBase.java | 6 +- .../src/test/resources/log4j.properties | 2 +- 16 files changed, 137 insertions(+), 114 deletions(-) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/{AbstractS3GuardDiagnostic.java => AbstractS3GuardDynamoDBDiagnostic.java} (92%) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/{DumpS3GuardTable.java => DumpS3GuardDynamoTable.java} (95%) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/{PurgeS3GuardTable.java => PurgeS3GuardDynamoTable.java} (89%) rename hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/{ITestS3GuardRootOperations.java => ITestS3GuardDDBRootOperations.java} (97%) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java index 55ce0a5e46586..647ffdf9996c5 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java @@ -159,7 +159,7 @@ public void testRmRootRecursive() throws Throwable { if (deleted) { assertPathDoesNotExist("expected file to be deleted", file); } else { - assertPathExists("expected file to be preserved", file);; + assertPathExists("expected file to be preserved", file); } } finally{ getFileSystem().delete(file, false); @@ -191,10 +191,10 @@ public void testListEmptyRootDirectory() throws IOException { for (FileStatus status : statuses) { ContractTestUtils.assertDeleted(fs, status.getPath(), true); } - FileStatus[] list1 = fs.listStatus(root); + FileStatus[] rootListStatus = fs.listStatus(root); assertEquals("listStatus on empty root-directory returned found: " - + join("\n", list1), - 0, list1.length); + + join("\n", rootListStatus), + 0, rootListStatus.length); assertNoElements("listFiles(/, false)", fs.listFiles(root, false)); assertNoElements("listFiles(/, true)", diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 5bc278358003b..ff330e52dc01e 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -186,11 +186,12 @@ **/ITestS3AHuge*.java **/ITestDynamoDBMetadataStoreScale.java - + **/ITestTerasort*.java + **/ITest*CommitMRJob.java - **/ITestS3GuardRootOperations*.java + **/ITestS3GuardDDBRootOperations.java @@ -228,12 +229,13 @@ **/ITestDynamoDBMetadataStoreScale.java + **/ITestTerasort*.java **/ITest*CommitMRJob.java **/ITestS3AContractRootDir.java - **/ITestS3GuardRootOperations*.java + **/ITestS3GuardDDBRootOperations.java diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDiagnostic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDynamoDBDiagnostic.java similarity index 92% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDiagnostic.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDynamoDBDiagnostic.java index aea000dcd3d4f..b11abb460b811 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDiagnostic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDynamoDBDiagnostic.java @@ -31,12 +31,14 @@ import org.apache.hadoop.service.launcher.LauncherExitCodes; import org.apache.hadoop.service.launcher.ServiceLaunchException; +import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_FAIL; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_USAGE; /** - * Entry point for diagnostics operations. + * Entry point for S3Guard diagnostics operations against DynamoDB tables. */ -public class AbstractS3GuardDiagnostic extends AbstractLaunchableService { +public class AbstractS3GuardDynamoDBDiagnostic + extends AbstractLaunchableService { private S3AFileSystem filesystem; @@ -50,7 +52,7 @@ public class AbstractS3GuardDiagnostic extends AbstractLaunchableService { * Constructor. * @param name entry point name. */ - public AbstractS3GuardDiagnostic(final String name) { + public AbstractS3GuardDynamoDBDiagnostic(final String name) { super(name); } @@ -62,7 +64,7 @@ public AbstractS3GuardDiagnostic(final String name) { * @param store optional metastore. * @param uri URI. Must be set if filesystem == null. */ - public AbstractS3GuardDiagnostic( + public AbstractS3GuardDynamoDBDiagnostic( final String name, @Nullable final S3AFileSystem filesystem, @Nullable final DynamoDBMetadataStore store, @@ -100,8 +102,9 @@ protected static void require(boolean condition, String error) { * @param ex optional nested exception. * @return an exception to throw */ - protected static ServiceLaunchException failure(String message, Throwable ex) { - return new ServiceLaunchException(LauncherExitCodes.EXIT_FAIL, message, ex); + protected static ServiceLaunchException failure(String message, + Throwable ex) { + return new ServiceLaunchException(EXIT_FAIL, message, ex); } /** @@ -110,7 +113,7 @@ protected static ServiceLaunchException failure(String message, Throwable ex) { * @return an exception to throw */ protected static ServiceLaunchException failure(String message) { - return new ServiceLaunchException(LauncherExitCodes.EXIT_FAIL, message); + return new ServiceLaunchException(EXIT_FAIL, message); } @Override diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java similarity index 95% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java index 3a51b116d7809..ce6500bcc2875 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java @@ -42,6 +42,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.s3a.Listing; import org.apache.hadoop.fs.s3a.S3AFileStatus; @@ -66,15 +67,15 @@ */ @InterfaceAudience.Private @InterfaceStability.Unstable -public class DumpS3GuardTable extends AbstractS3GuardDiagnostic { +public class DumpS3GuardDynamoTable extends AbstractS3GuardDynamoDBDiagnostic { private static final Logger LOG = - LoggerFactory.getLogger(DumpS3GuardTable.class); + LoggerFactory.getLogger(DumpS3GuardDynamoTable.class); /** * Application name. */ - public static final String NAME = "DumpS3GuardTable"; + public static final String NAME = "DumpS3GuardDynamoTable"; /** * Usage. @@ -115,20 +116,20 @@ public class DumpS3GuardTable extends AbstractS3GuardDiagnostic { /** * Path in the local filesystem to save the data. */ - protected String destPath; + private String destPath; /** * Instantiate. * @param name application name. */ - public DumpS3GuardTable(final String name) { + public DumpS3GuardDynamoTable(final String name) { super(name); } /** * Instantiate with default name. */ - public DumpS3GuardTable() { + public DumpS3GuardDynamoTable() { this(NAME); } @@ -139,7 +140,7 @@ public DumpS3GuardTable() { * @param destFile the base filename for output * @param uri URI of store -only needed if FS is null. */ - public DumpS3GuardTable( + public DumpS3GuardDynamoTable( final S3AFileSystem fs, final DynamoDBMetadataStore store, final File destFile, @@ -149,7 +150,7 @@ public DumpS3GuardTable( } /** - * Bind to the argument list, including validating the CLI + * Bind to the argument list, including validating the CLI. * @throws Exception failure. */ @Override @@ -173,7 +174,11 @@ public int execute() throws ServiceLaunchException, IOException { try { final File scanFile = new File( destPath + SCAN_CSV).getCanonicalFile(); - scanFile.getParentFile().mkdirs(); + File parentDir = scanFile.getParentFile(); + if (!parentDir.mkdirs() && !parentDir.isDirectory()) { + throw new PathIOException(parentDir.toString(), + "Could not create destination directory"); + } try (CsvFile csv = new CsvFile(scanFile); DurationInfo ignored = new DurationInfo(LOG, @@ -415,12 +420,13 @@ private static String stringify(long millis) { /** * This is the JVM entry point for the service launcher. * - * Converts the arguments to a list, then invokes {@link #serviceMain(List)} + * Converts the arguments to a list, then invokes + * {@link #serviceMain(List, AbstractS3GuardDynamoDBDiagnostic)}. * @param args command line arguments. */ public static void main(String[] args) { try { - serviceMain(Arrays.asList(args), new DumpS3GuardTable()); + serviceMain(Arrays.asList(args), new DumpS3GuardDynamoTable()); } catch (ExitUtil.ExitException e) { ExitUtil.terminate(e); } @@ -434,7 +440,7 @@ public static void main(String[] args) { */ static void serviceMain( final List argsList, - final AbstractS3GuardDiagnostic service) { + final AbstractS3GuardDynamoDBDiagnostic service) { ServiceLauncher serviceLauncher = new ServiceLauncher<>(service.getName()); @@ -480,7 +486,7 @@ public static void dumpStore( } ExitUtil.ExitException ex = serviceLauncher.launchService( conf, - new DumpS3GuardTable(fs, + new DumpS3GuardDynamoTable(fs, store, destFile, uri), diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java index 89e8860d69a81..be5de96b504d5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java @@ -633,24 +633,24 @@ public void deleteSubtree(Path path, ITtlTimeProvider ttlTimeProvider) try(AncestorState state = new AncestorState(this, BulkOperationState.OperationType.Delete, path)) { - // Execute via the bounded threadpool. - final List> futures = new ArrayList<>(); - for (DescendantsIterator desc = new DescendantsIterator(this, meta); - desc.hasNext();) { - final Path pathToDelete = desc.next().getPath(); - futures.add(submit(executor, () -> { - innerDelete(pathToDelete, true, ttlTimeProvider, state); - return null; - })); - if (futures.size() > S3GUARD_DDB_SUBMITTED_TASK_LIMIT) { - // first batch done; block for completion. - waitForCompletion(futures); - futures.clear(); - } + // Execute via the bounded threadpool. + final List> futures = new ArrayList<>(); + for (DescendantsIterator desc = new DescendantsIterator(this, meta); + desc.hasNext();) { + final Path pathToDelete = desc.next().getPath(); + futures.add(submit(executor, () -> { + innerDelete(pathToDelete, true, ttlTimeProvider, state); + return null; + })); + if (futures.size() > S3GUARD_DDB_SUBMITTED_TASK_LIMIT) { + // first batch done; block for completion. + waitForCompletion(futures); + futures.clear(); } - // now wait for the final set. - waitForCompletion(futures); } + // now wait for the final set. + waitForCompletion(futures); + } } /** @@ -2412,7 +2412,7 @@ private static void logPut( Item item) { if (OPERATIONS_LOG.isDebugEnabled()) { // log the operations - logPut(state, new Item[]{ item }); + logPut(state, new Item[]{item}); } } @@ -2599,7 +2599,7 @@ boolean findEntry( @Override public void close() throws IOException { if (LOG.isDebugEnabled() && store != null) { - LOG.debug("Auditing {}",stateAsString(this)); + LOG.debug("Auditing {}", stateAsString(this)); for (Map.Entry entry : ancestry .entrySet()) { Path path = entry.getKey(); @@ -2622,16 +2622,17 @@ public void close() throws IOException { + path + " deleted during bulk " + getOperation() + " operation"; LOG.debug(message); - } - if (actual.getFileStatus().isDirectory() != expected.getFileStatus() - .isDirectory()) { - // the type of the entry has changed - String message = "Metastore entry for path " - + path + " changed during bulk " - + getOperation() + " operation" - + " from " + expected - + " to " + actual; - LOG.debug(message); + } else { + if (actual.getFileStatus().isDirectory() != + expected.getFileStatus().isDirectory()) { + // the type of the entry has changed + String message = "Metastore entry for path " + + path + " changed during bulk " + + getOperation() + " operation" + + " from " + expected + + " to " + actual; + LOG.debug(message); + } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardDynamoTable.java similarity index 89% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardDynamoTable.java index f88ac66b9308d..244779abb939b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PurgeS3GuardDynamoTable.java @@ -43,7 +43,7 @@ import org.apache.hadoop.util.ExitUtil; import static com.google.common.base.Preconditions.checkNotNull; -import static org.apache.hadoop.fs.s3a.s3guard.DumpS3GuardTable.serviceMain; +import static org.apache.hadoop.fs.s3a.s3guard.DumpS3GuardDynamoTable.serviceMain; import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.PARENT; /** @@ -51,39 +51,49 @@ * that table. * Will fail if there is no table, or the store is in auth mode. *
    - *   hadoop org.apache.hadoop.fs.s3a.s3guard.PurgeS3GuardTable \
    + *   hadoop org.apache.hadoop.fs.s3a.s3guard.PurgeS3GuardDynamoTable \
      *   -force s3a://example-bucket/
      * 
    * */ @InterfaceAudience.Private @InterfaceStability.Unstable -public class PurgeS3GuardTable extends AbstractS3GuardDiagnostic { +public class PurgeS3GuardDynamoTable + extends AbstractS3GuardDynamoDBDiagnostic { private static final Logger LOG = - LoggerFactory.getLogger(PurgeS3GuardTable.class); + LoggerFactory.getLogger(PurgeS3GuardDynamoTable.class); - public static final String NAME = "PurgeS3GuardTable"; + public static final String NAME = "PurgeS3GuardDynamoTable"; + /** + * Name of the force option. + */ public static final String FORCE = "-force"; + /** + * Usage message. + */ private static final String USAGE_MESSAGE = NAME + " [-force] "; + /** + * Flag which actually triggers the delete. + */ private boolean force; private long filesFound; private long filesDeleted; - public PurgeS3GuardTable(final String name) { + public PurgeS3GuardDynamoTable(final String name) { super(name); } - public PurgeS3GuardTable() { + public PurgeS3GuardDynamoTable() { this(NAME); } - public PurgeS3GuardTable( + public PurgeS3GuardDynamoTable( final S3AFileSystem filesystem, final DynamoDBMetadataStore store, final URI uri, @@ -93,7 +103,7 @@ public PurgeS3GuardTable( } /** - * Bind to the argument list, including validating the CLI + * Bind to the argument list, including validating the CLI. * @throws Exception failure. */ @Override @@ -115,7 +125,7 @@ protected void serviceStart() throws Exception { /** * Extract the host from the FS URI, then scan and - * delete all entries from that bucket + * delete all entries from that bucket. * @return the exit code. * @throws ServiceLaunchException on failure. */ @@ -182,7 +192,7 @@ public int execute() throws ServiceLaunchException { */ public static void main(String[] args) { try { - serviceMain(Arrays.asList(args), new PurgeS3GuardTable()); + serviceMain(Arrays.asList(args), new PurgeS3GuardDynamoTable()); } catch (ExitUtil.ExitException e) { ExitUtil.terminate(e); } @@ -220,7 +230,7 @@ public static Pair purgeStore( store = (DynamoDBMetadataStore) checkNotNull(fs, "No filesystem") .getMetadataStore(); } - PurgeS3GuardTable purge = new PurgeS3GuardTable(fs, + PurgeS3GuardDynamoTable purge = new PurgeS3GuardDynamoTable(fs, store, uri, force); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java index bfb5c186dd719..19d4568d06db9 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java @@ -220,9 +220,7 @@ public void sourceObjectsDeleted( * @throws IOException failure. */ public void completeRename() throws IOException { - // if the state is doing any final actions, it should be - // completed now. - operationState.close(); + IOUtils.cleanupWithLogger(LOG, operationState); noteRenameFinished(); } diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md index 764c075a4d969..c1ce08ac668fc 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md @@ -845,7 +845,7 @@ it can be manually done: hadoop s3guard uploads -abort -force s3a://test-bucket/ * If you don't need it, destroy the S3Guard DDB table. - hadoop s3guard destroy s3a://bucket-x/ + hadoop s3guard destroy s3a://test-bucket/ The S3Guard tests will automatically create the Dynamo DB table in runs with `-Ds3guard -Ddynamo` set; default capacity of these buckets @@ -1084,13 +1084,14 @@ mvn -T 1C verify -Dtest=skip -Dit.test=ITestS3AMiscOperations -Ds3guard -Ddynamo 1. If the `s3guard` profile is not set, then the S3Guard properties are those of the test configuration set in `contract-test-options.xml` or `auth-keys.xml` -If the `s3guard` profile *is* set, +If the `s3guard` profile *is* set: 1. The S3Guard options from maven (the dynamo and authoritative flags) overwrite any previously set in the configuration files. 1. DynamoDB will be configured to create any missing tables. -1. When using DynamoDB and running ITestDynamoDBMetadataStore, the fs.s3a.s3guard.ddb.test.table -property should be configured, and the name of that table should be different - than what is used for fs.s3a.s3guard.ddb.table. The test table is destroyed +1. When using DynamoDB and running `ITestDynamoDBMetadataStore`, + the `fs.s3a.s3guard.ddb.test.table` +property MUST be configured, and the name of that table MUST be different + than what is used for `fs.s3a.s3guard.ddb.table`. The test table is destroyed and modified multiple times during the test. 1. Several of the tests create and destroy DynamoDB tables. The table names are prefixed with the value defined by @@ -1103,34 +1104,34 @@ property should be configured, and the name of that table should be different ### How to Dump the Table and Metastore State There's an unstable entry point to list the contents of a table -and S3 filesystem ot a set of TSV files +and S3 filesystem ot a set of Tab Separated Value files: ``` -hadoop org.apache.hadoop.fs.s3a.s3guard.DumpS3GuardTable s3a://bucket-x/ dir/out +hadoop org.apache.hadoop.fs.s3a.s3guard.DumpS3GuardDynamoTable s3a://bucket/ dir/out ``` -This generates a set of files prefixed `dir/out-` with different views of the worl. - which can then be viewed on the command line or editor: +This generates a set of files prefixed `dir/out-` with different views of the +world which can then be viewed on the command line or editor: ``` -"type" "deleted" "path" "is_auth_dir" "is_empty_dir" "len" "updated" "updated_s" "last_modified" "last_modified_s" "etag" "version" -"file" "true" "s3a://bucket/fork-0001/test/ITestS3AContractDistCp/testDirectWrite/remote" "false" "UNKNOWN" 0 1562171244451 "Wed Jul 03 17:27:24 BST 2019" 1562171244451 "Wed Jul 03 17:27:24 BST 2019" "" "" -"file" "true" "s3a://bucket/Users/stevel/Projects/hadoop-trunk/hadoop-tools/hadoop-aws/target/test-dir/1/5xlPpalRwv/test/new/newdir/file1" "false" "UNKNOWN" 0 1562171518435 "Wed Jul 03 17:31:58 BST 2019" 1562171518435 "Wed Jul 03 17:31:58 BST 2019" "" "" -"file" "true" "s3a://bucket/Users/stevel/Projects/hadoop-trunk/hadoop-tools/hadoop-aws/target/test-dir/1/5xlPpalRwv/test/new/newdir/subdir" "false" "UNKNOWN" 0 1562171518535 "Wed Jul 03 17:31:58 BST 2019" 1562171518535 "Wed Jul 03 17:31:58 BST 2019" "" "" -"file" "true" "s3a://bucket/test/DELAY_LISTING_ME/testMRJob" "false" "UNKNOWN" 0 1562172036299 "Wed Jul 03 17:40:36 BST 2019" 1562172036299 "Wed Jul 03 17:40:36 BST 2019" "" "" +"type" "deleted" "path" "is_auth_dir" "is_empty_dir" "len" "updated" "updated_s" "last_modified" "last_modified_s" "etag" "version" +"file" "true" "s3a://bucket/fork-0001/test/ITestS3AContractDistCp/testDirectWrite/remote" "false" "UNKNOWN" 0 1562171244451 "Wed Jul 03 17:27:24 BST 2019" 1562171244451 "Wed Jul 03 17:27:24 BST 2019" "" "" +"file" "true" "s3a://bucket/Users/stevel/Projects/hadoop-trunk/hadoop-tools/hadoop-aws/target/test-dir/1/5xlPpalRwv/test/new/newdir/file1" "false" "UNKNOWN" 0 1562171518435 "Wed Jul 03 17:31:58 BST 2019" 1562171518435 "Wed Jul 03 17:31:58 BST 2019" "" "" +"file" "true" "s3a://bucket/Users/stevel/Projects/hadoop-trunk/hadoop-tools/hadoop-aws/target/test-dir/1/5xlPpalRwv/test/new/newdir/subdir" "false" "UNKNOWN" 0 1562171518535 "Wed Jul 03 17:31:58 BST 2019" 1562171518535 "Wed Jul 03 17:31:58 BST 2019" "" "" +"file" "true" "s3a://bucket/test/DELAY_LISTING_ME/testMRJob" "false" "UNKNOWN" 0 1562172036299 "Wed Jul 03 17:40:36 BST 2019" 1562172036299 "Wed Jul 03 17:40:36 BST 2019" "" "" ``` This is unstable: the output format may change without warning. -To understand the meaning of the fields, consult the documentation. +To understand the meaning of the fields, consult the documentation. They are, currently: | field | meaning | source | |-------|---------| -------| | `type` | type | filestatus | -| `deleted` | tombstone marker | metadata | +| `deleted` | tombstone marker | metadata | | `path` | path of an entry | filestatus | -| `is_auth_dir` | directory entry authoritative status | metadata | -| `is_empty_dir` | does the entry represent an empty directory | metadata | +| `is_auth_dir` | directory entry authoritative status | metadata | +| `is_empty_dir` | does the entry represent an empty directory | metadata | | `len` | file length | filestatus | | `last_modified` | file status last modified | filestatus | | `last_modified_s` | file status last modified as string | filestatus | @@ -1150,27 +1151,28 @@ Files generated | `-s3.csv` | Dump of the S3 Store *only* | | `-scan-2.csv` | Scan of the store after the previous operations | -Why the two scan entries? The S3Guard+S3 listing/treewalk operations -may add new entries to the store. +Why the two scan entries? The S3A listing and treewalk operations +may add new entries to the metastore/DynamoDB table. Note 1: this is unstable; entry list and meaning may change, sorting of output, the listing algorithm, representation of types, etc. It's expected uses are: diagnostics, support calls and helping us developers work out what we've just broken. -Note 2: This *is* safe to use against an active store; the tables may be inconsistent -due to changes taking place during the dump sequence. +Note 2: This *is* safe to use against an active store; the tables may appear +to be inconsistent due to changes taking place during the dump sequence. -### Resetting the Metastore: `PurgeS3GuardTable` +### Resetting the Metastore: `PurgeS3GuardDynamoTable` -The `PurgeS3GuardTable` entry point `org.apache.hadoop.fs.s3a.s3guard.PurgeS3GuardTable` can +The `PurgeS3GuardDynamoTable` entry point +`org.apache.hadoop.fs.s3a.s3guard.PurgeS3GuardDynamoTable` can list all entries in a store for a specific filesystem, and delete them. It *only* deletes those entries in the store for that specific filesystem, even if the store is shared. ```bash -hadoop org.apache.hadoop.fs.s3a.s3guard.PurgeS3GuardTable \ - -force s3a://example-bucket/ +hadoop org.apache.hadoop.fs.s3a.s3guard.PurgeS3GuardDynamoTable \ + -force s3a://bucket/ ``` Without the `-force` option the table is scanned, but no entries deleted; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java index 799277e0f8498..f345c48882759 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java @@ -31,6 +31,7 @@ import org.apache.hadoop.fs.contract.s3a.S3AContract; import com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.Test; @@ -602,7 +603,8 @@ public void testListingReturnsVersionMetadata() throws Throwable { while (filesIterator.hasNext()) { files.add(filesIterator.next()); } - assertEquals(1, files.size()); + Assertions.assertThat(files) + .hasSize(1); // ensure eTag and versionId are preserved in directory listing S3ALocatedFileStatus locatedFileStatus = diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java index 45ec48845232a..93b48e772dd87 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java @@ -48,8 +48,6 @@ import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; -import org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation; import org.apache.hadoop.util.BlockingThreadPoolExecutorService; import org.apache.hadoop.util.DurationInfo; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java index 5bcaa57cd6562..68d39dcdb6dde 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStore.java @@ -1260,13 +1260,13 @@ public void testDumpTable() throws Throwable { File buildDir = new File(target).getAbsoluteFile(); String name = "ITestDynamoDBMetadataStore"; File destFile = new File(buildDir, name); - DumpS3GuardTable.dumpStore( + DumpS3GuardDynamoTable.dumpStore( null, ddbmsStatic, getFileSystem().getConf(), destFile, fsUri); - File storeFile = new File(buildDir, name + DumpS3GuardTable.SCAN_CSV); + File storeFile = new File(buildDir, name + DumpS3GuardDynamoTable.SCAN_CSV); try (BufferedReader in = new BufferedReader(new InputStreamReader( new FileInputStream(storeFile), Charset.forName("UTF-8")))) { for (String line : org.apache.commons.io.IOUtils.readLines(in)) { @@ -1280,7 +1280,7 @@ public void testPurgeTableNoForce() throws Throwable { describe("Purge the table"); putTombstone("/" + getMethodName(), getTime(), null); - Pair r = PurgeS3GuardTable.purgeStore( + Pair r = PurgeS3GuardDynamoTable.purgeStore( null, ddbmsStatic, getFileSystem().getConf(), @@ -1299,7 +1299,7 @@ public void testPurgeTableNoForce() throws Throwable { public void testPurgeTableForce() throws Throwable { describe("Purge the table -force"); putTombstone("/" + getMethodName(), getTime(), null); - Pair r = PurgeS3GuardTable.purgeStore( + Pair r = PurgeS3GuardDynamoTable.purgeStore( null, ddbmsStatic, getFileSystem().getConf(), @@ -1314,7 +1314,7 @@ public void testPurgeTableForce() throws Throwable { // second iteration will have zero entries - r = PurgeS3GuardTable.purgeStore( + r = PurgeS3GuardDynamoTable.purgeStore( null, ddbmsStatic, getFileSystem().getConf(), diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java index a4763c9dd6d65..1c6110bed9606 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java @@ -177,7 +177,7 @@ public void teardown() throws Exception { if (ddbms != null) { S3GuardTableAccess tableAccess = new S3GuardTableAccess(ddbms); ExpressionSpecBuilder builder = new ExpressionSpecBuilder(); - builder.withKeyCondition( + builder.withCondition( ExpressionSpecBuilder.S(PARENT).beginsWith("/test/")); Iterable entries = tableAccess.scanMetadata(builder); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java similarity index 97% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java index 029d0158498bd..ccd2f905e82d6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardRootOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java @@ -53,7 +53,7 @@ * The tests only run if DynamoDB is the metastore. */ @FixMethodOrder(MethodSorters.NAME_ASCENDING) -public class ITestS3GuardRootOperations extends AbstractS3ATestBase { +public class ITestS3GuardDDBRootOperations extends AbstractS3ATestBase { private StoreContext storeContext; @@ -123,7 +123,7 @@ private void assumeCleaningOperation() { public void test_050_dump_metastore() throws Throwable { File destFile = calculateDumpFileBase(); describe("Dumping S3Guard store under %s", destFile); - DumpS3GuardTable.dumpStore( + DumpS3GuardDynamoTable.dumpStore( null, metastore, getConfiguration(), @@ -135,7 +135,7 @@ public void test_050_dump_metastore() throws Throwable { public void test_060_dump_metastore_and_s3() throws Throwable { File destFile = calculateDumpFileBase(); describe("Dumping S3Guard store under %s", destFile); - DumpS3GuardTable.dumpStore( + DumpS3GuardDynamoTable.dumpStore( getFileSystem(), metastore, getConfiguration(), @@ -248,7 +248,7 @@ public void test_400_rm_root_recursive() throws Throwable { public void test_600_dump_metastore() throws Throwable { File destFile = calculateDumpFileBase(); describe("Dumping S3Guard store under %s", destFile); - DumpS3GuardTable.dumpStore( + DumpS3GuardDynamoTable.dumpStore( getFileSystem(), metastore, getConfiguration(), 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 28236eccf9177..bf05946691275 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 @@ -44,7 +44,6 @@ import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Destroy; @@ -306,7 +305,7 @@ public void testDumpTable() throws Throwable { describe("Dumping metastore %s to %s", fs.getMetadataStore(), destFile); - DumpS3GuardTable.dumpStore( + DumpS3GuardDynamoTable.dumpStore( fs, null, null, 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 27468d3d8a94a..303cf36bef490 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 @@ -1146,7 +1146,8 @@ protected PathMetadata getDirectory(final String pathStr) throws IOException { * @return the entry * @throws IOException IO failure. */ - protected PathMetadata getNonEmptyDirectory(final String pathStr) throws IOException { + protected PathMetadata getNonEmptyDirectory(final String pathStr) + throws IOException { PathMetadata meta = getDirectory(pathStr); assertNotEquals("Path " + pathStr + " is considered an empty dir " + meta, @@ -1162,7 +1163,8 @@ protected PathMetadata getNonEmptyDirectory(final String pathStr) throws IOExcep * @return the entry * @throws IOException IO failure. */ - protected PathMetadata getEmptyDirectory(final String pathStr) throws IOException { + protected PathMetadata getEmptyDirectory(final String pathStr) + throws IOException { PathMetadata meta = getDirectory(pathStr); assertEquals("Path " + pathStr + " is not considered an empty dir " + meta, diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties index 6e20fbcda7efd..f6162644e2535 100644 --- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties @@ -58,7 +58,7 @@ log4j.logger.org.apache.hadoop.ipc.Server=WARN # Log S3Guard classes #log4j.logger.org.apache.hadoop.fs.s3a.s3guard=DEBUG # if set to debug, this will log the PUT/DELETE operations on a store -log4j.logger.org.apache.hadoop.fs.s3a.s3guard.Operations=DEBUG +#log4j.logger.org.apache.hadoop.fs.s3a.s3guard.Operations=DEBUG # Log Committer classes #log4j.logger.org.apache.hadoop.fs.s3a.commit=DEBUG From 7db3b7c72bca23a8a2099338629a814568c4dc31 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 10 Jul 2019 23:22:25 +0100 Subject: [PATCH 19/23] HADOOP-16384 improve dump operation * summary prints the results * use queues to do the treewalks Change-Id: I4544ace79a5f004141bd81f0fb7a1849fa4b89c0 ITestS3GuardListConsistency: isolates an intermittently failing test within its own subtree ITestS3GuardRootOperations: turn on the pruning --- .../s3a/s3guard/DumpS3GuardDynamoTable.java | 226 ++++++++++++------ .../fs/s3a/ITestS3GuardListConsistency.java | 5 +- .../ITestS3GuardDDBRootOperations.java | 2 +- 3 files changed, 162 insertions(+), 71 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java index ce6500bcc2875..7a273a66c2938 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DumpS3GuardDynamoTable.java @@ -25,14 +25,17 @@ import java.io.IOException; import java.io.PrintWriter; import java.net.URI; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Date; +import java.util.Deque; import java.util.List; import com.amazonaws.services.dynamodbv2.xspec.ExpressionSpecBuilder; +import com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -118,6 +121,16 @@ public class DumpS3GuardDynamoTable extends AbstractS3GuardDynamoDBDiagnostic { */ private String destPath; + private Pair scanEntryResult; + + private Pair secondScanResult; + + private long rawObjectStoreCount; + + private long listStatusCount; + + private long treewalkCount; + /** * Instantiate. * @param name application name. @@ -183,7 +196,7 @@ public int execute() throws ServiceLaunchException, IOException { try (CsvFile csv = new CsvFile(scanFile); DurationInfo ignored = new DurationInfo(LOG, "scanFile dump to %s", scanFile)) { - scanMetastore(csv); + scanEntryResult = scanMetastore(csv); } if (getFilesystem() != null) { @@ -208,7 +221,7 @@ public int execute() throws ServiceLaunchException, IOException { try (CsvFile csv = new CsvFile(treewalkFile); DurationInfo ignored = new DurationInfo(LOG, "Treewalk to %s", treewalkFile)) { - treewalkFilesystem(csv, basePath); + treewalkCount = treewalkFilesystem(csv, basePath); } final File flatlistFile = new File( destPath + FLAT_CSV).getCanonicalFile(); @@ -216,7 +229,7 @@ public int execute() throws ServiceLaunchException, IOException { try (CsvFile csv = new CsvFile(flatlistFile); DurationInfo ignored = new DurationInfo(LOG, "Flat list to %s", flatlistFile)) { - listStatusFilesystem(csv, basePath); + listStatusCount = listStatusFilesystem(csv, basePath); } final File rawFile = new File( destPath + RAW_CSV).getCanonicalFile(); @@ -224,7 +237,7 @@ public int execute() throws ServiceLaunchException, IOException { try (CsvFile csv = new CsvFile(rawFile); DurationInfo ignored = new DurationInfo(LOG, "Raw dump to %s", rawFile)) { - dumpRawS3ObjectStore(csv); + rawObjectStoreCount = dumpRawS3ObjectStore(csv); } final File scanFile2 = new File( destPath + SCAN2_CSV).getCanonicalFile(); @@ -232,9 +245,8 @@ public int execute() throws ServiceLaunchException, IOException { try (CsvFile csv = new CsvFile(scanFile); DurationInfo ignored = new DurationInfo(LOG, "scanFile dump to %s", scanFile2)) { - scanMetastore(csv); + secondScanResult = scanMetastore(csv); } - } return LauncherExitCodes.EXIT_SUCCESS; @@ -245,34 +257,63 @@ public int execute() throws ServiceLaunchException, IOException { } /** - * Dump the filesystem via a recursive treewalk. + * Push all elements of a list to a queue, such that the first entry + * on the list becomes the head of the queue. + * @param queue queue to update + * @param entries list of entries to add. + * @param type of queue + */ + private void pushAll(Deque queue, List entries) { + List reversed = Lists.reverse(entries); + for (T t : reversed) { + queue.push(t); + } + } + + /** + * Dump the filesystem via a treewalk. * If metastore entries mark directories as deleted, this * walk will not explore them. * @param csv destination. + * @param base base path. * @return number of entries found. * @throws IOException IO failure. */ - protected int treewalkFilesystem( + protected long treewalkFilesystem( final CsvFile csv, - final Path path) throws IOException { - int count = 1; - FileStatus[] fileStatuses; - try { - fileStatuses = getFilesystem().listStatus(path); - } catch (FileNotFoundException e) { - LOG.warn("File {} was not found", path); - return 0; - } - // entries - for (FileStatus fileStatus : fileStatuses) { - csv.entry((S3AFileStatus) fileStatus); - } - for (FileStatus fileStatus : fileStatuses) { - if (fileStatus.isDirectory() - && !(fileStatus.getPath().equals(path))) { - count += treewalkFilesystem(csv, fileStatus.getPath()); - } else { - count++; + final Path base) throws IOException { + ArrayDeque queue = new ArrayDeque<>(); + queue.add(base); + long count = 0; + while (!queue.isEmpty()) { + Path path = queue.pop(); + count++; + FileStatus[] fileStatuses; + try { + fileStatuses = getFilesystem().listStatus(path); + } catch (FileNotFoundException e) { + LOG.warn("File {} was not found", path); + continue; + } + // entries + for (FileStatus fileStatus : fileStatuses) { + csv.entry((S3AFileStatus) fileStatus); + } + // scan through the list, building up a reverse list of all directories + // found. + List dirs = new ArrayList<>(fileStatuses.length); + for (FileStatus fileStatus : fileStatuses) { + if (fileStatus.isDirectory() + && !(fileStatus.getPath().equals(path))) { + // directory: add to the end of the queue. + dirs.add(fileStatus.getPath()); + } else { + // file: just increment the count + count++; + } + // now push the dirs list in reverse + // so that they have been added in the sort order as returned. + pushAll(queue, dirs); } } return count; @@ -284,10 +325,10 @@ protected int treewalkFilesystem( * @return number of entries found. * @throws IOException IO failure. */ - protected int listStatusFilesystem( + protected long listStatusFilesystem( final CsvFile csv, final Path path) throws IOException { - int count = 0; + long count = 0; RemoteIterator iterator = getFilesystem() .listFilesAndEmptyDirectories(path, true); while (iterator.hasNext()) { @@ -303,13 +344,13 @@ protected int listStatusFilesystem( * @return number of entries found. * @throws IOException IO failure. */ - protected int dumpRawS3ObjectStore( + protected long dumpRawS3ObjectStore( final CsvFile csv) throws IOException { S3AFileSystem fs = getFilesystem(); Path rootPath = fs.qualify(new Path("/")); Listing listing = new Listing(fs); S3ListRequest request = fs.createListObjectsRequest("", null); - int count = 0; + long count = 0; RemoteIterator st = listing.createFileStatusListingIterator(rootPath, request, ACCEPT_ALL, @@ -333,7 +374,7 @@ protected int dumpRawS3ObjectStore( */ protected void dumpMetastore(final CsvFile csv, final Path basePath) throws IOException { - dumpRecursively(csv, getStore().listChildren(basePath)); + dumpStoreEntries(csv, getStore().listChildren(basePath)); } /** @@ -343,32 +384,36 @@ protected void dumpMetastore(final CsvFile csv, * @return (directories, files) * @throws IOException failure */ - private Pair dumpRecursively( - CsvFile csv, DirListingMetadata dir) throws IOException { - int files = 0, dirs = 1; - List childDirs = new ArrayList<>(); - Collection listing = dir.getListing(); - // sort by name - List sorted = new ArrayList<>(listing); - sorted.sort(new PathOrderComparators.PathMetadataComparator( - (l, r) -> l.compareTo(r))); - - for (PathMetadata pmd : sorted) { - DDBPathMetadata ddbMd = (DDBPathMetadata) pmd; - dumpEntry(csv, ddbMd); - if (ddbMd.getFileStatus().isDirectory()) { - childDirs.add(ddbMd); - } else { - files++; + private Pair dumpStoreEntries( + CsvFile csv, + DirListingMetadata dir) throws IOException { + ArrayDeque queue = new ArrayDeque<>(); + queue.add(dir); + long files = 0, dirs = 1; + while (!queue.isEmpty()) { + DirListingMetadata next = queue.pop(); + List childDirs = new ArrayList<>(); + Collection listing = next.getListing(); + // sort by name + List sorted = new ArrayList<>(listing); + sorted.sort(new PathOrderComparators.PathMetadataComparator( + (l, r) -> l.compareTo(r))); + + for (PathMetadata pmd : sorted) { + DDBPathMetadata ddbMd = (DDBPathMetadata) pmd; + dumpEntry(csv, ddbMd); + if (ddbMd.getFileStatus().isDirectory()) { + childDirs.add(ddbMd); + } else { + files++; + } } - } - for (DDBPathMetadata childDir : childDirs) { - DirListingMetadata children = getStore().listChildren( - childDir.getFileStatus().getPath()); - Pair pair = dumpRecursively(csv, - children); - dirs += pair.getLeft(); - files += pair.getRight(); + List childMD = new ArrayList<>(childDirs.size()); + for (DDBPathMetadata childDir : childDirs) { + childMD.add(getStore().listChildren( + childDir.getFileStatus().getPath())); + } + pushAll(queue, childMD); } return Pair.of(dirs, files); @@ -381,7 +426,7 @@ private Pair dumpRecursively( * @param md metadata to log. */ private void dumpEntry(CsvFile csv, DDBPathMetadata md) { - LOG.info("{}", md.prettyPrint()); + LOG.debug("{}", md.prettyPrint()); csv.entry(md); } @@ -389,24 +434,49 @@ private void dumpEntry(CsvFile csv, DDBPathMetadata md) { * Scan the metastore for all entries and dump them. * There's no attempt to sort the output. * @param csv file - * @return count of the number of entries. + * @return tuple of (live entries, tombstones). */ - private int scanMetastore(CsvFile csv) { + private Pair scanMetastore(CsvFile csv) { S3GuardTableAccess tableAccess = new S3GuardTableAccess(getStore()); ExpressionSpecBuilder builder = new ExpressionSpecBuilder(); Iterable results = tableAccess.scanMetadata( builder); - int count = 0; + long live = 0; + long tombstone = 0; for (DDBPathMetadata md : results) { if (!(md instanceof S3GuardTableAccess.VersionMarker)) { - count++; // print it csv.entry(md); + if (md.isDeleted()) { + tombstone++; + } else { + live++; + } + } } - return count; + return Pair.of(live, tombstone); } + public Pair getScanEntryResult() { + return scanEntryResult; + } + + public Pair getSecondScanResult() { + return secondScanResult; + } + + public long getRawObjectStoreCount() { + return rawObjectStoreCount; + } + + public long getListStatusCount() { + return listStatusCount; + } + + public long getTreewalkCount() { + return treewalkCount; + } /** * Convert a timestamp in milliseconds to a human string. @@ -467,8 +537,9 @@ static void serviceMain( * @param destFile base name of the output files. * @param uri URI of store -only needed if FS is null. * @throws ExitUtil.ExitException failure. + * @return the store */ - public static void dumpStore( + public static DumpS3GuardDynamoTable dumpStore( @Nullable final S3AFileSystem fs, @Nullable DynamoDBMetadataStore store, @Nullable Configuration conf, @@ -484,18 +555,37 @@ public static void dumpStore( store = (DynamoDBMetadataStore) checkNotNull(fs, "No filesystem") .getMetadataStore(); } + DumpS3GuardDynamoTable dump = new DumpS3GuardDynamoTable(fs, + store, + destFile, + uri); ExitUtil.ExitException ex = serviceLauncher.launchService( conf, - new DumpS3GuardDynamoTable(fs, - store, - destFile, - uri), + dump, Collections.emptyList(), false, true); if (ex != null && ex.getExitCode() != 0) { throw ex; } + LOG.info("Results:"); + Pair r = dump.getScanEntryResult(); + LOG.info("Metastore entries: {}", r); + LOG.info("Metastore scan total {}, entries {}, tombstones {}", + r.getLeft() + r.getRight(), + r.getLeft(), + r.getRight()); + LOG.info("S3 count {}", dump.getRawObjectStoreCount()); + LOG.info("Treewalk Count {}", dump.getTreewalkCount()); + LOG.info("List Status Count {}", dump.getListStatusCount()); + r = dump.getSecondScanResult(); + if (r != null) { + LOG.info("Second metastore scan total {}, entries {}, tombstones {}", + r.getLeft() + r.getRight(), + r.getLeft(), + r.getRight()); + } + return dump; } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java index f345c48882759..1503fef98ff90 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java @@ -583,7 +583,8 @@ public void testListingReturnsVersionMetadata() throws Throwable { S3AFileSystem fs = getFileSystem(); // write simple file - Path file = path("file1"); + Path parent = path(getMethodName()); + Path file = new Path(parent, "file1"); try (FSDataOutputStream outputStream = fs.create(file)) { outputStream.writeChars("hello"); } @@ -598,7 +599,7 @@ public void testListingReturnsVersionMetadata() throws Throwable { // get status through recursive directory listing RemoteIterator filesIterator = fs.listFiles( - file.getParent(), true); + parent, true); List files = Lists.newArrayList(); while (filesIterator.hasNext()) { files.add(filesIterator.next()); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java index ccd2f905e82d6..bae39e9ae47f9 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java @@ -65,7 +65,7 @@ public class ITestS3GuardDDBRootOperations extends AbstractS3ATestBase { // this is a switch you can change in your IDE to enable // or disable those tests which clean up the metastore. - private final boolean cleaning = false; + private final boolean cleaning = true; /** * The test timeout is increased in case previous tests have created From 0f85ac793f1448575df2842e7a9b479c12ca2d4b Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 11 Jul 2019 00:02:32 +0100 Subject: [PATCH 20/23] HADOOP-16384 * AbstractITCommitMRJob disables FS caching * ITestS3GuardDDBRootOperations disables some superfluous calls which added ~30s to the runtime Change-Id: Ic5b5f7bdd509f0fbaeb4b7d578a3060515e3b963 --- .../hadoop/fs/s3a/commit/AbstractITCommitMRJob.java | 9 +++++++++ .../fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java | 3 +++ 2 files changed, 12 insertions(+) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitMRJob.java index c5e0265b964bd..ea3fd84ffdafe 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitMRJob.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitMRJob.java @@ -36,6 +36,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.S3AFileSystem; @@ -51,6 +52,7 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.util.DurationInfo; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_STAGING_UUID; /** @@ -61,6 +63,13 @@ public abstract class AbstractITCommitMRJob extends AbstractYarnClusterITest { private static final Logger LOG = LoggerFactory.getLogger(AbstractITCommitMRJob.class); + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + disableFilesystemCaching(conf); + return conf; + } + @Rule public final TemporaryFolder temp = new TemporaryFolder(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java index bae39e9ae47f9..e410abb08cb17 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java @@ -25,6 +25,7 @@ import org.assertj.core.api.Assertions; import org.junit.FixMethodOrder; +import org.junit.Ignore; import org.junit.Test; import org.junit.runners.MethodSorters; @@ -120,6 +121,7 @@ private void assumeCleaningOperation() { } @Test + @Ignore public void test_050_dump_metastore() throws Throwable { File destFile = calculateDumpFileBase(); describe("Dumping S3Guard store under %s", destFile); @@ -245,6 +247,7 @@ public void test_400_rm_root_recursive() throws Throwable { } @Test + @Ignore public void test_600_dump_metastore() throws Throwable { File destFile = calculateDumpFileBase(); describe("Dumping S3Guard store under %s", destFile); From a6235e81d064f93027adce3aa4c1441960c7612f Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 11 Jul 2019 13:25:43 +0100 Subject: [PATCH 21/23] HADOOP-16384: remove unused import from AbstractS3GuardDynamoDBDiagnostic Change-Id: I3a7bd5a76d1bb24d1a8d66a694310f065b5826c4 --- .../hadoop/fs/s3a/s3guard/AbstractS3GuardDynamoDBDiagnostic.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDynamoDBDiagnostic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDynamoDBDiagnostic.java index b11abb460b811..83495ca310623 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDynamoDBDiagnostic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardDynamoDBDiagnostic.java @@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.service.launcher.AbstractLaunchableService; -import org.apache.hadoop.service.launcher.LauncherExitCodes; import org.apache.hadoop.service.launcher.ServiceLaunchException; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_FAIL; From 6008e09b92ac223a5d7d6f8aecab3f5f1ff1720d Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 11 Jul 2019 17:49:12 +0100 Subject: [PATCH 22/23] HADOOP-16384 cut superflous test; correct prune tool usage string With the S3Guardroot operations test, there's no need to have a test of the Dump operation in ITestS3GuardToolDynamoDB; all it does is trigger test timeouts over slow network connections. Prune tool usage entry had "tombstone" entry in wrong place and without the leading "-" Change-Id: Ib71fcf077d407bc1ea5b333044fbb68c86642921 --- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 2 +- .../s3a/s3guard/ITestS3GuardToolDynamoDB.java | 30 ------------------- 2 files changed, 1 insertion(+), 31 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 defcfe19ca07f..08beae2fdec10 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 @@ -1028,8 +1028,8 @@ static class Prune extends S3GuardTool { "Common options:\n" + " -" + META_FLAG + " URL - Metadata repository details " + "(implementation-specific)\n" + + "[-" + TOMBSTONE + "]\n" + "Age options. Any combination of these integer-valued options:\n" + - "[" + TOMBSTONE + "]\n" + AGE_OPTIONS_USAGE + "\n" + "Amazon DynamoDB-specific options:\n" + " -" + REGION_FLAG + " REGION - Service region for connections\n" + 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 bf05946691275..d64da9d89e9ea 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,12 +18,7 @@ package org.apache.hadoop.fs.s3a.s3guard; -import java.io.BufferedReader; -import java.io.File; -import java.io.FileInputStream; import java.io.IOException; -import java.io.InputStreamReader; -import java.nio.charset.Charset; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -42,7 +37,6 @@ import org.junit.AssumptionViolatedException; import org.junit.Test; -import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileSystem; @@ -295,28 +289,4 @@ public void testDestroyUnknownTable() throws Throwable { "-meta", "dynamodb://" + getTestTableName(DYNAMODB_TABLE)); } - @Test - public void testDumpTable() throws Throwable { - String target = System.getProperty("test.build.dir", "target"); - File buildDir = new File(target).getAbsoluteFile(); - String name = "dump-table"; - File destFile = new File(buildDir, name); - S3AFileSystem fs = getFileSystem(); - describe("Dumping metastore %s to %s", - fs.getMetadataStore(), - destFile); - DumpS3GuardDynamoTable.dumpStore( - fs, - null, - null, - destFile, - getFileSystem().getUri()); - File storeFile = new File(buildDir, name + "-store.csv"); - try (BufferedReader in = new BufferedReader(new InputStreamReader( - new FileInputStream(storeFile), Charset.forName("UTF-8")))) { - for (String line : IOUtils.readLines(in)) { - LOG.info(line); - } - } - } } From d6f1241ae3bf841fb5ec3c69d0bd405f9bb990b8 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 11 Jul 2019 17:58:18 +0100 Subject: [PATCH 23/23] HADOOP-16384 remove the instanceof check (for gabor); address indentation (for yetus) Change-Id: I3652f2da54e6543dd7e2fa556d50b6b570db5b42 --- .../s3a/s3guard/ITestDynamoDBMetadataStoreScale.java | 8 +++----- .../fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java | 10 +++++----- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java index 1c6110bed9606..a39afa4c60b53 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestDynamoDBMetadataStoreScale.java @@ -183,11 +183,9 @@ public void teardown() throws Exception { Iterable entries = tableAccess.scanMetadata(builder); List list = new ArrayList<>(); entries.iterator().forEachRemaining(e -> { - if (!(e instanceof S3GuardTableAccess.VersionMarker)) { - Path p = e.getFileStatus().getPath(); - LOG.info("Deleting {}", p); - list.add(p); - } + Path p = e.getFileStatus().getPath(); + LOG.info("Deleting {}", p); + list.add(p); }); tableAccess.delete(list); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java index e410abb08cb17..74c406978e3e9 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java @@ -203,11 +203,11 @@ public void test_400_rm_root_recursive() throws Throwable { // recursive treewalk to delete all files // does not delete directories. applyLocatedFiles(fs.listFilesAndEmptyDirectories(root, true), - f -> { - Path p = f.getPath(); - fs.delete(p, true); - assertPathDoesNotExist("expected file to be deleted", p); - }); + f -> { + Path p = f.getPath(); + fs.delete(p, true); + assertPathDoesNotExist("expected file to be deleted", p); + }); ContractTestUtils.deleteChildren(fs, root, true); // everything must be done by now StringBuffer sb = new StringBuffer();