diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java index 9d5f2bf4b6ed1..9fe402366c5df 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.util.Map; +import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -108,20 +109,55 @@ public static T awaitFuture(final Future future, */ public static T raiseInnerCause(final ExecutionException e) throws IOException { + throw unwrapInnerException(e); + } + + /** + * Extract the cause of a completion failure and rethrow it if an IOE + * or RTE. + * @param e exception. + * @param type of return value. + * @return nothing, ever. + * @throws IOException either the inner IOException, or a wrapper around + * any non-Runtime-Exception + * @throws RuntimeException if that is the inner cause. + */ + public static T raiseInnerCause(final CompletionException e) + throws IOException { + throw unwrapInnerException(e); + } + + /** + * From the inner cause of an execution exception, extract the inner cause. + * If it is an RTE: throw immediately. + * If it is an IOE: Return. + * If it is a WrappedIOException: Unwrap and return + * Else: create a new IOException. + * + * Recursively handles wrapped Execution and Completion Exceptions in + * case something very complicated has happened. + * @param e exception. + * @return an IOException extracted or built from the cause. + * @throws RuntimeException if that is the inner cause. + */ + private static IOException unwrapInnerException(final Throwable e) { Throwable cause = e.getCause(); if (cause instanceof IOException) { - throw (IOException) cause; + return (IOException) cause; } else if (cause instanceof WrappedIOException){ - throw ((WrappedIOException) cause).getCause(); + return ((WrappedIOException) cause).getCause(); + } else if (cause instanceof CompletionException){ + return unwrapInnerException(cause); + } else if (cause instanceof ExecutionException){ + return unwrapInnerException(cause); } else if (cause instanceof RuntimeException){ throw (RuntimeException) cause; } else if (cause != null) { // other type: wrap with a new IOE - throw new IOException(cause); + return new IOException(cause); } else { - // this only happens if somebody deliberately raises - // an ExecutionException - throw new IOException(e); + // this only happens if there was no cause. + return new IOException(e); } } diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index 933130125f54a..ba976036f0e1b 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -1213,8 +1213,12 @@ fs.s3a.connection.maximum - 15 - Controls the maximum number of simultaneous connections to S3. + 72 + Controls the maximum number of simultaneous connections to S3. + This must be bigger than the value of fs.s3a.threads.max so as to stop + threads being blocked waiting for new HTTPS connections. + Why not equal? The AWS SDK transfer manager also uses these connections. + @@ -1312,7 +1316,7 @@ fs.s3a.threads.max - 10 + 64 The total number of threads available in the filesystem for data uploads *or any other queued filesystem operation*. @@ -1326,8 +1330,25 @@ fs.s3a.max.total.tasks - 5 - The number of operations which can be queued for execution + 32 + The number of operations which can be queued for execution. + This is in addition to the number of active threads in fs.s3a.threads.max. + + + + + fs.s3a.executor.capacity + 16 + The maximum number of submitted tasks which is a single + operation (e.g. rename(), delete()) may submit simultaneously for + execution -excluding the IO-heavy block uploads, whose capacity + is set in "fs.s3a.fast.upload.active.blocks" + + All tasks are submitted to the shared thread pool whose size is + set in "fs.s3a.threads.max"; the value of capacity should be less than that + of the thread pool itself, as the goal is to stop a single operation + from overloading that thread pool. + diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java index 5b76a753de170..2751294beb92c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java @@ -268,7 +268,7 @@ public void testRenamePopulatesFileAncestors() throws IOException { * @param dst the destination root to move * @param nestedPath the nested path to move */ - private void validateAncestorsMoved(Path src, Path dst, String nestedPath) + protected void validateAncestorsMoved(Path src, Path dst, String nestedPath) throws IOException { assertIsDirectory(dst); assertPathDoesNotExist("src path should not exist", path(src + nestedPath)); diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 9419e48532eb1..7464ee36564b0 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -422,6 +422,11 @@ wildfly-openssl runtime + + org.assertj + assertj-core + test + junit junit diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index 7a687943cfb7b..92cefbbed410b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -139,9 +139,15 @@ private Constants() { public static final String ASSUMED_ROLE_CREDENTIALS_DEFAULT = SimpleAWSCredentialsProvider.NAME; + + // the maximum number of tasks cached if all threads are already uploading + public static final String MAX_TOTAL_TASKS = "fs.s3a.max.total.tasks"; + + public static final int DEFAULT_MAX_TOTAL_TASKS = 32; + // number of simultaneous connections to s3 public static final String MAXIMUM_CONNECTIONS = "fs.s3a.connection.maximum"; - public static final int DEFAULT_MAXIMUM_CONNECTIONS = 15; + public static final int DEFAULT_MAXIMUM_CONNECTIONS = 48; // connect to s3 over ssl? public static final String SECURE_CONNECTIONS = @@ -200,10 +206,6 @@ private Constants() { public static final String KEEPALIVE_TIME = "fs.s3a.threads.keepalivetime"; public static final int DEFAULT_KEEPALIVE_TIME = 60; - // the maximum number of tasks cached if all threads are already uploading - public static final String MAX_TOTAL_TASKS = "fs.s3a.max.total.tasks"; - public static final int DEFAULT_MAX_TOTAL_TASKS = 5; - // size of each of or multipart pieces in bytes public static final String MULTIPART_SIZE = "fs.s3a.multipart.size"; public static final long DEFAULT_MULTIPART_SIZE = 104857600; // 100 MB @@ -289,6 +291,22 @@ private Constants() { @InterfaceStability.Unstable public static final int DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS = 4; + /** + * The capacity of executor queues for operations other than block + * upload, where {@link #FAST_UPLOAD_ACTIVE_BLOCKS} is used instead. + * This should be less than {@link #MAX_THREADS} for fair + * submission. + * Value: {@value}. + */ + public static final String EXECUTOR_CAPACITY = "fs.s3a.executor.capacity"; + + /** + * The capacity of executor queues for operations other than block + * upload, where {@link #FAST_UPLOAD_ACTIVE_BLOCKS} is used instead. + * Value: {@value} + */ + public static final int DEFAULT_EXECUTOR_CAPACITY = 16; + // Private | PublicRead | PublicReadWrite | AuthenticatedRead | // LogDeliveryWrite | BucketOwnerRead | BucketOwnerFullControl public static final String CANNED_ACL = "fs.s3a.acl.default"; 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 7c82aa6b90d6e..733d76e61ffce 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 @@ -27,12 +27,14 @@ import java.nio.file.AccessDeniedException; import java.text.DateFormat; import java.text.SimpleDateFormat; +import java.time.Instant; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Date; import java.util.EnumSet; -import java.util.HashSet; import java.util.List; import java.util.Locale; import java.util.Map; @@ -45,6 +47,8 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiFunction; import javax.annotation.Nullable; import com.amazonaws.AmazonClientException; @@ -78,10 +82,13 @@ import com.amazonaws.event.ProgressListener; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; import com.google.common.util.concurrent.ListeningExecutorService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.lang3.tuple.Triple; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; @@ -89,7 +96,13 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; +import org.apache.hadoop.fs.s3a.impl.FunctionsRaisingIOE; +import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport; +import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; +import org.apache.hadoop.fs.s3a.s3guard.RenameTracker; import org.apache.hadoop.fs.s3a.select.InternalSelectConstants; +import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.LambdaUtils; import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileStatus; @@ -142,6 +155,8 @@ import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3Operations; import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.TokenIssuingPolicy.NoTokensAvailable; import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.hasDelegationTokenBinding; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; /** @@ -198,6 +213,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private TransferManager transfers; private ListeningExecutorService boundedThreadPool; private ExecutorService unboundedThreadPool; + private int executorCapacity; private long multiPartThreshold; public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class); private static final Logger PROGRESS = @@ -228,8 +244,33 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, /** Principal who created the FS; recorded during initialization. */ private UserGroupInformation owner; - // The maximum number of entries that can be deleted in any call to s3 + /** + * The maximum number of entries that can be deleted in any bulk delete + * call to S3 {@value}. + */ private static final int MAX_ENTRIES_TO_DELETE = 1000; + + /** + * This is an arbitrary value: {@value}. + * It declares how many parallel copy operations + * in a single rename can be queued before the operation pauses + * and awaits completion. + * A very large value wouldn't just starve other threads from + * performing work, there's a risk that the S3 store itself would + * throttle operations (which all go to the same shard). + * It is not currently configurable just to avoid people choosing values + * which work on a microbenchmark (single rename, no other work, ...) + * but don't scale well to execution in a large process against a common + * store, all while separate processes are working with the same shard + * of storage. + * + * It should be a factor of {@link #MAX_ENTRIES_TO_DELETE} so that + * all copies will have finished before deletion is contemplated. + * (There's always a block for that, it just makes more sense to + * perform the bulk delete after another block of copies have completed). + */ + public static final int RENAME_PARALLEL_LIMIT = 10; + private String blockOutputBuffer; private S3ADataBlocks.BlockFactory blockFactory; private int blockOutputActiveBlocks; @@ -242,6 +283,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private S3Guard.ITtlTimeProvider ttlTimeProvider; + /** Where is the bucket? Null if the caller could not determine this. */ + private String bucketLocation; + /** Add any deprecated keys. */ @SuppressWarnings("deprecation") private static void addDeprecatedKeys() { @@ -337,14 +381,15 @@ public void initialize(URI name, Configuration originalConf) maxThreads, maxThreads + totalTasks, keepAliveTime, TimeUnit.SECONDS, - "s3a-transfer-shared"); + "s3a-transfer-shared-" + bucket); unboundedThreadPool = new ThreadPoolExecutor( maxThreads, Integer.MAX_VALUE, keepAliveTime, TimeUnit.SECONDS, new LinkedBlockingQueue(), BlockingThreadPoolExecutorService.newDaemonThreadFactory( - "s3a-transfer-unbounded")); - + "s3a-transfer-unbounded-" + bucket)); + executorCapacity = intOption(conf, + EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1); int listVersion = conf.getInt(LIST_VERSION, DEFAULT_LIST_VERSION); if (listVersion < 1 || listVersion > 2) { LOG.warn("Configured fs.s3a.list.version {} is invalid, forcing " + @@ -358,6 +403,13 @@ public void initialize(URI name, Configuration originalConf) // the encryption algorithms) bindAWSClient(name, delegationTokensEnabled); + try { + bucketLocation = getBucketLocation(); + } catch (IOException e) { + LOG.warn("Location of bucket {} unknown: {} ", + getUri(), e.toString()); + LOG.debug("getBucketLocation() failed", e); + } initTransferManager(); initCannedAcls(conf); @@ -1088,9 +1140,13 @@ public FSDataOutputStream append(Path f, int bufferSize, * @throws IOException on IO failure * @return true if rename is successful */ + @Retries.RetryTranslated public boolean rename(Path src, Path dst) throws IOException { - try { - return innerRename(src, dst); + try(DurationInfo ignored = new DurationInfo(LOG, false, + "rename(%s, %s", src, dst)) { + long bytesCopied = innerRename(src, dst); + LOG.debug("Copied {} bytes", bytesCopied); + return true; } catch (AmazonClientException e) { throw translateException("rename(" + src +", " + dst + ")", src, e); } catch (RenameFailedException e) { @@ -1116,12 +1172,13 @@ public boolean rename(Path src, Path dst) throws IOException { * rename was not met. This means work didn't happen; it's not something * which is reported upstream to the FileSystem APIs, for which the semantics * of "false" are pretty vague. + * @return the number of bytes copied. * @throws FileNotFoundException there's no source file. * @throws IOException on IO failure. * @throws AmazonClientException on failures inside the AWS SDK */ @Retries.RetryMixed - private boolean innerRename(Path source, Path dest) + private long innerRename(Path source, Path dest) throws RenameFailedException, FileNotFoundException, IOException, AmazonClientException { Path src = qualify(source); @@ -1199,115 +1256,256 @@ private boolean innerRename(Path source, Path dest) } } - // If we have a MetadataStore, track deletions/creations. - Collection srcPaths = null; - List dstMetas = null; - if (hasMetadataStore()) { - srcPaths = new HashSet<>(); // srcPaths need fast look up before put - dstMetas = new ArrayList<>(); - } - // TODO S3Guard HADOOP-13761: retries when source paths are not visible yet - // TODO S3Guard: performance: mark destination dirs as authoritative - - // Ok! Time to start - if (srcStatus.isFile()) { - LOG.debug("rename: renaming file {} to {}", src, dst); - long length = srcStatus.getLen(); - if (dstStatus != null && dstStatus.isDirectory()) { - String newDstKey = maybeAddTrailingSlash(dstKey); - String filename = - srcKey.substring(pathToKey(src.getParent()).length()+1); - newDstKey = newDstKey + filename; - copyFile(srcKey, newDstKey, length); - S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src, - keyToQualifiedPath(newDstKey), length, getDefaultBlockSize(dst), - username); - } else { - copyFile(srcKey, dstKey, srcStatus.getLen()); - S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src, dst, - length, getDefaultBlockSize(dst), username); - } - innerDelete(srcStatus, false); - } else { - LOG.debug("rename: renaming directory {} to {}", src, dst); + // Validation completed: time to begin the operation. + // The store-specific rename operation is used to keep the store + // to date with the in-progress operation. + // for the null store, these are all no-ops. + final RenameTracker renameTracker = + metadataStore.initiateRenameOperation( + createStoreContext(), + src, srcStatus, dest); + final AtomicLong bytesCopied = new AtomicLong(); + int renameParallelLimit = RENAME_PARALLEL_LIMIT; + final List> activeCopies = + new ArrayList<>(renameParallelLimit); + // aggregate operation to wait for the copies to complete then reset + // the list. + final FunctionsRaisingIOE.FunctionRaisingIOE + completeActiveCopies = (String reason) -> { + LOG.debug("Waiting for {} active copies to complete: {}", + activeCopies.size(), reason); + waitForCompletion(activeCopies); + activeCopies.clear(); + return null; + }; - // This is a directory to directory copy - dstKey = maybeAddTrailingSlash(dstKey); - srcKey = maybeAddTrailingSlash(srcKey); + try { + if (srcStatus.isFile()) { + // the source is a file. + Path copyDestinationPath = dst; + String copyDestinationKey = dstKey; + if (dstStatus != null && dstStatus.isDirectory()) { + // destination is a directory: build the final destination underneath + String newDstKey = maybeAddTrailingSlash(dstKey); + String filename = + srcKey.substring(pathToKey(src.getParent()).length()+1); + newDstKey = newDstKey + filename; + copyDestinationKey = newDstKey; + copyDestinationPath = keyToQualifiedPath(newDstKey); + } + // destination either does not exist or is a file to overwrite. + LOG.debug("rename: renaming file {} to {}", src, copyDestinationPath); + copySourceAndUpdateTracker(renameTracker, + src, + srcKey, + srcStatus, + copyDestinationPath, + copyDestinationKey, + false); + bytesCopied.addAndGet(srcStatus.getLen()); + // delete the source + deleteObjectAtPath(src, srcKey, true); + // and update the tracker + renameTracker.sourceObjectsDeleted(Lists.newArrayList(src)); + } else { + LOG.debug("rename: renaming directory {} to {}", src, dst); - //Verify dest is not a child of the source directory - if (dstKey.startsWith(srcKey)) { - throw new RenameFailedException(srcKey, dstKey, - "cannot rename a directory to a subdirectory of itself "); - } + // This is a directory-to-directory copy + dstKey = maybeAddTrailingSlash(dstKey); + srcKey = maybeAddTrailingSlash(srcKey); - List keysToDelete = new ArrayList<>(); - if (dstStatus != null && dstStatus.isEmptyDirectory() == Tristate.TRUE) { - // delete unnecessary fake directory. - keysToDelete.add(new DeleteObjectsRequest.KeyVersion(dstKey)); - } + //Verify dest is not a child of the source directory + if (dstKey.startsWith(srcKey)) { + throw new RenameFailedException(srcKey, dstKey, + "cannot rename a directory to a subdirectory of itself "); + } - Path parentPath = keyToQualifiedPath(srcKey); - RemoteIterator iterator = listFilesAndEmptyDirectories( - parentPath, true); - while (iterator.hasNext()) { - LocatedFileStatus status = iterator.next(); - long length = status.getLen(); - String key = pathToKey(status.getPath()); - if (status.isDirectory() && !key.endsWith("/")) { - key += "/"; + // These are the lists of keys to delete and of their paths, the + // latter being used to update the rename tracker. + final List keysToDelete = + new ArrayList<>(); + final List pathsToDelete = new ArrayList<>(); + // to update the lists of keys and paths. + final BiFunction queueToDelete = + (Path path, String key) -> { + pathsToDelete.add(path); + keysToDelete.add(new DeleteObjectsRequest.KeyVersion(key)); + return null; + }; + // a lambda-expression to block waiting for ay active copies to finish + // then delete all queued keys + paths to delete. + final FunctionsRaisingIOE.FunctionRaisingIOE + completeActiveCopiesAndDeleteSources = + (reason) -> { + completeActiveCopies.apply(reason); + removeSourceObjects(renameTracker, keysToDelete, pathsToDelete); + // now reset the lists. + keysToDelete.clear(); + pathsToDelete.clear(); + return null; + }; + + if (dstStatus != null && dstStatus.isEmptyDirectory() == Tristate.TRUE) { + // delete unnecessary fake directory at the destination. + // this MUST be done before anything else so that + // rollback code doesn't get confused and insert a tombstone + // marker. + deleteObjectAtPath(dstStatus.getPath(), dstKey, false); } - keysToDelete - .add(new DeleteObjectsRequest.KeyVersion(key)); - String newDstKey = - dstKey + key.substring(srcKey.length()); - copyFile(key, newDstKey, length); - - if (hasMetadataStore()) { - // with a metadata store, the object entries need to be updated, - // including, potentially, the ancestors - Path childSrc = keyToQualifiedPath(key); - Path childDst = keyToQualifiedPath(newDstKey); - if (objectRepresentsDirectory(key, length)) { - S3Guard.addMoveDir(metadataStore, srcPaths, dstMetas, childSrc, - childDst, username); - } else { - S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, childSrc, - childDst, length, getDefaultBlockSize(childDst), username); + + Path parentPath = keyToQualifiedPath(srcKey); + RemoteIterator iterator = listFilesAndEmptyDirectories( + parentPath, true); + while (iterator.hasNext()) { + LocatedFileStatus status = iterator.next(); + String k = pathToKey(status.getPath()); + String key = (status.isDirectory() && !k.endsWith("/")) + ? k + "/" + : k; + String newDstKey = + dstKey + key.substring(srcKey.length()); + Path childSourcePath = keyToQualifiedPath(key); + + queueToDelete.apply(childSourcePath, key); + + Path childDestPath = keyToQualifiedPath(newDstKey); + + // queue this copy for execution. + CompletableFuture copy = submit(boundedThreadPool, () -> + copySourceAndUpdateTracker(renameTracker, + childSourcePath, + key, + status, + childDestPath, + newDstKey, + true)); + bytesCopied.addAndGet(srcStatus.getLen()); + activeCopies.add(copy); + if (activeCopies.size() == renameParallelLimit) { + // the limit of active copies has been reached; + // wait for completion or errors to surface. + LOG.debug("Waiting for active copies to complete"); + completeActiveCopies.apply("batch threshold reached"); + } + if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) { + // finish ongoing copies then delete all queued keys. + // provided the parallel limit is a factor of the max entry + // constant, this will not need to block for the copy, and + // simply jump straight to the delete. + completeActiveCopiesAndDeleteSources.apply("paged delete"); } - // Ancestor directories may not be listed, so we explicitly add them - S3Guard.addMoveAncestors(metadataStore, srcPaths, dstMetas, - keyToQualifiedPath(srcKey), childSrc, childDst, username); } + // end of iteration - if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) { - removeKeys(keysToDelete, true, false); - } + // await the final set of copies and then delete + // This will notify the renameTracker that these objects + // have been deleted. + completeActiveCopiesAndDeleteSources.apply("final copy and delete"); + + // We moved all the children, now move the top-level dir + // Empty directory should have been added as the object summary + renameTracker.moveSourceDirectory(); } - if (!keysToDelete.isEmpty()) { - removeKeys(keysToDelete, false, false); + } catch (AmazonClientException | IOException ex) { + // rename failed. + // block for all ongoing copies to complete, successfully or not + try { + completeActiveCopies.apply("failure handling"); + } catch (IOException e) { + // a failure to update the metastore after a rename failure is what + // we'd see on a network problem, expired credentials and other + // unrecoverable errors. + // Downgrading to warn because an exception is already + // about to be thrown. + LOG.warn("While completing all active copies", e); } - // We moved all the children, now move the top-level dir - // Empty directory should have been added as the object summary - if (hasMetadataStore() - && srcPaths != null - && !srcPaths.contains(src)) { - LOG.debug("To move the non-empty top-level dir src={} and dst={}", - src, dst); - S3Guard.addMoveDir(metadataStore, srcPaths, dstMetas, src, dst, - username); - } + // update the store state to reflect this + throw renameTracker.renameFailed(ex); } - metadataStore.move(srcPaths, dstMetas); + // At this point the rename has completed in the S3 store. + // Tell the metastore this fact and let it complete its changes + renameTracker.completeRename(); if (!src.getParent().equals(dst.getParent())) { LOG.debug("source & dest parents are different; fix up dir markers"); deleteUnnecessaryFakeDirectories(dst.getParent()); maybeCreateFakeParentDirectory(src); } - return true; + return bytesCopied.get(); + } + + /** + * Remove source objects and update the metastore by way of + * the rename tracker. + * @param renameTracker rename tracker to update. + * @param keysToDelete list of keys to delete + * @param pathsToDelete list of paths matching the keys to delete 1:1. + * @throws IOException failure + */ + @Retries.RetryMixed + private void removeSourceObjects( + final RenameTracker renameTracker, + final List keysToDelete, + final List pathsToDelete) + throws IOException { + List undeletedObjects = new ArrayList<>(); + try { + // remove the keys + // this does will update the metastore on a failure, but on + // a successful operation leaves the store as is. + removeKeys(keysToDelete, false, undeletedObjects); + // and clear the list. + } catch (AmazonClientException | IOException e) { + // failed, notify the rename operation. + // removeKeys will have already purged the metastore of + // all keys it has known to delete; this is just a final + // bit of housekeeping and a chance to tune exception + // reporting + throw renameTracker.deleteFailed(e, pathsToDelete, undeletedObjects); + } + renameTracker.sourceObjectsDeleted(pathsToDelete); + } + + /** + * This invoked to copy a file or directory marker then update the + * rename operation on success. + * It may be called in its own thread. + * @param renameTracker operation to update + * @param sourcePath source path of the copy; may have a trailing / on it. + * @param srcKey source key + * @param sourceStatus status of the source object + * @param destPath destination as a qualified path. + * @param destKey destination key + * @param addAncestors should ancestors be added to the metastore? + * @return the destination path. + * @throws IOException failure + */ + @Retries.RetryTranslated + private Path copySourceAndUpdateTracker( + final RenameTracker renameTracker, + final Path sourcePath, + final String srcKey, + final FileStatus sourceStatus, + final Path destPath, + final String destKey, + final boolean addAncestors) throws IOException { + copyFile(srcKey, destKey, sourceStatus, sourceStatus.getLen()); + if (objectRepresentsDirectory(srcKey, sourceStatus.getLen())) { + renameTracker.directoryMarkerCopied( + sourceStatus, + destPath, + addAncestors); + } else { + renameTracker.fileCopied( + sourcePath, + sourceStatus, + destPath, + getDefaultBlockSize(destPath), + addAncestors); + } + return destPath; } /** @@ -1520,16 +1718,19 @@ protected S3ListResult listObjects(S3ListRequest request) throws IOException { incrementReadOperations(); incrementStatistic(OBJECT_LIST_REQUESTS); validateListArguments(request); - return invoker.retryUntranslated( - request.toString(), - true, - () -> { - if (useListV1) { - return S3ListResult.v1(s3.listObjects(request.getV1())); - } else { - return S3ListResult.v2(s3.listObjectsV2(request.getV2())); - } - }); + try(DurationInfo ignored = + new DurationInfo(LOG, false, "LIST")) { + return invoker.retryUntranslated( + request.toString(), + true, + () -> { + if (useListV1) { + return S3ListResult.v1(s3.listObjects(request.getV1())); + } else { + return S3ListResult.v2(s3.listObjectsV2(request.getV2())); + } + }); + } } /** @@ -1557,20 +1758,24 @@ protected S3ListResult continueListObjects(S3ListRequest request, S3ListResult prevResult) throws IOException { incrementReadOperations(); validateListArguments(request); - return invoker.retryUntranslated( - request.toString(), - true, - () -> { - incrementStatistic(OBJECT_CONTINUE_LIST_REQUESTS); - if (useListV1) { - return S3ListResult.v1( - s3.listNextBatchOfObjects(prevResult.getV1())); - } else { - request.getV2().setContinuationToken(prevResult.getV2() - .getNextContinuationToken()); - return S3ListResult.v2(s3.listObjectsV2(request.getV2())); - } - }); + + try(DurationInfo ignored = + new DurationInfo(LOG, false, "LIST (continued)")) { + return invoker.retryUntranslated( + request.toString(), + true, + () -> { + incrementStatistic(OBJECT_CONTINUE_LIST_REQUESTS); + if (useListV1) { + return S3ListResult.v1( + s3.listNextBatchOfObjects(prevResult.getV1())); + } else { + request.getV2().setContinuationToken(prevResult.getV2() + .getNextContinuationToken()); + return S3ListResult.v2(s3.listObjectsV2(request.getV2())); + } + }); + } } /** @@ -1608,6 +1813,7 @@ protected void deleteObject(String key) throws AmazonClientException, IOException { blockRootDelete(key); incrementWriteOperations(); + LOG.debug("DELETE {}", key); invoker.retryUntranslated("Delete "+ bucket + ":/" + key, DELETE_CONSIDERED_IDEMPOTENT, ()-> { @@ -1625,9 +1831,9 @@ protected void deleteObject(String key) * @param key key of entry * @param isFile is the path a file (used for instrumentation only) * @throws AmazonClientException problems working with S3 - * @throws IOException IO failure + * @throws IOException IO failure in the metastore */ - @Retries.RetryRaw + @Retries.RetryMixed void deleteObjectAtPath(Path f, String key, boolean isFile) throws AmazonClientException, IOException { if (isFile) { @@ -1666,7 +1872,8 @@ private void blockRootDelete(String key) throws InvalidRequestException { private void deleteObjects(DeleteObjectsRequest deleteRequest) throws MultiObjectDeleteException, AmazonClientException, IOException { incrementWriteOperations(); - try { + try(DurationInfo ignored = new DurationInfo(LOG, + false, "DELETE %d keys", deleteRequest.getKeys().size())) { invoker.retryUntranslated("delete", DELETE_CONSIDERED_IDEMPOTENT, () -> { @@ -1802,7 +2009,7 @@ PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest) PutObjectResult result = s3.putObject(putObjectRequest); incrementPutCompletedStatistics(true, len); // update metadata - finishedWrite(putObjectRequest.getKey(), len); + finishedWrite(putObjectRequest.getKey(), len, null); return result; } catch (AmazonClientException e) { incrementPutCompletedStatistics(false, len); @@ -1903,23 +2110,23 @@ public void incrementPutProgressStatistics(String key, long bytes) { } /** - * A helper method to delete a list of keys on a s3-backend. + * Delete a list of keys on a s3-backend. + * This does not update the metastore. * Retry policy: retry untranslated; delete considered idempotent. * @param keysToDelete collection of keys to delete on the s3-backend. * if empty, no request is made of the object store. - * @param clearKeys clears the keysToDelete-list after processing the list - * when set to true * @param deleteFakeDir indicates whether this is for deleting fake dirs * @throws InvalidRequestException if the request was rejected due to * a mistaken attempt to delete the root directory. * @throws MultiObjectDeleteException one or more of the keys could not * be deleted in a multiple object delete operation. - * @throws AmazonClientException amazon-layer failure. + * The number of rejected objects will be added to the metric + * {@link Statistic#FILES_DELETE_REJECTED}. + * @throws AmazonClientException other amazon-layer failure. */ - @VisibleForTesting @Retries.RetryRaw - void removeKeys(List keysToDelete, - boolean clearKeys, boolean deleteFakeDir) + private void removeKeysS3(List keysToDelete, + boolean deleteFakeDir) throws MultiObjectDeleteException, AmazonClientException, IOException { if (keysToDelete.isEmpty()) { @@ -1929,22 +2136,112 @@ void removeKeys(List keysToDelete, for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) { blockRootDelete(keyVersion.getKey()); } - if (enableMultiObjectsDelete) { - deleteObjects(new DeleteObjectsRequest(bucket) - .withKeys(keysToDelete) - .withQuiet(true)); - } else { - for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) { - deleteObject(keyVersion.getKey()); + try { + if (enableMultiObjectsDelete) { + deleteObjects(new DeleteObjectsRequest(bucket) + .withKeys(keysToDelete) + .withQuiet(true)); + } else { + for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) { + deleteObject(keyVersion.getKey()); + } } + } catch (MultiObjectDeleteException ex) { + // partial delete. + // Update the stats with the count of the actual number of successful + // deletions. + int rejected = ex.getErrors().size(); + noteDeleted(keysToDelete.size() - rejected, deleteFakeDir); + incrementStatistic(FILES_DELETE_REJECTED, rejected); + throw ex; } + noteDeleted(keysToDelete.size(), deleteFakeDir); + } + + /** + * Note the deletion of files or fake directories deleted. + * @param count count of keys deleted. + * @param deleteFakeDir are the deletions fake directories? + */ + private void noteDeleted(final int count, final boolean deleteFakeDir) { if (!deleteFakeDir) { - instrumentation.fileDeleted(keysToDelete.size()); + instrumentation.fileDeleted(count); } else { - instrumentation.fakeDirsDeleted(keysToDelete.size()); + instrumentation.fakeDirsDeleted(count); } - if (clearKeys) { - keysToDelete.clear(); + } + + /** + * Invoke {@link #removeKeysS3(List, boolean)} with handling of + * {@code MultiObjectDeleteException} in which S3Guard is updated with all + * deleted entries, before the exception is rethrown. + * + * If an exception is not raised. the metastore is not updated. + * @param keysToDelete collection of keys to delete on the s3-backend. + * if empty, no request is made of the object store. + * @param deleteFakeDir indicates whether this is for deleting fake dirs + * @throws InvalidRequestException if the request was rejected due to + * a mistaken attempt to delete the root directory. + * @throws MultiObjectDeleteException one or more of the keys could not + * be deleted in a multiple object delete operation. + * @throws AmazonClientException amazon-layer failure. + * @throws IOException other IO Exception. + */ + @VisibleForTesting + @Retries.RetryMixed + void removeKeys( + final List keysToDelete, + final boolean deleteFakeDir) + throws MultiObjectDeleteException, AmazonClientException, + IOException { + removeKeys(keysToDelete, deleteFakeDir, new ArrayList<>()); + } + + /** + * Invoke {@link #removeKeysS3(List, boolean)} with handling of + * {@code MultiObjectDeleteException} in which S3Guard is updated with all + * deleted entries, before the exception is rethrown. + * + * @param keysToDelete collection of keys to delete on the s3-backend. + * if empty, no request is made of the object store. + * @param deleteFakeDir indicates whether this is for deleting fake dirs + * @param undeletedObjectsOnFailure List which will be built up of all + * files that were not deleted. This happens even as an exception + * is raised. + * @throws InvalidRequestException if the request was rejected due to + * a mistaken attempt to delete the root directory. + * @throws MultiObjectDeleteException one or more of the keys could not + * be deleted in a multiple object delete operation. + * @throws AmazonClientException amazon-layer failure. + * @throws IOException other IO Exception. + */ + @VisibleForTesting + @Retries.RetryMixed + void removeKeys( + final List keysToDelete, + final boolean deleteFakeDir, + final List undeletedObjectsOnFailure) + throws MultiObjectDeleteException, AmazonClientException, + IOException { + undeletedObjectsOnFailure.clear(); + try(DurationInfo ignored = new DurationInfo(LOG, false, "Deleting")) { + removeKeysS3(keysToDelete, deleteFakeDir); + } catch (MultiObjectDeleteException ex) { + LOG.debug("Partial delete failure"); + // what to do if an IOE was raised? Given an exception was being + // raised anyway, and the failures are logged, do nothing. + Triple, List, List>> results = + new MultiObjectDeleteSupport(createStoreContext()) + .processDeleteFailure(ex, keysToDelete); + undeletedObjectsOnFailure.addAll(results.getMiddle()); + throw ex; + } catch (AmazonClientException | IOException ex) { + List paths = new MultiObjectDeleteSupport( + createStoreContext()) + .processDeleteFailureGenericException(ex, keysToDelete); + // other failures. Assume nothing was deleted + undeletedObjectsOnFailure.addAll(paths); + throw ex; } } @@ -2041,22 +2338,25 @@ private boolean innerDelete(S3AFileStatus status, boolean recursive) LOG.debug("Got object to delete {}", summary.getKey()); if (keys.size() == MAX_ENTRIES_TO_DELETE) { - removeKeys(keys, true, false); + // delete a single page of keys + removeKeys(keys, false); + keys.clear(); } } if (objects.isTruncated()) { objects = continueListObjects(request, objects); } else { - if (!keys.isEmpty()) { - // TODO: HADOOP-13761 S3Guard: retries - removeKeys(keys, false, false); - } + // there is no more data: delete the final set of entries. + removeKeys(keys, false); break; } } } - metadataStore.deleteSubtree(f); + try(DurationInfo ignored = + new DurationInfo(LOG, false, "Delete metastore")) { + metadataStore.deleteSubtree(f); + } } else { LOG.debug("delete: Path is a file"); deleteObjectAtPath(f, key, true); @@ -2272,6 +2572,37 @@ public UserGroupInformation getOwner() { return owner; } + /** + * Build an immutable store context. + * If called while the FS is being initialized, + * some of the context will be incomplete. + * new store context instances should be created as appropriate. + * @return the store context of this FS. + */ + @InterfaceAudience.Private + public StoreContext createStoreContext() { + return new StoreContext( + getUri(), + getBucket(), + getConf(), + getUsername(), + owner, + boundedThreadPool, + executorCapacity, + invoker, + getInstrumentation(), + getStorageStatistics(), + getInputPolicy(), + changeDetectionPolicy, + enableMultiObjectsDelete, + metadataStore, + this::keyToQualifiedPath, + bucketLocation, + useListV1, + false, + (prefix, len) -> createTmpFileForWrite(prefix, len, getConf())); + } + /** * * Make the given path and all non-existent parents into @@ -2383,8 +2714,11 @@ S3AFileStatus innerGetFileStatus(final Path f, Set tombstones = Collections.emptySet(); if (pm != null) { if (pm.isDeleted()) { + OffsetDateTime deletedAt = OffsetDateTime.ofInstant( + Instant.ofEpochMilli(pm.getFileStatus().getModificationTime()), + ZoneOffset.UTC); throw new FileNotFoundException("Path " + f + " is recorded as " + - "deleted by S3Guard"); + "deleted by S3Guard at " + deletedAt.toString()); } // if ms is not authoritative, check S3 if there's any recent @@ -2734,7 +3068,7 @@ UploadResult executePut(PutObjectRequest putObjectRequest, UploadResult result = waitForUploadCompletion(key, info); listener.uploadCompleted(); // post-write actions - finishedWrite(key, info.getLength()); + finishedWrite(key, info.getLength(), null); return result; } @@ -2898,13 +3232,16 @@ public List listAWSPolicyRules( * Callers must implement. * @param srcKey source object path * @param dstKey destination object path + * @param sourceStatus * @param size object size - * @throws AmazonClientException on failures inside the AWS SDK * @throws InterruptedIOException the operation was interrupted * @throws IOException Other IO problems */ - @Retries.RetryMixed - private void copyFile(String srcKey, String dstKey, long size) + @Retries.OnceTranslated + private void copyFile(String srcKey, + String dstKey, + final FileStatus sourceStatus, + long size) throws IOException, InterruptedIOException { LOG.debug("copyFile {} -> {} ", srcKey, dstKey); @@ -2918,9 +3255,16 @@ private void copyFile(String srcKey, String dstKey, long size) } }; + // get the object header, handling the possibility that a + // newly created file is not yet present. + final Invoker inv = s3guardInvoker != null ? s3guardInvoker : invoker; + final ObjectMetadata srcom = inv.retry("HEAD", srcKey, true, + () -> getObjectMetadata(srcKey)); + + // there is no retry logic here on the expectation that the transfer + // manager is doing the work once("copyFile(" + srcKey + ", " + dstKey + ")", srcKey, () -> { - ObjectMetadata srcom = getObjectMetadata(srcKey); ObjectMetadata dstom = cloneObjectMetadata(srcom); setOptionalObjectMetadata(dstom); CopyObjectRequest copyObjectRequest = @@ -2928,6 +3272,8 @@ private void copyFile(String srcKey, String dstKey, long size) setOptionalCopyObjectRequestParameters(copyObjectRequest); copyObjectRequest.setCannedAccessControlList(cannedACL); copyObjectRequest.setNewObjectMetadata(dstom); + Optional.ofNullable(srcom.getStorageClass()) + .ifPresent(copyObjectRequest::setStorageClass); Copy copy = transfers.copy(copyObjectRequest); copy.addProgressListener(progressListener); try { @@ -3032,7 +3378,7 @@ private Optional generateSSECustomerKey() { /** * Perform post-write actions. * Calls {@link #deleteUnnecessaryFakeDirectories(Path)} and then - * {@link S3Guard#addAncestors(MetadataStore, Path, String)}}. + * {@link S3Guard#addAncestors(MetadataStore, Path, String, BulkOperationState)}}. * This operation MUST be called after any PUT/multipart PUT completes * successfully. * @@ -3044,6 +3390,7 @@ private Optional generateSSECustomerKey() { * * @param key key written to * @param length total length of file written + * @param operationState state of any ongoing bulk operation. * @throws MetadataPersistenceException if metadata about the write could * not be saved to the metadata store and * fs.s3a.metadatastore.fail.on.write.error=true @@ -3051,7 +3398,9 @@ private Optional generateSSECustomerKey() { @InterfaceAudience.Private @Retries.RetryTranslated("Except if failOnMetadataWriteError=false, in which" + " case RetryExceptionsSwallowed") - void finishedWrite(String key, long length) + void finishedWrite(String key, + long length, + @Nullable final BulkOperationState operationState) throws MetadataPersistenceException { LOG.debug("Finished write to {}, len {}", key, length); Path p = keyToQualifiedPath(key); @@ -3061,11 +3410,12 @@ void finishedWrite(String key, long length) // See note about failure semantics in S3Guard documentation try { if (hasMetadataStore()) { - S3Guard.addAncestors(metadataStore, p, username); + S3Guard.addAncestors(metadataStore, p, username, operationState); S3AFileStatus status = createUploadFileStatus(p, S3AUtils.objectRepresentsDirectory(key, length), length, getDefaultBlockSize(p), username); - S3Guard.putAndReturn(metadataStore, status, instrumentation); + S3Guard.putAndReturn(metadataStore, status, instrumentation, + operationState); } } catch (IOException e) { if (failOnMetadataWriteError) { @@ -3094,7 +3444,7 @@ private void deleteUnnecessaryFakeDirectories(Path path) { path = path.getParent(); } try { - removeKeys(keysToRemove, false, true); + removeKeys(keysToRemove, true); } catch(AmazonClientException | IOException e) { instrumentation.errorIgnored(); if (LOG.isDebugEnabled()) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java index 2bb8f682d8d42..14c602f7c5033 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java @@ -183,10 +183,13 @@ public class S3AInstrumentation implements Closeable, MetricsSource { COMMITTER_MAGIC_FILES_CREATED, S3GUARD_METADATASTORE_PUT_PATH_REQUEST, S3GUARD_METADATASTORE_INITIALIZATION, + S3GUARD_METADATASTORE_RECORD_READS, + S3GUARD_METADATASTORE_RECORD_WRITES, S3GUARD_METADATASTORE_RETRY, S3GUARD_METADATASTORE_THROTTLED, STORE_IO_THROTTLED, - DELEGATION_TOKENS_ISSUED + DELEGATION_TOKENS_ISSUED, + FILES_DELETE_REJECTED }; private static final Statistic[] GAUGES_TO_CREATE = { @@ -1061,6 +1064,23 @@ public void throttled() { public void retrying() { // counters are incremented by owner. } + + /** + * Records have been read. + * @param count the number of records read + */ + public void recordsRead(int count) { + incrementCounter(S3GUARD_METADATASTORE_RECORD_READS, count); + } + + /** + * records have been written (including deleted). + * @param count number of records written. + */ + public void recordsWritten(int count) { + incrementCounter(S3GUARD_METADATASTORE_RECORD_WRITES, count); + } + } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java index 85181c3af8904..afc3431758413 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java @@ -87,6 +87,7 @@ import static org.apache.commons.lang3.StringUtils.isEmpty; import static org.apache.hadoop.fs.s3a.Constants.*; +import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.translateDeleteException; /** * Utility methods for S3A code. @@ -287,7 +288,7 @@ public static IOException translateException(@Nullable String operation, case 200: if (exception instanceof MultiObjectDeleteException) { // failure during a bulk delete - return translateMultiObjectDeleteException(message, + return translateDeleteException(message, (MultiObjectDeleteException) exception); } // other 200: FALL THROUGH @@ -451,40 +452,6 @@ public static IOException translateDynamoDBException(final String path, return result; } - /** - * A MultiObjectDeleteException is raised if one or more delete objects - * listed in a bulk DELETE operation failed. - * The top-level exception is therefore just "something wasn't deleted", - * but doesn't include the what or the why. - * This translation will extract an AccessDeniedException if that's one of - * the causes, otherwise grabs the status code and uses it in the - * returned exception. - * @param message text for the exception - * @param ex exception to translate - * @return an IOE with more detail. - */ - public static IOException translateMultiObjectDeleteException(String message, - MultiObjectDeleteException ex) { - List keys; - StringBuffer result = new StringBuffer(ex.getErrors().size() * 100); - result.append(message).append(": "); - String exitCode = ""; - for (MultiObjectDeleteException.DeleteError error : ex.getErrors()) { - String code = error.getCode(); - result.append(String.format("%s: %s: %s%n", code, error.getKey(), - error.getMessage())); - if (exitCode.isEmpty() || "AccessDenied".equals(code)) { - exitCode = code; - } - } - if ("AccessDenied".equals(exitCode)) { - return (IOException) new AccessDeniedException(result.toString()) - .initCause(ex); - } else { - return new AWSS3IOException(result.toString(), ex); - } - } - /** * Get low level details of an amazon exception for logging; multi-line. * @param e exception @@ -998,7 +965,7 @@ public static String stringify(S3ObjectSummary summary) { * @return the value * @throws IllegalArgumentException if the value is below the minimum */ - static int intOption(Configuration conf, String key, int defVal, int min) { + public static int intOption(Configuration conf, String key, int defVal, int min) { int v = conf.getInt(key, defVal); Preconditions.checkArgument(v >= min, String.format("Value of %s: %d is below the minimum value %d", @@ -1016,7 +983,7 @@ static int intOption(Configuration conf, String key, int defVal, int min) { * @return the value * @throws IllegalArgumentException if the value is below the minimum */ - static long longOption(Configuration conf, + public static long longOption(Configuration conf, String key, long defVal, long min) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java index 54a2c60254167..71f9ddb98114a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java @@ -42,6 +42,8 @@ public enum Statistic { "Total number of files created through the object store."), FILES_DELETED("files_deleted", "Total number of files deleted from the object store."), + FILES_DELETE_REJECTED("files_delete_rejected", + "Total number of files whose delete request was rejected"), FAKE_DIRECTORIES_CREATED("fake_directories_created", "Total number of fake directory entries created in the object store."), FAKE_DIRECTORIES_DELETED("fake_directories_deleted", @@ -207,6 +209,12 @@ public enum Statistic { "S3Guard metadata store put one metadata path latency"), S3GUARD_METADATASTORE_INITIALIZATION("s3guard_metadatastore_initialization", "S3Guard metadata store initialization times"), + S3GUARD_METADATASTORE_RECORD_READS( + "s3guard_metadatastore_record_reads", + "S3Guard metadata store records read"), + S3GUARD_METADATASTORE_RECORD_WRITES( + "s3guard_metadatastore_record_writes", + "S3Guard metadata store records written"), S3GUARD_METADATASTORE_RETRY("s3guard_metadatastore_retry", "S3Guard metadata store retry events"), S3GUARD_METADATASTORE_THROTTLED("s3guard_metadatastore_throttled", 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 ea091720c2705..7a45a883f7251 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a; +import javax.annotation.Nullable; import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -41,13 +42,15 @@ import com.amazonaws.services.s3.model.UploadPartResult; import com.amazonaws.services.s3.transfer.model.UploadResult; import com.google.common.base.Preconditions; -import org.apache.hadoop.conf.Configuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; +import org.apache.hadoop.fs.s3a.s3guard.S3Guard; import org.apache.hadoop.fs.s3a.select.SelectBinding; import org.apache.hadoop.util.DurationInfo; @@ -226,13 +229,14 @@ public String initiateMultiPartUpload(String destKey) throws IOException { /** * Finalize a multipart PUT operation. * This completes the upload, and, if that works, calls - * {@link S3AFileSystem#finishedWrite(String, long)} to update the filesystem. + * {@link S3AFileSystem#finishedWrite(String, long, BulkOperationState)} to update the filesystem. * Retry policy: retrying, translated. * @param destKey destination of the commit * @param uploadId multipart operation Id * @param partETags list of partial uploads * @param length length of the upload * @param retrying retrying callback + * @param operationState (nullable) operational state for a bulk update * @return the result of the operation. * @throws IOException on problems. */ @@ -242,7 +246,8 @@ private CompleteMultipartUploadResult finalizeMultipartUpload( String uploadId, List partETags, long length, - Retried retrying) throws IOException { + Retried retrying, + @Nullable BulkOperationState operationState) throws IOException { if (partETags.isEmpty()) { throw new IOException( "No upload parts in multipart upload to " + destKey); @@ -260,7 +265,7 @@ private CompleteMultipartUploadResult finalizeMultipartUpload( new ArrayList<>(partETags))); } ); - owner.finishedWrite(destKey, length); + owner.finishedWrite(destKey, length, operationState); return uploadResult; } @@ -295,7 +300,8 @@ public CompleteMultipartUploadResult completeMPUwithRetries( uploadId, partETags, length, - (text, e, r, i) -> errorCount.incrementAndGet()); + (text, e, r, i) -> errorCount.incrementAndGet(), + null); } /** @@ -486,6 +492,51 @@ public void revertCommit(String destKey) throws IOException { ); } + /** + * This completes a multipart upload to the destination key via + * {@code finalizeMultipartUpload()}. + * Retry policy: retrying, translated. + * Retries increment the {@code errorCount} counter. + * @param destKey destination + * @param uploadId multipart operation Id + * @param partETags list of partial uploads + * @param length length of the upload + * @param operationState operational state for a bulk update + * @return the result of the operation. + * @throws IOException if problems arose which could not be retried, or + * the retry count was exceeded + */ + @Retries.RetryTranslated + public CompleteMultipartUploadResult commitUpload( + String destKey, + String uploadId, + List partETags, + long length, + @Nullable BulkOperationState operationState) + throws IOException { + checkNotNull(uploadId); + checkNotNull(partETags); + LOG.debug("Completing multipart upload {} with {} parts", + uploadId, partETags.size()); + return finalizeMultipartUpload(destKey, + uploadId, + partETags, + length, + Invoker.NO_OP, + operationState); + } + + /** + * Initiate a commit operation through any metastore. + * @param path path under which the writes will all take place. + * @return an possibly null operation state from the metastore. + * @throws IOException failure to instantiate. + */ + public BulkOperationState initiateCommitOperation( + Path path) throws IOException { + return S3Guard.initiateBulkWrite(owner.getMetadataStore(), path); + } + /** * Upload part of a multi-partition file. * @param request request diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java index ed608cb983186..a49ab52b1ffd2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java @@ -442,14 +442,27 @@ protected void commitPendingUploads(JobContext context, } LOG.debug("{}: committing the output of {} task(s)", getRole(), pending.size()); - Tasks.foreach(pending) - .stopOnFailure() - .executeWith(buildThreadPool(context)) - .onFailure((commit, exception) -> - getCommitOperations().abortSingleCommit(commit)) - .abortWith(commit -> getCommitOperations().abortSingleCommit(commit)) - .revertWith(commit -> getCommitOperations().revertCommit(commit)) - .run(commit -> getCommitOperations().commitOrFail(commit)); + try(CommitOperations.CommitContext commitContext + = initiateCommitOperation()) { + Tasks.foreach(pending) + .stopOnFailure() + .executeWith(buildThreadPool(context)) + .onFailure((commit, exception) -> + commitContext.abortSingleCommit(commit)) + .abortWith(commitContext::abortSingleCommit) + .revertWith(commitContext::revertCommit) + .run(commitContext::commitOrFail); + } + } + + /** + * Start the final commit/abort commit operations. + * @return a commit context through which the operations can be invoked. + * @throws IOException failure. + */ + protected CommitOperations.CommitContext initiateCommitOperation() + throws IOException { + return getCommitOperations().initiateCommitOperation(getOutputPath()); } /** @@ -531,7 +544,9 @@ protected void abortPendingUploadsInCleanup( Path dest = getOutputPath(); try (DurationInfo d = new DurationInfo(LOG, "Aborting all pending commits under %s", - dest)) { + dest); + CommitOperations.CommitContext commitContext + = initiateCommitOperation()) { CommitOperations ops = getCommitOperations(); List pending; try { @@ -544,7 +559,8 @@ protected void abortPendingUploadsInCleanup( Tasks.foreach(pending) .executeWith(buildThreadPool(getJobContext())) .suppressExceptions(suppressExceptions) - .run(u -> ops.abortMultipartCommit(u.getKey(), u.getUploadId())); + .run(u -> commitContext.abortMultipartCommit( + u.getKey(), u.getUploadId())); } } @@ -752,11 +768,13 @@ protected void abortPendingUploads(JobContext context, LOG.info("{}: no pending commits to abort", getRole()); } else { try (DurationInfo d = new DurationInfo(LOG, - "Aborting %s uploads", pending.size())) { + "Aborting %s uploads", pending.size()); + CommitOperations.CommitContext commitContext + = initiateCommitOperation()) { Tasks.foreach(pending) .executeWith(buildThreadPool(context)) .suppressExceptions(suppressExceptions) - .run(commit -> getCommitOperations().abortSingleCommit(commit)); + .run(commitContext::abortSingleCommit); } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java index 55ace17b8a21e..ce80945b25a29 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java @@ -18,13 +18,14 @@ package org.apache.hadoop.fs.s3a.commit; +import javax.annotation.Nullable; +import java.io.Closeable; import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -49,6 +50,8 @@ import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; +import org.apache.hadoop.io.IOUtils; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; @@ -128,10 +131,12 @@ protected S3AInstrumentation.CommitterStatistics getStatistics() { /** * Commit the operation, throwing an exception on any failure. * @param commit commit to execute + * @param operationState S3Guard state of ongoing operation. * @throws IOException on a failure */ - public void commitOrFail(SinglePendingCommit commit) throws IOException { - commit(commit, commit.getFilename()).maybeRethrow(); + private void commitOrFail(SinglePendingCommit commit, + final BulkOperationState operationState) throws IOException { + commit(commit, commit.getFilename(), operationState).maybeRethrow(); } /** @@ -139,16 +144,20 @@ public void commitOrFail(SinglePendingCommit commit) throws IOException { * and converted to an outcome. * @param commit entry to commit * @param origin origin path/string for outcome text + * @param operationState S3Guard state of ongoing operation. * @return the outcome */ - public MaybeIOE commit(SinglePendingCommit commit, String origin) { + private MaybeIOE commit( + final SinglePendingCommit commit, + final String origin, + final BulkOperationState operationState) { LOG.debug("Committing single commit {}", commit); MaybeIOE outcome; String destKey = "unknown destination"; try { commit.validate(); destKey = commit.getDestinationKey(); - long l = innerCommit(commit); + long l = innerCommit(commit, operationState); LOG.debug("Successful commit of file length {}", l); outcome = MaybeIOE.NONE; statistics.commitCompleted(commit.getLength()); @@ -171,17 +180,20 @@ public MaybeIOE commit(SinglePendingCommit commit, String origin) { /** * Inner commit operation. * @param commit entry to commit + * @param operationState S3Guard state of ongoing operation. * @return bytes committed. * @throws IOException failure */ - private long innerCommit(SinglePendingCommit commit) throws IOException { + private long innerCommit( + final SinglePendingCommit commit, + final BulkOperationState operationState) throws IOException { // finalize the commit - writeOperations.completeMPUwithRetries( + writeOperations.commitUpload( commit.getDestinationKey(), commit.getUploadId(), toPartEtags(commit.getEtags()), commit.getLength(), - new AtomicInteger(0)); + operationState); return commit.getLength(); } @@ -249,7 +261,7 @@ public IOException makeIOE(String key, Exception ex) { * @throws FileNotFoundException if the abort ID is unknown * @throws IOException on any failure */ - public void abortSingleCommit(SinglePendingCommit commit) + private void abortSingleCommit(SinglePendingCommit commit) throws IOException { String destKey = commit.getDestinationKey(); String origin = commit.getFilename() != null @@ -268,7 +280,7 @@ public void abortSingleCommit(SinglePendingCommit commit) * @throws FileNotFoundException if the abort ID is unknown * @throws IOException on any failure */ - public void abortMultipartCommit(String destKey, String uploadId) + private void abortMultipartCommit(String destKey, String uploadId) throws IOException { try { writeOperations.abortMultipartCommit(destKey, uploadId); @@ -520,6 +532,116 @@ public void jobCompleted(boolean success) { statistics.jobCompleted(success); } + /** + * Begin the final commit. + * @param path path for all work. + * @return the commit context to pass in. + * @throws IOException failure. + */ + public CommitContext initiateCommitOperation(Path path) throws IOException { + return new CommitContext(writeOperations.initiateCommitOperation(path)); + } + + /** + * Commit context. + * + * It is used to manage the final commit sequence where files become + * visible. It contains a {@link BulkOperationState} field, which, if + * there is a metastore, will be requested from the store so that it + * can track multiple creation operations within the same overall operation. + * This will be null if there is no metastore, or the store chooses not + * to provide one. + * + * This can only be created through {@link #initiateCommitOperation(Path)}. + * + * Once the commit operation has completed, it must be closed. + * It must not be reused. + */ + public final class CommitContext implements Closeable { + + /** + * State of any metastore. + */ + private final BulkOperationState operationState; + + private CommitContext(@Nullable final BulkOperationState operationState) { + this.operationState = operationState; + } + + /** + * Commit the operation, throwing an exception on any failure. + * See {@link CommitOperations#commitOrFail(SinglePendingCommit, BulkOperationState)}. + * @param commit commit to execute + * @throws IOException on a failure + */ + public void commitOrFail(SinglePendingCommit commit) throws IOException { + CommitOperations.this.commitOrFail(commit, operationState); + } + + /** + * Commit a single pending commit; exceptions are caught + * and converted to an outcome. + * See {@link CommitOperations#commit(SinglePendingCommit, String, BulkOperationState)}. + * @param commit entry to commit + * @param origin origin path/string for outcome text + * @return the outcome + */ + public MaybeIOE commit(SinglePendingCommit commit, + String origin) { + return CommitOperations.this.commit(commit, origin, operationState); + } + + /** + * See {@link CommitOperations#abortSingleCommit(SinglePendingCommit)}. + * @param commit pending commit to abort + * @throws FileNotFoundException if the abort ID is unknown + * @throws IOException on any failure + */ + public void abortSingleCommit(final SinglePendingCommit commit) + throws IOException { + CommitOperations.this.abortSingleCommit(commit); + } + + /** + * See {@link CommitOperations#revertCommit(SinglePendingCommit)}. + * @param commit pending commit + * @throws IOException failure + */ + public void revertCommit(final SinglePendingCommit commit) + throws IOException { + CommitOperations.this.revertCommit(commit); + } + + /** + * See {@link CommitOperations#abortMultipartCommit(String, String)}.. + * @param destKey destination key + * @param uploadId upload to cancel + * @throws FileNotFoundException if the abort ID is unknown + * @throws IOException on any failure + */ + public void abortMultipartCommit( + final String destKey, + final String uploadId) + throws IOException { + CommitOperations.this.abortMultipartCommit(destKey, uploadId); + } + + @Override + public void close() throws IOException { + IOUtils.cleanupWithLogger(LOG, operationState); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "CommitContext{"); + sb.append("operationState=").append(operationState); + sb.append('}'); + return sb.toString(); + } + + } + /** * A holder for a possible IOException; the call {@link #maybeRethrow()} * will throw any exception passed into the constructor, and be a no-op diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java index f26384de49e83..518d789718a80 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java @@ -41,6 +41,7 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter; import org.apache.hadoop.fs.s3a.commit.CommitConstants; +import org.apache.hadoop.fs.s3a.commit.CommitOperations; import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants; import org.apache.hadoop.fs.s3a.commit.Tasks; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; @@ -729,9 +730,14 @@ protected int commitTaskInternal(final TaskAttemptContext context, LOG.error( "{}: Exception during commit process, aborting {} commit(s)", getRole(), commits.size()); - Tasks.foreach(commits) - .suppressExceptions() - .run(commit -> getCommitOperations().abortSingleCommit(commit)); + try(CommitOperations.CommitContext commitContext + = initiateCommitOperation(); + DurationInfo ignored = new DurationInfo(LOG, + "Aborting %s uploads", commits.size())) { + Tasks.foreach(commits) + .suppressExceptions() + .run(commitContext::abortSingleCommit); + } deleteTaskAttemptPathQuietly(context); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java new file mode 100644 index 0000000000000..761b4f587d3d8 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CallableSupplier.java @@ -0,0 +1,121 @@ +/* + * 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.impl; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.CancellationException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; +import java.util.function.Supplier; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.impl.WrappedIOException; +import org.apache.hadoop.util.DurationInfo; + +import static org.apache.hadoop.fs.impl.FutureIOSupport.raiseInnerCause; + +/** + * A bridge from Callable to Supplier; catching exceptions + * raised by the callable and wrapping them as appropriate. + * @param return type. + */ +public final class CallableSupplier implements Supplier { + + private static final Logger LOG = + LoggerFactory.getLogger(CallableSupplier.class); + + private final Callable call; + + CallableSupplier(final Callable call) { + this.call = call; + } + + @Override + public Object get() { + try { + return call.call(); + } catch (RuntimeException e) { + throw e; + } catch (IOException e) { + throw new WrappedIOException(e); + } catch (Exception e) { + throw new WrappedIOException(new IOException(e)); + } + } + + /** + * Submit a callable into a completable future. + * RTEs are rethrown. + * Non RTEs are caught and wrapped; IOExceptions to + * {@link WrappedIOException} instances. + * @param executor executor. + * @param call call to invoke + * @param type + * @return the future to wait for + */ + @SuppressWarnings("unchecked") + public static CompletableFuture submit( + final Executor executor, + final Callable call) { + return CompletableFuture.supplyAsync( + new CallableSupplier(call), executor); + } + + /** + * Wait for a list of futures to complete. + * @param futures list of futures. + * @throws IOException if one of the called futures raised an IOE. + * @throws RuntimeException if one of the futures raised one. + */ + public static void waitForCompletion( + final List> futures) + throws IOException { + if (futures.isEmpty()) { + return; + } + // await completion + waitForCompletion(CompletableFuture.allOf( + futures.toArray(new CompletableFuture[0]))); + } + + /** + * Wait for a single of future to complete, extracting IOEs afterwards. + * @param future future to wait for. + * @throws IOException if one of the called futures raised an IOE. + * @throws RuntimeException if one of the futures raised one. + */ + public static void waitForCompletion( + final CompletableFuture future) + throws IOException { + try(DurationInfo ignore = + new DurationInfo(LOG, false, "Waiting for task completion")) { + future.join(); + } catch (CancellationException e) { + throw new IOException(e); + } catch (CompletionException e) { + raiseInnerCause(e); + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java new file mode 100644 index 0000000000000..e2f7caab5330c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/FunctionsRaisingIOE.java @@ -0,0 +1,54 @@ +/* + * 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.impl; + +import java.io.IOException; + +/** + * Evolving support for functional programming/lambda-expressions. + * Currently in the s3a module for experimentation and evolution; at + * some point it could be pulled into hadoop-common. + */ +public final class FunctionsRaisingIOE { + + private FunctionsRaisingIOE() { + } + + /** + * Function of arity 1 which may raise an IOException. + * @param type of arg1 + * @param type of return value. + */ + @FunctionalInterface + public interface FunctionRaisingIOE { + + R apply(T p) throws IOException; + } + + /** + * This is a callable which only raises an IOException. + * @param return type + */ + @FunctionalInterface + public interface CallableRaisingIOE { + + R apply() throws IOException; + + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java new file mode 100644 index 0000000000000..fac0779cc6a12 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteSupport.java @@ -0,0 +1,243 @@ +/* + * 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.impl; + +import java.io.IOException; +import java.nio.file.AccessDeniedException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; + +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.amazonaws.services.s3.model.MultiObjectDeleteException; +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.lang3.tuple.Triple; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.AWSS3IOException; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; + +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * Support for Multi Object Deletion. + */ +public final class MultiObjectDeleteSupport { + + private static final Logger LOG = LoggerFactory.getLogger( + MultiObjectDeleteSupport.class); + + private final StoreContext context; + + /** + * Initiate with a store context. + * @param context store context. + */ + public MultiObjectDeleteSupport(final StoreContext context) { + this.context = context; + } + + /** + * This is the exception exit code if access was denied on a delete. + * {@value}. + */ + public static final String ACCESS_DENIED = "AccessDenied"; + + /** + * A {@code MultiObjectDeleteException} is raised if one or more + * paths listed in a bulk DELETE operation failed. + * The top-level exception is therefore just "something wasn't deleted", + * but doesn't include the what or the why. + * This translation will extract an AccessDeniedException if that's one of + * the causes, otherwise grabs the status code and uses it in the + * returned exception. + * @param message text for the exception + * @param deleteException the delete exception. to translate + * @return an IOE with more detail. + */ + public static IOException translateDeleteException( + final String message, + final MultiObjectDeleteException deleteException) { + final StringBuilder result = new StringBuilder( + deleteException.getErrors().size() * 256); + result.append(message).append(": "); + String exitCode = ""; + for (MultiObjectDeleteException.DeleteError error : + deleteException.getErrors()) { + String code = error.getCode(); + result.append(String.format("%s: %s: %s%n", code, error.getKey(), + error.getMessage())); + if (exitCode.isEmpty() || ACCESS_DENIED.equals(code)) { + exitCode = code; + } + } + if (ACCESS_DENIED.equals(exitCode)) { + return (IOException) new AccessDeniedException(result.toString()) + .initCause(deleteException); + } else { + return new AWSS3IOException(result.toString(), deleteException); + } + } + + /** + * Process a multi object delete exception by building two paths from + * the delete request: one of all deleted files, one of all undeleted values. + * The latter are those rejected in the delete call. + * @param deleteException the delete exception. + * @param keysToDelete the keys in the delete request + * @return tuple of (undeleted, deleted) paths. + */ + public Pair, List> splitUndeletedKeys( + final MultiObjectDeleteException deleteException, + final Collection keysToDelete) { + LOG.debug("Processing delete failure; keys to delete count = {};" + + " errors in exception {}; successful deletions = {}", + keysToDelete.size(), + deleteException.getErrors().size(), + deleteException.getDeletedObjects().size()); + // convert the collection of keys being deleted into paths + final List pathsBeingDeleted = keysToPaths(keysToDelete); + // Take this is list of paths + // extract all undeleted entries contained in the exception and + // then removes them from the original list. + List undeleted = removeUndeletedPaths(deleteException, pathsBeingDeleted, + context.getKeyToPathQualifier()); + return Pair.of(undeleted, pathsBeingDeleted); + } + + /** + * Given a list of delete requests, convert them all to paths. + * @param keysToDelete list of keys for the delete operation. + * @return the paths. + */ + public List keysToPaths( + final Collection keysToDelete) { + Function qualifier + = context.getKeyToPathQualifier(); + return convertToPaths(keysToDelete, qualifier); + } + + /** + * Given a list of delete requests, convert them all to paths. + * @param keysToDelete list of keys for the delete operation. + * @param qualifier path qualifier + * @return the paths. + */ + public static List convertToPaths( + final Collection keysToDelete, + final Function qualifier) { + return keysToDelete.stream() + .map((keyVersion) -> + qualifier.apply(keyVersion.getKey())) + .collect(Collectors.toList()); + } + + /** + * Process a delete failure by removing from the metastore all entries + * which where deleted, as inferred from the delete failures exception + * and the original list of files to delete declares to have been deleted. + * @param deleteException the delete exception. + * @param keysToDelete collection of keys which had been requested. + * @return a tuple of (undeleted, deleted, failures) + */ + public Triple, List, List>> + processDeleteFailure( + + final MultiObjectDeleteException deleteException, + final List keysToDelete) { + final MetadataStore metadataStore = + checkNotNull(context.getMetadataStore(), "context metadatastore"); + final List> failures = new ArrayList<>(); + final Pair, List> outcome = + splitUndeletedKeys(deleteException, keysToDelete); + List deleted = outcome.getRight(); + List undeleted = outcome.getLeft(); + // delete the paths but recover + deleted.forEach(path -> { + try { + metadataStore.delete(path); + } catch (IOException e) { + // trouble: we failed to delete the far end entry + // try with the next one. + // if this is a big network failure, this is going to be noisy. + LOG.warn("Failed to update S3Guard store with deletion of {}", path); + failures.add(Pair.of(path, e)); + } + }); + if (LOG.isDebugEnabled()) { + undeleted.forEach(p -> LOG.debug("Deleted {}", p)); + } + return Triple.of(undeleted, deleted, failures); + } + + /** + * Build a list of undeleted paths from a {@code MultiObjectDeleteException}. + * Outside of unit tests, the qualifier function should be + * {@link S3AFileSystem#keyToQualifiedPath(String)}. + * @param deleteException the delete exception. + * @param qualifierFn function to qualify paths + * @return the possibly empty list of paths. + */ + @VisibleForTesting + public static List extractUndeletedPaths( + final MultiObjectDeleteException deleteException, + final Function qualifierFn) { + return deleteException.getErrors().stream() + .map((e) -> qualifierFn.apply(e.getKey())) + .collect(Collectors.toList()); + } + + /** + * Process a {@code MultiObjectDeleteException} by + * removing all undeleted paths from the list of paths being deleted. + * The original list is updated, and so becomes the list of successfully + * deleted paths. + * @param deleteException the delete exception. + * @param pathsBeingDeleted list of paths which were being deleted. + * This has all undeleted paths removed, leaving only those deleted. + * @return the list of undeleted entries + */ + @VisibleForTesting + static List removeUndeletedPaths( + final MultiObjectDeleteException deleteException, + final Collection pathsBeingDeleted, + final Function qualifier) { + List undeleted = extractUndeletedPaths(deleteException, qualifier); + pathsBeingDeleted.removeAll(undeleted); + return undeleted; + } + + /** + * A delete operation failed. + * Currently just returns the list of all paths. + * @param ex exception. + * @param keysToDelete the keys which were being deleted. + * @return all paths which were not deleted. + */ + public List processDeleteFailureGenericException(Exception ex, + final List keysToDelete) { + return keysToPaths(keysToDelete); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java new file mode 100644 index 0000000000000..92e2ffd5b0700 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContext.java @@ -0,0 +1,334 @@ +/* + * 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.impl; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.util.Optional; +import java.util.function.Function; + +import com.google.common.util.concurrent.ListeningExecutorService; + +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.Invoker; +import org.apache.hadoop.fs.s3a.S3AInputPolicy; +import org.apache.hadoop.fs.s3a.S3AInstrumentation; +import org.apache.hadoop.fs.s3a.S3AStorageStatistics; +import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.SemaphoredDelegatingExecutor; + +/** + * This class provides the core context of the S3A filesystem to subsidiary + * components, without exposing the entire parent class. + * This is eliminate explicit recursive coupling. + * + * Where methods on the FS are to be invoked, they are all passed in + * via functional interfaces, so test setups can pass in mock callbacks + * instead. + * + * Warning: this really is private and unstable. Do not use + * outside the org.apache.hadoop.fs.s3a package. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class StoreContext { + + /** Filesystem URI. */ + private final URI fsURI; + + /** Bucket name. */ + private final String bucket; + + /** FS configuration after all per-bucket overrides applied. */ + private final Configuration configuration; + + /** Username. */ + private final String username; + + /** Principal who created the FS. */ + private final UserGroupInformation owner; + + /** + * Location of a bucket. + * Optional as the AWS call to evaluate this may fail from a permissions + * or other IOE. + */ + private final Optional bucketLocation; + + /** + * Bounded thread pool for async operations. + */ + private final ListeningExecutorService executor; + + /** + * Capacity of new executors created. + */ + private final int executorCapacity; + + /** Invoker of operations. */ + private final Invoker invoker; + + /** Instrumentation and statistics. */ + private final S3AInstrumentation instrumentation; + private final S3AStorageStatistics storageStatistics; + + /** Seek policy. */ + private final S3AInputPolicy inputPolicy; + + /** How to react to changes in etags and versions. */ + private final ChangeDetectionPolicy changeDetectionPolicy; + + /** Evaluated options. */ + private final boolean multiObjectDeleteEnabled; + + /** List algorithm. */ + private final boolean useListV1; + + /** Is the store versioned? */ + private final boolean versioned; + + /** + * To allow this context to be passed down to the metastore, this field + * wll be null until initialized. + */ + private final MetadataStore metadataStore; + + /** Function to take a key and return a path. */ + private final Function keyToPathQualifier; + + /** Factory for temporary files. */ + private final TempFileFactory tempFileFactory; + + /** + * Instantiate. + * No attempt to use a builder here as outside tests + * this should only be created in the S3AFileSystem. + */ + public StoreContext(final URI fsURI, + final String bucket, + final Configuration configuration, + final String username, + final UserGroupInformation owner, + final ListeningExecutorService executor, + final int executorCapacity, + final Invoker invoker, + final S3AInstrumentation instrumentation, + final S3AStorageStatistics storageStatistics, + final S3AInputPolicy inputPolicy, + final ChangeDetectionPolicy changeDetectionPolicy, + final boolean multiObjectDeleteEnabled, + final MetadataStore metadataStore, + final Function keyToPathQualifier, + final String bucketLocation, + final boolean useListV1, + final boolean versioned, + final TempFileFactory tempFileFactory) { + this.fsURI = fsURI; + this.bucket = bucket; + this.configuration = configuration; + this.username = username; + this.owner = owner; + this.executor = executor; + this.executorCapacity = executorCapacity; + this.invoker = invoker; + this.instrumentation = instrumentation; + this.storageStatistics = storageStatistics; + this.inputPolicy = inputPolicy; + this.changeDetectionPolicy = changeDetectionPolicy; + this.multiObjectDeleteEnabled = multiObjectDeleteEnabled; + this.metadataStore = metadataStore; + this.keyToPathQualifier = keyToPathQualifier; + this.bucketLocation = Optional.ofNullable(bucketLocation); + this.useListV1 = useListV1; + this.versioned = versioned; + this.tempFileFactory = tempFileFactory; + } + + @Override + protected Object clone() throws CloneNotSupportedException { + return super.clone(); + } + + public URI getFsURI() { + return fsURI; + } + + public String getBucket() { + return bucket; + } + + public Configuration getConfiguration() { + return configuration; + } + + public String getUsername() { + return username; + } + + public Optional getBucketLocation() { + return bucketLocation; + } + + public ListeningExecutorService getExecutor() { + return executor; + } + + public Invoker getInvoker() { + return invoker; + } + + public S3AInstrumentation getInstrumentation() { + return instrumentation; + } + + public S3AInputPolicy getInputPolicy() { + return inputPolicy; + } + + public ChangeDetectionPolicy getChangeDetectionPolicy() { + return changeDetectionPolicy; + } + + public boolean isMultiObjectDeleteEnabled() { + return multiObjectDeleteEnabled; + } + + public MetadataStore getMetadataStore() { + return metadataStore; + } + + public boolean isUseListV1() { + return useListV1; + } + + public boolean isVersioned() { + return versioned; + } + + public Function getKeyToPathQualifier() { + return keyToPathQualifier; + } + + /** + * Get the storage statistics of this filesystem. + * @return the storage statistics + */ + public S3AStorageStatistics getStorageStatistics() { + return storageStatistics; + } + + /** + * Increment a statistic by 1. + * This increments both the instrumentation and storage statistics. + * @param statistic The operation to increment + */ + public void incrementStatistic(Statistic statistic) { + incrementStatistic(statistic, 1); + } + + /** + * Increment a statistic by a specific value. + * This increments both the instrumentation and storage statistics. + * @param statistic The operation to increment + * @param count the count to increment + */ + public void incrementStatistic(Statistic statistic, long count) { + instrumentation.incrementCounter(statistic, count); + storageStatistics.incrementCounter(statistic, count); + } + + /** + * Decrement a gauge by a specific value. + * @param statistic The operation to decrement + * @param count the count to decrement + */ + public void decrementGauge(Statistic statistic, long count) { + instrumentation.decrementGauge(statistic, count); + } + + /** + * Increment a gauge by a specific value. + * @param statistic The operation to increment + * @param count the count to increment + */ + public void incrementGauge(Statistic statistic, long count) { + instrumentation.incrementGauge(statistic, count); + } + + /** + * Create a new executor service with a given capacity. + * This executor submits works to the {@link #executor}, using a + * {@link SemaphoredDelegatingExecutor} to limit the number + * of requests coming in from a specific client. + * + * Because this delegates to an existing thread pool, the cost of + * creating a new instance here is low. + * As the throttling is per instance, separate instances + * should be created for each operation which wishes to execute work in + * parallel without saturating the base executor. + * This is important if either the duration of each operation is long + * or the submission rate of work is high. + * @param capacity maximum capacity of this executor. + * @return an executor for submitting work. + */ + public ListeningExecutorService createThrottledExecutor(int capacity) { + return new SemaphoredDelegatingExecutor(executor, + capacity, true); + } + + /** + * Create a new executor with the capacity defined in + * {@link #executorCapacity}. + * @return a new executor for exclusive use by the caller. + */ + public ListeningExecutorService createThrottledExecutor() { + return createThrottledExecutor(executorCapacity); + } + + public UserGroupInformation getOwner() { + return owner; + } + + /** + * Create a temporary file somewhere. + * @param prefix prefix for the temporary file + * @param size expected size. + * @return a file reference. + * @throws IOException failure. + */ + public File createTempFile(String prefix, long size) throws IOException { + return tempFileFactory.createTempFile(prefix, size); + } + + /** + * The interface for temporary files. + * The standard Java 8 BiFunction cannot be used as it doesn't raise an + * IOE. + */ + @FunctionalInterface + public interface TempFileFactory { + File createTempFile(String prefix, long size) throws IOException; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreOperation.java new file mode 100644 index 0000000000000..76d7c7b13b2d6 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreOperation.java @@ -0,0 +1,42 @@ +/* + * 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.impl; + +/** + * Base class of operation in the store. + * An operation is something which executes against the context to + * perform a single function. + * It is expected to have a limited lifespan. + */ +public abstract class StoreOperation { + + private final StoreContext storeContext; + + /** + * constructor. + * @param storeContext store context. + */ + public StoreOperation(final StoreContext storeContext) { + this.storeContext = storeContext; + } + + public StoreContext getStoreContext() { + return storeContext; + } +} 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 new file mode 100644 index 0000000000000..9f2b90c2b5256 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/BulkOperationState.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.s3guard; + +import java.io.Closeable; +import java.io.IOException; + +/** + * This represents state which may be passed to bulk IO operations + * to enable them to store information about the state of the ongoing + * operation across invocations. + * + * A bulk operation state MUST only be be used for the single store + * from which it was created, and MUSTonly for the duration of a single + * bulk update operation. + * + * After the operation has completed, it MUST be closed so + * as to guarantee that all state is released. + */ +public class BulkOperationState implements Closeable { + + @Override + public void close() throws IOException { + + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java new file mode 100644 index 0000000000000..5c5f52377948a --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DelayedUpdateRenameTracker.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.s3guard; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import com.amazonaws.SdkBaseException; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.Tristate; +import org.apache.hadoop.fs.s3a.impl.StoreContext; + +import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveAncestors; +import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveDir; +import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveFile; + +/** + * This is the rename updating strategy originally used: + * a collection of source paths and a list of destinations are created, + * then updated at the end (possibly slow). + */ +public class DelayedUpdateRenameTracker extends RenameTracker { + + private final MetadataStore metadataStore; + + private final Collection sourcePaths = new HashSet<>(); + + private final List destMetas = new ArrayList<>(); + + private final List deletedPaths = new ArrayList<>(); + + public DelayedUpdateRenameTracker( + final StoreContext storeContext, + final MetadataStore metadataStore, + final Path sourceRoot, + final Path dest, + final BulkOperationState operationState) { + super("DelayedUpdateRenameTracker", storeContext, metadataStore, + sourceRoot, dest, operationState); + this.metadataStore = storeContext.getMetadataStore(); + } + + @Override + public synchronized void fileCopied( + final Path sourcePath, + final FileStatus sourceStatus, + final Path destPath, + final long blockSize, + final boolean addAncestors) throws IOException { + addMoveFile(metadataStore, + sourcePaths, + destMetas, + sourcePath, + destPath, + sourceStatus.getLen(), + blockSize, + getOwner()); + // Ancestor directories may not be listed, so we explicitly add them + if (addAncestors) { + addMoveAncestors(metadataStore, + sourcePaths, + destMetas, + getSourceRoot(), + sourceStatus.getPath(), + destPath, + getOwner()); + } + } + + @Override + public synchronized void directoryMarkerCopied(final FileStatus sourceStatus, + final Path destPath, + final boolean addAncestors) throws IOException { + addMoveDir(metadataStore, sourcePaths, destMetas, + sourceStatus.getPath(), + destPath, getOwner()); + // Ancestor directories may not be listed, so we explicitly add them + if (addAncestors) { + addMoveAncestors(metadataStore, + sourcePaths, + destMetas, + getSourceRoot(), + sourceStatus.getPath(), + destPath, + getOwner()); + } + } + + @Override + public synchronized void moveSourceDirectory() throws IOException { + if (!sourcePaths.contains(getSourceRoot())) { + addMoveDir(metadataStore, sourcePaths, destMetas, + getSourceRoot(), + getDest(), getOwner()); + } + } + + @Override + public synchronized void sourceObjectsDeleted( + final List paths) throws IOException { + // convert to paths. + deletedPaths.addAll(paths); + } + + @Override + public void completeRename() throws IOException { + metadataStore.move(sourcePaths, destMetas, getOperationState()); + super.completeRename(); + } + + @Override + public IOException renameFailed(final Exception ex) { + LOG.warn("Rename has failed; updating s3guard with destination state"); + try { + // the destination paths are updated; the source is left alone. + // either the delete operation didn't begin, or the + metadataStore.move(new ArrayList<>(0), destMetas, getOperationState()); + for (Path deletedPath : deletedPaths) { + // this is not ideal in that it may leave parent stuff around. + metadataStore.delete(deletedPath); + } + deleteParentPaths(); + } catch (IOException | SdkBaseException e) { + LOG.warn("Ignoring error raised in AWS SDK ", e); + } + + return super.renameFailed(ex); + } + + /** + * Delete all the parent paths we know to be empty (by walking up the tree + * deleting as appropriate). + * @throws IOException failure + */ + private void deleteParentPaths() throws IOException { + Set parentPaths = new HashSet<>(); + for (Path deletedPath : deletedPaths) { + Path parent = deletedPath.getParent(); + if (!parent.equals(getSourceRoot())) { + parentPaths.add(parent); + } + } + // now there's a set of parent paths. We now want to + // get them ordered by depth, so that deeper entries come first + // that way: when we check for a parent path existing we can + // see if it really is empty. + List parents = new ArrayList<>(parentPaths); + parents.sort(PathOrderComparators.TOPMOST_PATH_LAST); + for (Path parent : parents) { + PathMetadata md = metadataStore.get(parent, true); + if (md != null && md.isEmptyDirectory().equals(Tristate.TRUE)) { + // if were confident that this is empty: delete it. + metadataStore.delete(parent); + } + } + } +} 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 769d3d4c4c376..696c128c94475 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.s3guard; +import javax.annotation.Nullable; import java.io.FileNotFoundException; import java.io.IOException; import java.io.InterruptedIOException; @@ -35,6 +36,7 @@ import java.util.Objects; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -72,6 +74,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import com.google.common.util.concurrent.ListeningExecutorService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,15 +98,19 @@ import org.apache.hadoop.fs.s3a.auth.RoleModel; import org.apache.hadoop.fs.s3a.auth.RolePolicies; import org.apache.hadoop.fs.s3a.auth.delegation.AWSPolicyProvider; +import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.BlockingThreadPoolExecutorService; import org.apache.hadoop.util.ReflectionUtils; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowAllDynamoDBOperations; import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3GuardClientOperations; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion; import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*; import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.*; @@ -180,7 +187,8 @@ * sub-tree. * * Some mutating operations, notably {@link #deleteSubtree(Path)} and - * {@link #move(Collection, Collection)}, are less efficient with this schema. + * {@link MetadataStore#move(Collection, Collection, BulkOperationState)} + * are less efficient with this schema. * They require mutating multiple items in the DynamoDB table. * * By default, DynamoDB access is performed within the same AWS region as @@ -247,6 +255,13 @@ public class DynamoDBMetadataStore implements MetadataStore, private static ValueMap deleteTrackingValueMap = new ValueMap().withBoolean(":false", false); + /** + * The maximum number of oustanding operations to submit at a time + * in any operation whch submits work through the executors. + * Value: {@value}. + */ + private static final int S3GUARD_DDB_SUBMITTED_TASK_LIMIT = 50; + private AmazonDynamoDB amazonDynamoDB; private DynamoDB dynamoDB; private AWSCredentialProviderList credentials; @@ -296,6 +311,8 @@ public class DynamoDBMetadataStore implements MetadataStore, */ private AtomicInteger throttleEventCount = new AtomicInteger(0); + private ListeningExecutorService executor; + /** * A utility function to create DynamoDB instance. * @param conf the file system configuration @@ -386,8 +403,10 @@ public void initialize(FileSystem fs) throws IOException { void bindToOwnerFilesystem(final S3AFileSystem fs) { owner = fs; conf = owner.getConf(); - instrumentation = owner.getInstrumentation().getS3GuardInstrumentation(); - username = owner.getUsername(); + StoreContext context = owner.createStoreContext(); + instrumentation = context.getInstrumentation().getS3GuardInstrumentation(); + username = context.getUsername(); + executor = context.createThrottledExecutor(); } /** @@ -432,6 +451,18 @@ public void initialize(Configuration config) throws IOException { dynamoDB = createDynamoDB(conf, region, null, credentials); username = UserGroupInformation.getCurrentUser().getShortUserName(); + // without an executor from the owner FS, create one using + // the executor capacity for work. + int executorCapacity = intOption(conf, + EXECUTOR_CAPACITY, DEFAULT_EXECUTOR_CAPACITY, 1); + executor = BlockingThreadPoolExecutorService.newInstance( + executorCapacity, + executorCapacity * 2, + longOption(conf, KEEPALIVE_TIME, + DEFAULT_KEEPALIVE_TIME, 0), + TimeUnit.SECONDS, + "s3a-ddb-" + tableName); + initDataAccessRetries(conf); initTable(); @@ -525,16 +556,27 @@ public void deleteSubtree(Path path) throws IOException { return; } + // Execute via the bounded threadpool. + final List> futures = new ArrayList<>(); for (DescendantsIterator desc = new DescendantsIterator(this, meta); desc.hasNext();) { - innerDelete(desc.next().getPath(), true); + final Path pathToDelete = desc.next().getPath(); + futures.add(submit(executor, () -> { + innerDelete(pathToDelete, true); + return null; + })); + if (futures.size() > S3GUARD_DDB_SUBMITTED_TASK_LIMIT) { + // first batch done; block for completion. + waitForCompletion(futures); + futures.clear(); + } } + waitForCompletion(futures); } /** * Get a consistent view of an item. * @param path path to look up in the database - * @param path entry * @return the result * @throws IOException failure */ @@ -544,10 +586,12 @@ private Item getConsistentItem(final Path path) throws IOException { final GetItemSpec spec = new GetItemSpec() .withPrimaryKey(key) .withConsistentRead(true); // strictly consistent read - return readOp.retry("get", + Item item = readOp.retry("get", path.toString(), true, () -> table.getItem(spec)); + recordsRead(1); + return item; } @Override @@ -690,14 +734,19 @@ DirListingMetadata getDirListingMetadataFromDirMetaAndList(Path path, } /** - * build the list of all parent entries. + * Build the list of all parent entries. + * Thread safety: none. Callers are expected to + * synchronize on ancestorState as required. * @param pathsToCreate paths to create + * @param ancestorState ongoing ancestor state. * @return the full ancestry paths */ - Collection completeAncestry( - Collection pathsToCreate) { + private Collection completeAncestry( + final Collection pathsToCreate, + final AncestorState ancestorState) { // Key on path to allow fast lookup - Map ancestry = new HashMap<>(); + Map ancestry = ancestorState.getAncestry(); + List ancestorsToAdd = new ArrayList<>(0); for (DDBPathMetadata meta : pathsToCreate) { Preconditions.checkArgument(meta != null); @@ -705,24 +754,56 @@ Collection completeAncestry( if (path.isRoot()) { break; } - ancestry.put(path, new DDBPathMetadata(meta)); + // add the new entry + DDBPathMetadata entry = new DDBPathMetadata(meta); + DDBPathMetadata oldEntry = ancestry.put(path, entry); + if (oldEntry != null) { + // check for and warn if the existing bulk operation overwrote it. + // this should never occur outside tests. + LOG.warn("Overwriting a S3Guard entry created in the operation: {}", + oldEntry); + continue; + } + ancestorsToAdd.add(entry); Path parent = path.getParent(); while (!parent.isRoot() && !ancestry.containsKey(parent)) { LOG.debug("auto-create ancestor path {} for child path {}", parent, path); final FileStatus status = makeDirStatus(parent, username); - ancestry.put(parent, new DDBPathMetadata(status, Tristate.FALSE, - false)); + DDBPathMetadata md = new DDBPathMetadata(status, Tristate.FALSE, + false); + ancestry.put(parent, md); + ancestorsToAdd.add(md); parent = parent.getParent(); } } - return ancestry.values(); + return ancestorsToAdd; } + /** + * {@inheritDoc}. + * + * The DDB implementation sorts all the paths such that new items + * are ordered highest level entry first; deleted items are ordered + * lowest entry first. + * + * This is to ensure that if a client failed partway through the update, + * there will no entries in the table which lack parent entries. + * @param pathsToDelete Collection of all paths that were removed from the + * source directory tree of the move. + * @param pathsToCreate Collection of all PathMetadata for the new paths + * that were created at the destination of the rename + * (). + * @param operationState Any ongoing state supplied to the rename tracker + * which is to be passed in with each move operation. + * @throws IOException if there is an error + */ @Override @Retries.RetryTranslated - public void move(Collection pathsToDelete, - Collection pathsToCreate) throws IOException { + public void move( + @Nullable Collection pathsToDelete, + @Nullable Collection pathsToCreate, + @Nullable final BulkOperationState operationState) throws IOException { if (pathsToDelete == null && pathsToCreate == null) { return; } @@ -739,14 +820,29 @@ public void move(Collection pathsToDelete, // Following code is to maintain this invariant by putting all ancestor // directories of the paths to create. // ancestor paths that are not explicitly added to paths to create - Collection newItems = new ArrayList<>(); + AncestorState ancestorState = extractOrCreate(operationState); + List newItems = new ArrayList<>(); if (pathsToCreate != null) { - newItems.addAll(completeAncestry(pathMetaToDDBPathMeta(pathsToCreate))); + // create all parent entries. + // this is synchronized on the move state so that across both serialized + // and parallelized renames, duplicate ancestor entries are not created. + synchronized (ancestorState) { + newItems.addAll( + completeAncestry( + pathMetaToDDBPathMeta(pathsToCreate), + ancestorState)); + } } + // sort all the new items topmost first. + newItems.sort(PathOrderComparators.TOPMOST_PM_FIRST); if (pathsToDelete != null) { + List tombstones = new ArrayList<>(pathsToDelete.size()); for (Path meta : pathsToDelete) { - newItems.add(new DDBPathMetadata(PathMetadata.tombstone(meta))); + tombstones.add(new DDBPathMetadata(PathMetadata.tombstone(meta))); } + // sort all the tombstones lowest first. + tombstones.sort(PathOrderComparators.TOPMOST_PM_LAST); + newItems.addAll(tombstones); } processBatchWriteRequest(null, pathMetadataToItem(newItems)); @@ -771,6 +867,7 @@ private int processBatchWriteRequest(PrimaryKey[] keysToDelete, while (count < totalToDelete + totalToPut) { final TableWriteItems writeItems = new TableWriteItems(tableName); int numToDelete = 0; + int batchSize = 0; if (keysToDelete != null && count < totalToDelete) { numToDelete = Math.min(S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT, @@ -778,6 +875,7 @@ private int processBatchWriteRequest(PrimaryKey[] keysToDelete, writeItems.withPrimaryKeysToDelete( Arrays.copyOfRange(keysToDelete, count, count + numToDelete)); count += numToDelete; + batchSize = numToDelete; } if (numToDelete < S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT @@ -790,6 +888,7 @@ private int processBatchWriteRequest(PrimaryKey[] keysToDelete, writeItems.withItemsToPut( Arrays.copyOfRange(itemsToPut, index, index + numToPut)); count += numToPut; + batchSize += numToPut; } // if there's a retry and another process updates things then it's not @@ -802,10 +901,12 @@ private int processBatchWriteRequest(PrimaryKey[] keysToDelete, () -> dynamoDB.batchWriteItem(writeItems)); // Check for unprocessed keys in case of exceeding provisioned throughput Map> unprocessed = res.getUnprocessedItems(); + recordsWritten(batchSize - unprocessed.size()); int retryCount = 0; while (!unprocessed.isEmpty()) { batchWriteCapacityExceededEvents.incrementAndGet(); batches++; + batchSize = unprocessed.size(); retryBackoffOnBatchWrite(retryCount++); // use a different reference to keep the compiler quiet final Map> upx = unprocessed; @@ -815,6 +916,7 @@ private int processBatchWriteRequest(PrimaryKey[] keysToDelete, true, () -> dynamoDB.batchWriteItemUnprocessed(upx)); unprocessed = res.getUnprocessedItems(); + recordsWritten(batchSize - unprocessed.size()); } } return batches; @@ -867,7 +969,15 @@ private void retryBackoffOnBatchWrite(int retryCount) throws IOException { @Override @Retries.RetryTranslated - public void put(PathMetadata meta) throws IOException { + public void put(final PathMetadata meta) throws IOException { + put(meta, null); + } + + @Override + @Retries.RetryTranslated + public void put( + final PathMetadata meta, + @Nullable final BulkOperationState operationState) throws IOException { // For a deeply nested path, this method will automatically create the full // ancestry and save respective item in DynamoDB table. // So after put operation, we maintain the invariant that if a path exists, @@ -878,32 +988,51 @@ public void put(PathMetadata meta) throws IOException { Collection wrapper = new ArrayList<>(1); wrapper.add(meta); - put(wrapper); + put(wrapper, operationState); } @Override @Retries.RetryTranslated - public void put(Collection metas) throws IOException { - innerPut(pathMetaToDDBPathMeta(metas)); + public void put( + final Collection metas, + @Nullable final BulkOperationState operationState) throws IOException { + innerPut(pathMetaToDDBPathMeta(metas), operationState); } - @Retries.OnceRaw - private void innerPut(Collection metas) throws IOException { - Item[] items = pathMetadataToItem(completeAncestry(metas)); + @Retries.RetryTranslated + private void innerPut( + final Collection metas, + @Nullable final BulkOperationState operationState) throws IOException { + if (metas.isEmpty()) { + // this sometimes to appear in the logs, so log the full stack to + // identify it. + LOG.debug("Ignoring empty list of entries to put", + new Exception("source")); + return; + } + final AncestorState ancestorState = extractOrCreate(operationState); + + Item[] items; + synchronized (ancestorState) { + items = pathMetadataToItem( + completeAncestry(metas, ancestorState)); + } LOG.debug("Saving batch of {} items to table {}, region {}", items.length, tableName, region); processBatchWriteRequest(null, items); } /** - * Helper method to get full path of ancestors that are nonexistent in table. + * Get full path of ancestors that are nonexistent in table. + * + * This queries DDB. */ @VisibleForTesting @Retries.RetryTranslated - Collection fullPathsToPut(DDBPathMetadata meta) + List fullPathsToPut(DDBPathMetadata meta) throws IOException { checkPathMetadata(meta); - final Collection metasToPut = new ArrayList<>(); + final List metasToPut = new ArrayList<>(); // root path is not persisted if (!meta.getFileStatus().getPath().isRoot()) { metasToPut.add(meta); @@ -963,11 +1092,14 @@ public void put(DirListingMetadata meta) throws IOException { false, meta.isAuthoritative(), meta.getLastUpdated()); // First add any missing ancestors... - final Collection metasToPut = fullPathsToPut(ddbPathMeta); + final List metasToPut = fullPathsToPut(ddbPathMeta); // next add all children of the directory metasToPut.addAll(pathMetaToDDBPathMeta(meta.getListing())); + // 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)); } @@ -1056,33 +1188,38 @@ public void prune(long modTime, String keyPrefix) throws IOException { S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_DEFAULT, TimeUnit.MILLISECONDS); Set parentPathSet = new HashSet<>(); + Set clearedParentPathSet = new HashSet<>(); for (Item item : expiredFiles(modTime, keyPrefix)) { DDBPathMetadata md = PathMetadataDynamoDBTranslation .itemToPathMetadata(item, username); Path path = md.getFileStatus().getPath(); deletionBatch.add(path); - // add parent path of what we remove + // add parent path of what we remove if it has not + // already been processed Path parentPath = path.getParent(); - if (parentPath != null) { + if (parentPath != null && !clearedParentPathSet.contains(parentPath)) { parentPathSet.add(parentPath); } itemCount++; if (deletionBatch.size() == S3GUARD_DDB_BATCH_WRITE_REQUEST_LIMIT) { - Thread.sleep(delay); processBatchWriteRequest(pathToKey(deletionBatch), null); // set authoritative false for each pruned dir listing removeAuthoritativeDirFlag(parentPathSet); + // already cleared parent paths. + clearedParentPathSet.addAll(parentPathSet); parentPathSet.clear(); deletionBatch.clear(); + if (delay > 0) { + Thread.sleep(delay); + } } } // final batch of deletes if (!deletionBatch.isEmpty()) { - Thread.sleep(delay); processBatchWriteRequest(pathToKey(deletionBatch), null); // set authoritative false for each pruned dir listing @@ -1121,7 +1258,9 @@ private void removeAuthoritativeDirFlag(Set pathSet) try { LOG.debug("innerPut on metas: {}", metas); - innerPut(metas); + if (!metas.isEmpty()) { + innerPut(metas, null); + } } catch (IOException e) { String msg = String.format("IOException while setting false " + "authoritative directory flag on: %s.", metas); @@ -1707,6 +1846,51 @@ public Invoker getInvoker() { return invoker; } + /** + * Record the number of records written. + * @param count count of records. + */ + private void recordsWritten(final int count) { + if (instrumentation != null) { + instrumentation.recordsWritten(count); + } + } + + /** + * Record the number of records read. + * @param count count of records. + */ + private void recordsRead(final int count) { + if (instrumentation != null) { + instrumentation.recordsRead(count); + } + } + + /** + * Initiate the rename operation by creating the tracker and the ongoing + * move state. + * @param storeContext store context. + * @param source source path + * @param sourceStatus status of the source file/dir + * @param dest destination path. + * @return the rename tracker + */ + @Override + public RenameTracker initiateRenameOperation( + final StoreContext storeContext, + final Path source, + final FileStatus sourceStatus, + final Path dest) { + return new ProgressiveRenameTracker(storeContext, this, source, dest, + new AncestorState(dest)); + } + + @Override + public AncestorState initiateBulkWrite(final Path dest) + throws IOException { + return new AncestorState(dest); + } + /** * Take an {@code IllegalArgumentException} raised by a DDB operation * and if it contains an inner SDK exception, unwrap it. @@ -1748,4 +1932,59 @@ static IOException translateTableWaitFailure( return new IOException(e); } } + + /** + * Get the move state passed in; create a new one if needed. + * @param state state. + * @return the cast or created state. + */ + @VisibleForTesting + static AncestorState extractOrCreate(@Nullable BulkOperationState state) { + if (state != null) { + return (AncestorState) state; + } else { + return new AncestorState(null); + } + } + + /** + * This tracks all the ancestors created, + * across multiple move/write operations. + * This is to avoid duplicate creation of ancestors during bulk commits + * and rename operations managed by a rename tracker. + */ + @VisibleForTesting + static final class AncestorState extends BulkOperationState { + + private final Map ancestry = new HashMap<>(); + + private final Path dest; + + AncestorState(final Path dest) { + super(); + this.dest = dest; + } + + private Map getAncestry() { + return ancestry; + } + + int size() { + return ancestry.size(); + } + + public Path getDest() { + return dest; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "AncestorState{"); + sb.append("dest=").append(dest); + sb.append("size=").append(size()); + sb.append('}'); + return sb.toString(); + } + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java index b8f9635dcd283..bc728b6258944 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java @@ -18,6 +18,8 @@ package org.apache.hadoop.fs.s3a.s3guard; +import javax.annotation.Nullable; + import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; @@ -29,12 +31,15 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.Tristate; +import org.apache.hadoop.fs.s3a.impl.StoreContext; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.net.URI; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; import java.util.Map; @@ -189,14 +194,18 @@ public synchronized DirListingMetadata listChildren(Path p) throws } @Override - public void move(Collection pathsToDelete, - Collection pathsToCreate) throws IOException { + public void move( + @Nullable Collection pathsToDelete, + @Nullable Collection pathsToCreate, + @Nullable final BulkOperationState operationState) throws IOException { LOG.info("Move {} to {}", pathsToDelete, pathsToCreate); - Preconditions.checkNotNull(pathsToDelete, "pathsToDelete is null"); - Preconditions.checkNotNull(pathsToCreate, "pathsToCreate is null"); - Preconditions.checkArgument(pathsToDelete.size() == pathsToCreate.size(), - "Must supply same number of paths to delete/create."); + if (pathsToCreate == null) { + pathsToCreate = Collections.emptyList(); + } + if (pathsToDelete == null) { + pathsToDelete = Collections.emptyList(); + } // I feel dirty for using reentrant lock. :-| synchronized (this) { @@ -210,7 +219,7 @@ public void move(Collection pathsToDelete, // 2. Create new destination path metadata for (PathMetadata meta : pathsToCreate) { LOG.debug("move: adding metadata {}", meta); - put(meta); + put(meta, null); } // 3. We now know full contents of all dirs in destination subtree @@ -228,7 +237,13 @@ public void move(Collection pathsToDelete, } @Override - public void put(PathMetadata meta) throws IOException { + public void put(final PathMetadata meta) throws IOException { + put(meta, null); + } + + @Override + public void put(PathMetadata meta, + final BulkOperationState operationState) throws IOException { Preconditions.checkNotNull(meta); FileStatus status = meta.getFileStatus(); @@ -308,13 +323,14 @@ public synchronized void put(DirListingMetadata meta) throws IOException { } else { entry.setDirListingMetadata(meta); } - put(meta.getListing()); + put(meta.getListing(), null); } - public synchronized void put(Collection metas) throws + public synchronized void put(Collection metas, + final BulkOperationState operationState) throws IOException { for (PathMetadata meta : metas) { - put(meta); + put(meta, operationState); } } @@ -530,4 +546,11 @@ DirListingMetadata getDirListingMeta(Path p){ } } + @Override + public RenameTracker initiateRenameOperation(final StoreContext storeContext, + final Path source, + final FileStatus sourceStatus, final Path dest) throws IOException { + return new ProgressiveRenameTracker(storeContext, this, source, dest, + null); + } } 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 746fd82950b27..5d46ba9a956e0 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.s3guard; +import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; import java.util.Collection; @@ -27,9 +28,11 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.Retries.RetryTranslated; +import org.apache.hadoop.fs.s3a.impl.StoreContext; /** * {@code MetadataStore} defines the set of operations that any metadata store @@ -149,12 +152,15 @@ PathMetadata get(Path path, boolean wantEmptyDirectoryFlag) * @param pathsToDelete Collection of all paths that were removed from the * source directory tree of the move. * @param pathsToCreate Collection of all PathMetadata for the new paths - * that were created at the destination of the rename - * (). + * that were created at the destination of the rename(). + * @param operationState Any ongoing state supplied to the rename tracker + * which is to be passed in with each move operation. * @throws IOException if there is an error */ - void move(Collection pathsToDelete, - Collection pathsToCreate) throws IOException; + void move( + @Nullable Collection pathsToDelete, + @Nullable Collection pathsToCreate, + @Nullable BulkOperationState operationState) throws IOException; /** * Saves metadata for exactly one path. @@ -169,15 +175,33 @@ void move(Collection pathsToDelete, @RetryTranslated void put(PathMetadata meta) throws IOException; + /** + * Saves metadata for exactly one path, potentially + * using any bulk operation state to eliminate duplicate work. + * + * Implementations may pre-create all the path's ancestors automatically. + * Implementations must update any {@code DirListingMetadata} objects which + * track the immediate parent of this file. + * + * @param meta the metadata to save + * @param operationState operational state for a bulk update + * @throws IOException if there is an error + */ + @RetryTranslated + void put(PathMetadata meta, + @Nullable BulkOperationState operationState) throws IOException; + /** * Saves metadata for any number of paths. * * Semantics are otherwise the same as single-path puts. * * @param metas the metadata to save + * @param operationState (nullable) operational state for a bulk update * @throws IOException if there is an error */ - void put(Collection metas) throws IOException; + void put(Collection metas, + @Nullable BulkOperationState operationState) throws IOException; /** * Save directory listing metadata. Callers may save a partial directory @@ -252,4 +276,33 @@ void prune(long modTime, String keyPrefix) * @throws IOException if there is an error */ void updateParameters(Map parameters) throws IOException; + + /** + * Start a rename operation. + * + * @param storeContext store context. + * @param source source path + * @param sourceStatus status of the source file/dir + * @param dest destination path. + * @return the rename tracker + * @throws IOException Failure. + */ + RenameTracker initiateRenameOperation( + StoreContext storeContext, + Path source, + FileStatus sourceStatus, + Path dest) + throws IOException; + + /** + * Initiate a bulk update and create an operation state for it. + * This may then be passed into put operations. + * @param dest path under which updates will be explicitly put. + * @return null or a store-specific state to pass into the put operations. + * @throws IOException failure + */ + default BulkOperationState initiateBulkWrite(Path dest) throws IOException { + return null; + } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java index 04704e7ea73d7..67d07a3d93a89 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java @@ -19,8 +19,10 @@ package org.apache.hadoop.fs.s3a.s3guard; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.impl.StoreContext; import java.io.IOException; import java.util.Collection; @@ -76,15 +78,22 @@ public DirListingMetadata listChildren(Path path) throws IOException { @Override public void move(Collection pathsToDelete, - Collection pathsToCreate) throws IOException { + Collection pathsToCreate, + final BulkOperationState operationState) throws IOException { } @Override - public void put(PathMetadata meta) throws IOException { + public void put(final PathMetadata meta) throws IOException { } @Override - public void put(Collection meta) throws IOException { + public void put(PathMetadata meta, + final BulkOperationState operationState) throws IOException { + } + + @Override + public void put(Collection meta, + final BulkOperationState operationState) throws IOException { } @Override @@ -120,4 +129,33 @@ public Map getDiagnostics() throws IOException { public void updateParameters(Map parameters) throws IOException { } + + @Override + public RenameTracker initiateRenameOperation(final StoreContext storeContext, + final Path source, + final FileStatus sourceStatus, + final Path dest) + throws IOException { + return new NullRenameTracker(storeContext, source, dest, this); + } + + private static final class NullRenameTracker extends RenameTracker { + + private NullRenameTracker( + final StoreContext storeContext, + final Path source, + final Path dest, MetadataStore metadataStore) { + super("null tracker", storeContext, metadataStore, source, dest, null); + } + + @Override + public void fileCopied(final Path childSource, + final FileStatus sourceStatus, + final Path destPath, + final long blockSize, + final boolean addAncestors) throws IOException { + + } + + } } 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 c6f70bf277f44..5df54f562f0f8 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 @@ -51,7 +51,8 @@ */ @InterfaceAudience.Private @InterfaceStability.Evolving -final class PathMetadataDynamoDBTranslation { +@VisibleForTesting +public final class PathMetadataDynamoDBTranslation { /** The HASH key name of each item. */ @VisibleForTesting @@ -279,7 +280,8 @@ static KeyAttribute pathToParentKeyAttribute(Path path) { * @param path path to convert * @return string for parent key */ - static String pathToParentKey(Path path) { + @VisibleForTesting + public static String pathToParentKey(Path path) { Preconditions.checkNotNull(path); Preconditions.checkArgument(path.isUriPathAbsolute(), "Path not absolute"); URI uri = path.toUri(); 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 new file mode 100644 index 0000000000000..62542d55375c7 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathOrderComparators.java @@ -0,0 +1,133 @@ +/* + * 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.Serializable; +import java.util.Comparator; + +import org.apache.hadoop.fs.Path; + +/** + * Comparator of path ordering for sorting collections. + * + * The definition of "topmost" is: + *
    + *
  1. The depth of a path is the primary comparator.
  2. + *
  3. Root is topmost, "0"
  4. + *
  5. If two paths are of equal depth, {@link Path#compareTo(Path)}
  6. + * is used. This delegates to URI compareTo. + *
  7. repeated sorts do not change the order
  8. + *
+ */ +final class PathOrderComparators { + + private PathOrderComparators() { + } + + /** + * The shallowest paths come first. + * This is to be used when adding entries. + */ + static final Comparator TOPMOST_PATH_FIRST + = new TopmostFirst(); + + /** + * The leaves come first. + * This is to be used when deleting entries. + */ + static final Comparator TOPMOST_PATH_LAST + = new TopmostLast(); + + /** + * The shallowest paths come first. + * This is to be used when adding entries. + */ + static final Comparator TOPMOST_PM_FIRST + = new PathMetadataComparator(TOPMOST_PATH_FIRST); + + /** + * The leaves come first. + * This is to be used when deleting entries. + */ + static final Comparator TOPMOST_PM_LAST + = new PathMetadataComparator(TOPMOST_PATH_LAST); + + private static class TopmostFirst implements Comparator, Serializable { + + @Override + public int compare(Path pathL, Path pathR) { + // exist fast on equal values. + if (pathL.equals(pathR)) { + return 0; + } + int depthL = pathL.depth(); + int depthR = pathR.depth(); + if (depthL < depthR) { + // left is higher up than the right. + return -1; + } + if (depthR < depthL) { + // right is higher up than the left + return 1; + } + // and if they are of equal depth, use the "classic" comparator + // of paths. + return pathL.compareTo(pathR); + } + } + + /** + * Compare the topmost last. + * For some reason the .reverse() option wasn't giving the + * correct outcome. + */ + private static final class TopmostLast extends TopmostFirst { + + @Override + public int compare(final Path pathL, final Path pathR) { + int compare = super.compare(pathL, pathR); + if (compare < 0) { + return 1; + } + if (compare > 0) { + return -1; + } + return 0; + } + } + + /** + * Compare on path status. + */ + private static final class PathMetadataComparator implements + Comparator, Serializable { + + private final Comparator inner; + + private PathMetadataComparator(final Comparator inner) { + this.inner = inner; + } + + @Override + public int compare(final PathMetadata o1, final PathMetadata o2) { + return inner.compare(o1.getFileStatus().getPath(), + o2.getFileStatus().getPath()); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java new file mode 100644 index 0000000000000..184e85919550d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ProgressiveRenameTracker.java @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.s3guard; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.util.DurationInfo; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveAncestors; +import static org.apache.hadoop.fs.s3a.s3guard.S3Guard.addMoveDir; + +/** + * This rename tracker progressively updates the metadata store + * as it proceeds, during the parallelized copy operation. + * + * Algorithm + *
    + *
  1. + * As {@link #fileCopied(Path, FileStatus, Path, long, boolean)} callbacks + * are raised, the metastore is updated with the new file entry. + *
  2. + *
  3. + * Including parent entries, as appropriate. + *
  4. + *
  5. + * All directories which have been created are tracked locally, + * to avoid needing to read the store; this is a thread-safe structure. + *
  6. + *
  7. + * The actual update is performed out of any synchronized block. + *
  8. + *
  9. + * When deletes are executed, the store is also updated. + *
  10. + *
  11. + * And at the completion of a successful rename, the source directory + * is also removed. + *
  12. + *
+ *
+ *
+ * 
+ */ +public class ProgressiveRenameTracker extends RenameTracker { + + /** + * The collection of paths to delete; this is added as individual files + * are renamed. + * The metastore is only updated with these entries after the DELETE + * call succeeds. + */ + private final Collection pathsToDelete = new HashSet<>(); + + private final List destMetas = new ArrayList<>(); + + public ProgressiveRenameTracker( + final StoreContext storeContext, + final MetadataStore metadataStore, + final Path sourceRoot, + final Path dest, + final BulkOperationState operationState) { + super("ProgressiveRenameTracker", + storeContext, metadataStore, sourceRoot, dest, operationState); + } + + /** + * When a file is copied, any ancestors + * are calculated and then the store is updated with + * the destination entries. + * + * The source entries are added to the {@link #pathsToDelete} list. + * @param sourcePath path of source + * @param sourceStatus status of source. + * @param destPath destination path. + * @param blockSize block size. + * @param addAncestors should ancestors be added? + * @throws IOException failure + */ + @Override + public void fileCopied( + final Path sourcePath, + final FileStatus sourceStatus, + final Path destPath, + final long blockSize, + final boolean addAncestors) throws IOException { + + // build the list of entries to add in a synchronized block. + final List entriesToAdd = new ArrayList<>(1); + LOG.debug("Updating store with copied file {}", sourcePath); + MetadataStore store = getMetadataStore(); + synchronized (this) { + checkArgument(!pathsToDelete.contains(sourcePath), + "File being renamed is already processed %s", destPath); + // create the file metadata and update the local structures. + S3Guard.addMoveFile( + store, + pathsToDelete, + entriesToAdd, + sourcePath, + destPath, + sourceStatus.getLen(), + blockSize, + getOwner()); + LOG.debug("New metastore entry : {}", entriesToAdd.get(0)); + if (addAncestors) { + // add all new ancestors. + addMoveAncestors( + store, + pathsToDelete, + entriesToAdd, + getSourceRoot(), + sourcePath, + destPath, + getOwner()); + } + } + + // outside the lock, the entriesToAdd list has all new files to create. + // ...so update the store. + store.move(null, entriesToAdd, getOperationState()); + } + + /** + * A directory marker has been added. + * Add the new entry and record the source path as another entry to delete. + * @param sourceStatus status of source. + * @param destPath destination path. + * @param addAncestors should ancestors be added? + * @throws IOException failure. + */ + @Override + public void directoryMarkerCopied(final FileStatus sourceStatus, + final Path destPath, + final boolean addAncestors) throws IOException { + // this list is created on demand. + final List entriesToAdd = new ArrayList<>(1); + MetadataStore store = getMetadataStore(); + synchronized (this) { + addMoveDir(store, + pathsToDelete, + entriesToAdd, + sourceStatus.getPath(), + destPath, + getOwner()); + // Ancestor directories may not be listed, so we explicitly add them + if (addAncestors) { + addMoveAncestors(store, + pathsToDelete, + entriesToAdd, + getSourceRoot(), + sourceStatus.getPath(), + destPath, + getOwner()); + } + } + // outside the lock, the entriesToAdd list has all new files to create. + // ...so update the store. + try (DurationInfo ignored = new DurationInfo(LOG, false, + "adding %s metastore entries", entriesToAdd.size())) { + store.move(null, entriesToAdd, getOperationState()); + } + } + + @Override + public synchronized void moveSourceDirectory() throws IOException { + if (!pathsToDelete.contains(getSourceRoot())) { + addMoveDir(getMetadataStore(), pathsToDelete, destMetas, + getSourceRoot(), + getDest(), + getOwner()); + } + } + + /** + * As source objects are deleted, so is the list of entries. + * @param paths keys of objects deleted. + * @throws IOException failure. + */ + @Override + public void sourceObjectsDeleted( + final List paths) throws IOException { + + // delete the paths from the metastore + try (DurationInfo ignored = new DurationInfo(LOG, false, + "delete %s metastore entries", paths.size())) { + getMetadataStore().move(paths, null, getOperationState()); + } + } + + @Override + public void completeRename() throws IOException { + // this should all have happened. + LOG.debug("Rename completed for {}", this); + getMetadataStore().move(pathsToDelete, destMetas, getOperationState()); + super.completeRename(); + } + +} 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 new file mode 100644 index 0000000000000..fac1784d43d0a --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/RenameTracker.java @@ -0,0 +1,267 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.s3guard; + +import java.io.IOException; +import java.util.List; + +import com.amazonaws.SdkBaseException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.fs.s3a.impl.StoreOperation; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.util.DurationInfo; + +import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.hadoop.fs.s3a.S3AUtils.translateException; + +/** + * A class which manages updating the metastore with the rename process + * as initiated in the S3AFilesystem rename. + * Subclasses must provide an implementation and return it in + * {@link MetadataStore#initiateRenameOperation(StoreContext, Path, FileStatus, Path)}. + * The {@link #operationState} field/constructor argument is an opaque state to + * be passed down to the metastore in its move operations; this allows the + * stores to manage ongoing state -while still being able to share + * rename tracker implementations. + * This is to avoid performance problems wherein the progressive rename + * tracker causes the store to repeatedly create and write duplicate + * ancestor entries for every file added. + */ +public abstract class RenameTracker extends StoreOperation { + + public static final Logger LOG = LoggerFactory.getLogger( + RenameTracker.class); + + /** source path. */ + private final Path sourceRoot; + + /** destination path. */ + private final Path dest; + + /** + * Track the duration of this operation. + */ + private final DurationInfo durationInfo; + + /** + * Generated name for strings. + */ + private final String name; + + /** + * Any ongoing state supplied to the rename tracker + * which is to be passed in with each move operation. + * This must be closed at the end of the tracker's life. + */ + private final BulkOperationState operationState; + + /** + * The metadata store for this tracker. + * Always non-null. + * This is passed in separate from the store context to guarantee + * that whichever store creates a tracker is explicitly bound to that + * instance. + */ + private final MetadataStore metadataStore; + + /** + * Constructor. + * @param name tracker name for logs. + * @param storeContext store context. + * @param metadataStore the stopre + * @param sourceRoot source path. + * @param dest destination path. + * @param operationState ongoing move state. + */ + protected RenameTracker( + final String name, + final StoreContext storeContext, + final MetadataStore metadataStore, + final Path sourceRoot, + final Path dest, + final BulkOperationState operationState) { + super(checkNotNull(storeContext)); + checkNotNull(storeContext.getUsername(), "No username"); + this.metadataStore = checkNotNull(metadataStore); + this.sourceRoot = checkNotNull(sourceRoot); + this.dest = checkNotNull(dest); + this.operationState = operationState; + this.name = String.format("%s (%s, %s)", name, sourceRoot, dest); + durationInfo = new DurationInfo(LOG, false, + name +" (%s, %s)", sourceRoot, dest); + } + + @Override + public String toString() { + return name; + } + + public Path getSourceRoot() { + return sourceRoot; + } + + public Path getDest() { + return dest; + } + + public String getOwner() { + return getStoreContext().getUsername(); + } + + public BulkOperationState getOperationState() { + return operationState; + } + + /** + * Get the metadata store. + * @return a non-null store. + */ + protected MetadataStore getMetadataStore() { + return metadataStore; + } + + /** + * A file has been copied. + * + * @param childSource source of the file. This may actually be different + * from the path of the sourceStatus. + * @param sourceStatus status of source. + * @param destPath destination path. + * @param blockSize block size. + * @param addAncestors should ancestors be added? + * @throws IOException failure. + */ + public abstract void fileCopied( + Path childSource, + FileStatus sourceStatus, + Path destPath, + long blockSize, + boolean addAncestors) throws IOException; + + /** + * A directory marker has been copied. + * @param sourceStatus status of source. + * @param destPath destination path. + * @param addAncestors should ancestors be added? + * @throws IOException failure. + */ + public void directoryMarkerCopied( + FileStatus sourceStatus, + Path destPath, + boolean addAncestors) throws IOException { + + } + + /** + * The delete failed. + * By the time this is called, the metastore will already have + * been updated with the results of any partial delete failure, + * such that all files known to have been deleted will have been + * removed. + * @param e exception + * @param pathsToDelete paths which were to be deleted. + * @param undeletedObjects list of objects which were not deleted. + */ + public IOException deleteFailed( + final Exception e, + final List pathsToDelete, + final List undeletedObjects) { + + return convertToIOException(e); + } + + /** + * Top level directory move. + * @throws IOException on failure + */ + public void moveSourceDirectory() throws IOException { + + } + + /** + * Note that source objects have been deleted. + * The metastore will already have been updated. + * @param paths keys of objects deleted. + */ + public void sourceObjectsDeleted( + final List paths) throws IOException { + } + + /** + * Complete the operation. + * @throws IOException failure. + */ + public void completeRename() throws IOException { + IOUtils.cleanupWithLogger(LOG, operationState); + noteRenameFinished(); + } + + /** + * Note that the rename has finished by closing the duration info; + * this will log the duration of the operation at debug. + */ + protected void noteRenameFinished() { + durationInfo.close(); + } + + /** + * Rename has failed. + * The metastore now needs to be updated with its current state + * even though the operation is incomplete. + * Implementations MUST NOT throw exceptions here, as this is going to + * be invoked in an exception handler. + * catch and log or catch and return/wrap. + * + * The base implementation returns the IOE passed in and translates + * any AWS exception into an IOE. + * @param ex the exception which caused the failure. + * This is either an IOException or and AWS exception + * @return an IOException to throw in an exception. + */ + public IOException renameFailed(Exception ex) { + LOG.debug("Rename has failed", ex); + IOUtils.cleanupWithLogger(LOG, operationState); + noteRenameFinished(); + return convertToIOException(ex); + } + + /** + * Convert a passed in exception (expected to be an IOE or AWS exception + * into an IOException. + * @param ex exception caught + * @return the exception to throw in the failure handler. + */ + protected IOException convertToIOException(final Exception ex) { + if (ex instanceof IOException) { + return (IOException) ex; + } else if (ex instanceof SdkBaseException) { + return translateException("rename " + sourceRoot + " to " + dest, + sourceRoot.toString(), + (SdkBaseException) ex); + } else { + // should never happen, but for strictness + return new IOException(ex); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java index 8234777c3b4a2..791b58aa3669d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.s3guard; +import javax.annotation.Nullable; import java.io.FileNotFoundException; import java.io.IOException; import java.net.URI; @@ -135,7 +136,6 @@ static Class getMetadataStoreClass( return aClass; } - /** * Helper function which puts a given S3AFileStatus into the MetadataStore and * returns the same S3AFileStatus. Instrumentation monitors the put operation. @@ -149,14 +149,58 @@ static Class getMetadataStoreClass( public static S3AFileStatus putAndReturn(MetadataStore ms, S3AFileStatus status, S3AInstrumentation instrumentation) throws IOException { + return putAndReturn(ms, status, instrumentation, null); + } + + /** + * Helper function which puts a given S3AFileStatus into the MetadataStore and + * returns the same S3AFileStatus. Instrumentation monitors the put operation. + * @param ms MetadataStore to {@code put()} into. + * @param status status to store + * @param instrumentation instrumentation of the s3a file system + * @param operationState possibly-null metastore state tracker. + * @return The same status as passed in + * @throws IOException if metadata store update failed + */ + @RetryTranslated + public static S3AFileStatus putAndReturn( + final MetadataStore ms, + final S3AFileStatus status, + final S3AInstrumentation instrumentation, + @Nullable final BulkOperationState operationState) throws IOException { long startTimeNano = System.nanoTime(); - ms.put(new PathMetadata(status)); - instrumentation.addValueToQuantiles(S3GUARD_METADATASTORE_PUT_PATH_LATENCY, - (System.nanoTime() - startTimeNano)); - instrumentation.incrementCounter(S3GUARD_METADATASTORE_PUT_PATH_REQUEST, 1); + try { + ms.put(new PathMetadata(status), operationState); + } finally { + instrumentation.addValueToQuantiles( + S3GUARD_METADATASTORE_PUT_PATH_LATENCY, + (System.nanoTime() - startTimeNano)); + instrumentation.incrementCounter( + S3GUARD_METADATASTORE_PUT_PATH_REQUEST, + 1); + } return status; } + /** + * Initiate a bulk update and create an operation state for it. + * This may then be passed into put operations. + * @param metastore store + * @param path path under which updates will be explicitly put. + * @return a store-specific state to pass into the put operations, or null + * @throws IOException failure + */ + public static BulkOperationState initiateBulkWrite( + @Nullable final MetadataStore metastore, + final Path path) throws IOException { + if (metastore == null || isNullMetadataStore(metastore)) { + return null; + } else { + return metastore.initiateBulkWrite(path); + } + } + + /** * Convert the data of a directory listing to an array of {@link FileStatus} * entries. Tombstones are filtered out at this point. If the listing is null @@ -242,7 +286,7 @@ public static FileStatus[] dirListingUnion(MetadataStore ms, Path path, if (status != null && s.getModificationTime() > status.getModificationTime()) { LOG.debug("Update ms with newer metadata of: {}", status); - ms.put(new PathMetadata(s)); + ms.put(new PathMetadata(s), null); } } @@ -357,7 +401,7 @@ public static void makeDirsOrdered(MetadataStore ms, List dirs, } // Batched put - ms.put(pathMetas); + ms.put(pathMetas, null); } catch (IOException ioe) { LOG.error("MetadataStore#put() failure:", ioe); } @@ -458,8 +502,19 @@ public static void addMoveAncestors(MetadataStore ms, } } + /** + * This adds all new ancestors of a path as directories. + * @param metadataStore store + * @param qualifiedPath path to update + * @param username username to use in all new FileStatus entries. + * @param operationState (nullable) operational state for a bulk update + * @throws IOException failure + */ + @Retries.RetryTranslated public static void addAncestors(MetadataStore metadataStore, - Path qualifiedPath, String username) throws IOException { + Path qualifiedPath, + String username, + @Nullable final BulkOperationState operationState) throws IOException { Collection newDirs = new ArrayList<>(); Path parent = qualifiedPath.getParent(); while (!parent.isRoot()) { @@ -474,7 +529,9 @@ public static void addAncestors(MetadataStore metadataStore, } parent = parent.getParent(); } - metadataStore.put(newDirs); + if (!newDirs.isEmpty()) { + metadataStore.put(newDirs, operationState); + } } private static void addMoveStatus(Collection srcPaths, 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 1ac167f5a6dc6..c7700f2d065d9 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.s3guard; +import javax.annotation.Nullable; import java.io.FileNotFoundException; import java.io.IOException; import java.io.PrintStream; @@ -694,9 +695,11 @@ public String getUsage() { * Put parents into MS and cache if the parents are not presented. * * @param f the file or an empty directory. + * @param operationState store's bulk update state. * @throws IOException on I/O errors. */ - private void putParentsIfNotPresent(FileStatus f) throws IOException { + private void putParentsIfNotPresent(FileStatus f, + @Nullable BulkOperationState operationState) throws IOException { Preconditions.checkNotNull(f); Path parent = f.getPath().getParent(); while (parent != null) { @@ -705,7 +708,7 @@ private void putParentsIfNotPresent(FileStatus f) throws IOException { } FileStatus dir = DynamoDBMetadataStore.makeDirStatus(parent, f.getOwner()); - getStore().put(new PathMetadata(dir)); + getStore().put(new PathMetadata(dir), operationState); dirCache.add(parent); parent = parent.getParent(); } @@ -718,10 +721,11 @@ private void putParentsIfNotPresent(FileStatus f) throws IOException { */ private long importDir(FileStatus status) throws IOException { Preconditions.checkArgument(status.isDirectory()); + BulkOperationState operationState = getStore().initiateBulkWrite( + status.getPath()); RemoteIterator it = getFilesystem() .listFilesAndEmptyDirectories(status.getPath(), true); long items = 0; - while (it.hasNext()) { LocatedFileStatus located = it.next(); FileStatus child; @@ -736,8 +740,8 @@ private long importDir(FileStatus status) throws IOException { located.getBlockSize(), located.getOwner()); } - putParentsIfNotPresent(child); - getStore().put(new PathMetadata(child)); + putParentsIfNotPresent(child, operationState); + getStore().put(new PathMetadata(child), operationState); items++; } return items; @@ -772,7 +776,7 @@ public int run(String[] args, PrintStream out) throws Exception { long items = 1; if (status.isFile()) { PathMetadata meta = new PathMetadata(status); - getStore().put(meta); + getStore().put(meta, null); } else { items = importDir(status); } 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 bb09d576dcf95..352be731bb296 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 @@ -1215,6 +1215,18 @@ sync. See [Fail on Error](#fail-on-error) for more detail. +### Error `Attempt to change a resource which is still in use: Table is being deleted` + +``` +com.amazonaws.services.dynamodbv2.model.ResourceInUseException: + Attempt to change a resource which is still in use: Table is being deleted: + s3guard.test.testDynamoDBInitDestroy351245027 + (Service: AmazonDynamoDBv2; Status Code: 400; Error Code: ResourceInUseException;) +``` + +You have attempted to call `hadoop s3guard destroy` on a table which is already +being destroyed. + ## Other Topics For details on how to test S3Guard, see [Testing S3Guard](./testing.html#s3guard) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java index 433964998cd9a..4a050fbc11504 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRename.java @@ -18,14 +18,23 @@ package org.apache.hadoop.fs.contract.s3a; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.contract.AbstractContractRenameTest; import org.apache.hadoop.fs.contract.AbstractFSContract; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.S3ATestUtils; +import org.apache.hadoop.fs.s3a.Statistic; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; +import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents; import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset; +import static org.apache.hadoop.fs.s3a.S3ATestConstants.S3A_TEST_TIMEOUT; import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeEnableS3Guard; /** @@ -33,6 +42,15 @@ */ public class ITestS3AContractRename extends AbstractContractRenameTest { + public static final Logger LOG = LoggerFactory.getLogger( + ITestS3AContractRename.class); + + + @Override + protected int getTestTimeoutMillis() { + return S3A_TEST_TIMEOUT; + } + /** * Create a configuration, possibly patching in S3Guard options. * @return a configuration @@ -50,6 +68,12 @@ protected AbstractFSContract createContract(Configuration conf) { return new S3AContract(conf); } + @Override + public void teardown() throws Exception { + describe("\nTeardown\n"); + super.teardown(); + } + @Override public void testRenameDirIntoExistingDir() throws Throwable { describe("Verify renaming a dir into an existing dir puts the files" @@ -64,12 +88,55 @@ public void testRenameDirIntoExistingDir() throws Throwable { Path destDir = path("dest"); Path destFilePath = new Path(destDir, "dest-512.txt"); - byte[] destDateset = dataset(512, 'A', 'Z'); - writeDataset(fs, destFilePath, destDateset, destDateset.length, 1024, + byte[] destDataset = dataset(512, 'A', 'Z'); + writeDataset(fs, destFilePath, destDataset, destDataset.length, 1024, false); assertIsFile(destFilePath); boolean rename = fs.rename(srcDir, destDir); assertFalse("s3a doesn't support rename to non-empty directory", rename); } + + /** + * Test that after renaming, the nested file is moved along with all its + * ancestors. It is similar to {@link #testRenamePopulatesDirectoryAncestors}. + * + * This is an extension testRenamePopulatesFileAncestors + * of the superclass version which does better + * logging of the state of the store before the assertions. + */ + @Test + public void testRenamePopulatesFileAncestors2() throws Exception { + final S3AFileSystem fs = (S3AFileSystem)getFileSystem(); + Path base = path("testRenamePopulatesFileAncestors2"); + final Path src = new Path(base, "src"); + Path dest = new Path(base, "dest"); + fs.mkdirs(src); + final String nestedFile = "/dir1/dir2/dir3/fileA"; + // size of file to create + int filesize = 16 * 1024; + byte[] srcDataset = dataset(filesize, 'a', 'z'); + Path srcFile = path(src + nestedFile); + Path destFile = path(dest + nestedFile); + writeDataset(fs, srcFile, srcDataset, srcDataset.length, + 1024, false); + + S3ATestUtils.MetricDiff fileCopyDiff = new S3ATestUtils.MetricDiff(fs, + Statistic.FILES_COPIED); + S3ATestUtils.MetricDiff fileCopyBytes = new S3ATestUtils.MetricDiff(fs, + Statistic.FILES_COPIED_BYTES); + + fs.rename(src, dest); + + describe("Rename has completed, examining data under " + base); + fileCopyDiff.assertDiffEquals("Number of files copied", 1); + fileCopyBytes.assertDiffEquals("Number of bytes copied", filesize); + // log everything in the base directory. + S3ATestUtils.lsR(fs, base, true); + // look at the data. + verifyFileContents(fs, destFile, srcDataset); + describe("validating results"); + validateAncestorsMoved(src, dest, nestedFile); + + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java index 8f8d8605653b1..3a5786f458ae3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java @@ -20,9 +20,14 @@ import com.amazonaws.services.s3.model.DeleteObjectsRequest; import com.amazonaws.services.s3.model.MultiObjectDeleteException; +import com.google.common.collect.Lists; +import org.junit.Assume; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport; import org.junit.Test; import org.slf4j.Logger; @@ -31,13 +36,16 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.nio.file.AccessDeniedException; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.getLandsatCSVPath; +import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.failIf; +import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.*; +import static org.apache.hadoop.fs.s3a.impl.TestPartialDeleteFailures.keysToDelete; import static org.apache.hadoop.test.LambdaTestUtils.*; /** - * Test S3A Failure translation. + * ITest for failure handling, primarily multipart deletion. */ public class ITestS3AFailureHandling extends AbstractS3ATestBase { private static final Logger LOG = @@ -69,12 +77,17 @@ public void testMultiObjectDeleteNoFile() throws Throwable { private void removeKeys(S3AFileSystem fileSystem, String... keys) throws IOException { + fileSystem.removeKeys(buildDeleteRequest(keys), false); + } + + private List buildDeleteRequest( + final String[] keys) { List request = new ArrayList<>( keys.length); for (String key : keys) { request.add(new DeleteObjectsRequest.KeyVersion(key)); } - fileSystem.removeKeys(request, false, false); + return request; } @Test @@ -87,12 +100,86 @@ public void testMultiObjectDeleteSomeFiles() throws Throwable { timer.end("removeKeys"); } + + private Path maybeGetCsvPath() { + Configuration conf = getConfiguration(); + String csvFile = conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE); + Assume.assumeTrue("CSV test file is not the default", + DEFAULT_CSVTEST_FILE.equals(csvFile)); + return new Path(csvFile); + } + + /** + * Test low-level failure handling with low level delete request. + */ @Test public void testMultiObjectDeleteNoPermissions() throws Throwable { - Path testFile = getLandsatCSVPath(getConfiguration()); - S3AFileSystem fs = (S3AFileSystem)testFile.getFileSystem( + describe("Delete the landsat CSV file and expect it to fail"); + Path csvPath = maybeGetCsvPath(); + S3AFileSystem fs = (S3AFileSystem)csvPath.getFileSystem(getConfiguration()); + List keys + = buildDeleteRequest( + new String[]{ + fs.pathToKey(csvPath), + "missing-key.csv" + }); + MultiObjectDeleteException ex = intercept( + MultiObjectDeleteException.class, + () -> fs.removeKeys(keys, false)); + + final List undeleted + = extractUndeletedPaths(ex, fs::keyToQualifiedPath); + String undeletedFiles = join(undeleted); + failIf(undeleted.size() != 2, + "undeleted list size wrong: " + undeletedFiles, + ex); + assertTrue("no CSV in " +undeletedFiles, undeleted.contains(csvPath)); + + // and a full split, after adding a new key + String marker = "/marker"; + Path markerPath = fs.keyToQualifiedPath(marker); + keys.add(new DeleteObjectsRequest.KeyVersion(marker)); + + Pair, List> pair = + new MultiObjectDeleteSupport(fs.createStoreContext()) + .splitUndeletedKeys(ex, keys); + assertEquals(undeleted, pair.getLeft()); + List right = pair.getRight(); + assertEquals("Wrong size for " + join(right), 1, right.size()); + assertEquals(markerPath, right.get(0)); + } + + /** + * See what happens when you delete two entries which do not exist. + * It must not raise an exception. + */ + @Test + public void testMultiObjectDeleteMissingEntriesSucceeds() throws Throwable { + describe("Delete keys which don't exist"); + Path base = path("missing"); + S3AFileSystem fs = getFileSystem(); + List keys = keysToDelete( + Lists.newArrayList(new Path(base, "1"), new Path(base, "2"))); + fs.removeKeys(keys, false); + } + + private String join(final Iterable iterable) { + return "[" + StringUtils.join(iterable, ",") + "]"; + } + + /** + * Test low-level failure handling with a single-entry file. + * This is deleted as a single call, so isn't that useful. + */ + @Test + public void testSingleObjectDeleteNoPermissionsTranslated() throws Throwable { + describe("Delete the landsat CSV file and expect it to fail"); + Path csvPath = maybeGetCsvPath(); + S3AFileSystem fs = (S3AFileSystem) csvPath.getFileSystem( getConfiguration()); - intercept(MultiObjectDeleteException.class, - () -> removeKeys(fs, fs.pathToKey(testFile))); + AccessDeniedException aex = intercept(AccessDeniedException.class, + () -> fs.delete(csvPath, false)); + Throwable cause = aex.getCause(); + failIf(cause == null, "no nested exception", aex); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetadataPersistenceException.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetadataPersistenceException.java index 26661a36090ed..3662194d34b50 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetadataPersistenceException.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMetadataPersistenceException.java @@ -33,6 +33,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff; +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore; @@ -129,8 +130,15 @@ private IOExceptionMetadataStore(IOException ioException) { } @Override - public void put(PathMetadata meta) throws IOException { + public void put(PathMetadata meta, + final BulkOperationState operationState) throws IOException { throw ioException; } + + @Override + public void put(final PathMetadata meta) throws IOException { + put(meta, null); + } + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java index 51ff299e7be08..fbabfa138bbf8 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java @@ -39,6 +39,7 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase; +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.util.Progressable; /** @@ -177,8 +178,9 @@ public boolean exists(Path f) throws IOException { } @Override - void finishedWrite(String key, long length) { - + void finishedWrite(String key, + long length, + final BulkOperationState operationState) { } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java index 1ac52c4e3a239..72613538a42f6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java @@ -23,9 +23,7 @@ import java.io.IOException; import java.net.URI; import java.nio.file.AccessDeniedException; -import java.util.ArrayList; import java.util.List; -import java.util.stream.Collectors; import java.util.stream.IntStream; import com.amazonaws.auth.AWSCredentials; @@ -53,7 +51,6 @@ import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; -import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; @@ -524,78 +521,6 @@ public Path methodPath() throws IOException { return path(getMethodName()); } - @Test - public void testRestrictedRename() throws Throwable { - describe("rename with parent paths not writeable"); - executeRestrictedRename(createAssumedRoleConfig()); - } - - @Test - public void testRestrictedSingleDeleteRename() throws Throwable { - describe("rename with parent paths not writeable" - + " and multi-object delete disabled"); - Configuration conf = createAssumedRoleConfig(); - conf.setBoolean(ENABLE_MULTI_DELETE, false); - executeRestrictedRename(conf); - } - - /** - * Execute a sequence of rename operations with access locked down. - * @param conf FS configuration - */ - public void executeRestrictedRename(final Configuration conf) - throws IOException { - Path basePath = methodPath(); - Path restrictedDir = new Path(basePath, "renameSrc"); - Path destPath = new Path(basePath, "renameDest"); - Path child = new Path(restrictedDir, "child"); - // the full FS - S3AFileSystem fs = getFileSystem(); - fs.delete(basePath, true); - - bindRolePolicyStatements(conf, - STATEMENT_S3GUARD_CLIENT, - STATEMENT_ALLOW_SSE_KMS_RW, - STATEMENT_ALL_BUCKET_READ_ACCESS, - new Statement(Effects.Allow) - .addActions(S3_PATH_RW_OPERATIONS) - .addResources(directory(restrictedDir)) - .addResources(directory(destPath)) - ); - roleFS = (S3AFileSystem) restrictedDir.getFileSystem(conf); - - roleFS.getFileStatus(ROOT); - roleFS.mkdirs(restrictedDir); - // you can create an adjacent child - touch(roleFS, child); - - roleFS.delete(destPath, true); - // as dest doesn't exist, this will map child -> dest - assertRenameOutcome(roleFS, child, destPath, true); - - assertIsFile(destPath); - assertIsDirectory(restrictedDir); - Path renamedDestPath = new Path(restrictedDir, destPath.getName()); - assertRenameOutcome(roleFS, destPath, restrictedDir, true); - assertIsFile(renamedDestPath); - roleFS.delete(restrictedDir, true); - roleFS.delete(destPath, true); - } - - @Test - public void testRestrictedRenameReadOnlyData() throws Throwable { - describe("rename with source read only, multidelete"); - executeRenameReadOnlyData(createAssumedRoleConfig()); - } - - @Test - public void testRestrictedRenameReadOnlySingleDelete() throws Throwable { - describe("rename with source read only single delete"); - Configuration conf = createAssumedRoleConfig(); - conf.setBoolean(ENABLE_MULTI_DELETE, false); - executeRenameReadOnlyData(conf); - } - /** * Without simulation of STS failures, and with STS overload likely to * be very rare, there'll be no implicit test coverage of @@ -615,102 +540,6 @@ public void testAssumedRoleRetryHandler() throws Throwable { } } - /** - * Execute a sequence of rename operations where the source - * data is read only to the client calling rename(). - * This will cause the inner delete() operations to fail, whose outcomes - * are explored. - * Multiple files are created (in parallel) for some renames, so exploring - * the outcome on bulk delete calls, including verifying that a - * MultiObjectDeleteException is translated to an AccessDeniedException. - *
    - *
  1. The exception raised is AccessDeniedException, - * from single and multi DELETE calls.
  2. - *
  3. It happens after the COPY. Not ideal, but, well, we can't pretend - * it's a filesystem forever.
  4. - *
- * @param conf FS configuration - */ - public void executeRenameReadOnlyData(final Configuration conf) - throws Exception { - assume("Does not work with S3Guard", !getFileSystem().hasMetadataStore()); - Path basePath = methodPath(); - Path destDir = new Path(basePath, "renameDest"); - Path readOnlyDir = new Path(basePath, "readonlyDir"); - Path readOnlyFile = new Path(readOnlyDir, "readonlyChild"); - - // the full FS - S3AFileSystem fs = getFileSystem(); - fs.delete(basePath, true); - - // this file is readable by the roleFS, but cannot be deleted - touch(fs, readOnlyFile); - - bindRolePolicyStatements(conf, - STATEMENT_S3GUARD_CLIENT, - STATEMENT_ALL_BUCKET_READ_ACCESS, - new Statement(Effects.Allow) - .addActions(S3_PATH_RW_OPERATIONS) - .addResources(directory(destDir)) - ); - roleFS = (S3AFileSystem) destDir.getFileSystem(conf); - - roleFS.delete(destDir, true); - roleFS.mkdirs(destDir); - // rename will fail in the delete phase - forbidden(readOnlyFile.toString(), - () -> roleFS.rename(readOnlyFile, destDir)); - - // and the source file is still there - assertIsFile(readOnlyFile); - - // but so is the copied version, because there's no attempt - // at rollback, or preflight checking on the delete permissions - Path renamedFile = new Path(destDir, readOnlyFile.getName()); - - assertIsFile(renamedFile); - - ContractTestUtils.assertDeleted(roleFS, renamedFile, true); - assertFileCount("Empty Dest Dir", roleFS, - destDir, 0); - // create a set of files - // this is done in parallel as it is 10x faster on a long-haul test run. - int range = 10; - touchFiles(fs, readOnlyDir, range); - // don't forget about that original file! - final long createdFiles = range + 1; - // are they all there? - assertFileCount("files ready to rename", roleFS, - readOnlyDir, createdFiles); - - // try to rename the directory - LOG.info("Renaming readonly files {} to {}", readOnlyDir, destDir); - AccessDeniedException ex = forbidden("", - () -> roleFS.rename(readOnlyDir, destDir)); - LOG.info("Result of renaming read-only files is AccessDeniedException", ex); - assertFileCount("files copied to the destination", roleFS, - destDir, createdFiles); - assertFileCount("files in the source directory", roleFS, - readOnlyDir, createdFiles); - - // and finally (so as to avoid the delay of POSTing some more objects, - // delete that r/o source - forbidden("", () -> roleFS.delete(readOnlyDir, true)); - } - - /** - * Parallel-touch a set of files in the destination directory. - * @param fs filesystem - * @param destDir destination - * @param range range 1..range inclusive of files to create. - */ - public void touchFiles(final S3AFileSystem fs, - final Path destDir, - final int range) { - IntStream.rangeClosed(1, range).parallel().forEach( - (i) -> eval(() -> touch(fs, new Path(destDir, "file-" + i)))); - } - @Test public void testRestrictedCommitActions() throws Throwable { describe("Attempt commit operations against a path with restricted rights"); @@ -779,12 +608,16 @@ public void testRestrictedCommitActions() throws Throwable { // all those commits must fail List commits = pendingCommits.getLeft().getCommits(); assertEquals(range, commits.size()); - commits.parallelStream().forEach( - (c) -> { - CommitOperations.MaybeIOE maybeIOE = operations.commit(c, "origin"); - Path path = c.destinationPath(); - assertCommitAccessDenied(path, maybeIOE); - }); + try(CommitOperations.CommitContext commitContext + = operations.initiateCommitOperation(uploadDest)) { + commits.parallelStream().forEach( + (c) -> { + CommitOperations.MaybeIOE maybeIOE = + commitContext.commit(c, "origin"); + Path path = c.destinationPath(); + assertCommitAccessDenied(path, maybeIOE); + }); + } // fail of all list and abort of .pending files. LOG.info("abortAllSinglePendingCommits({})", readOnlyDir); @@ -840,24 +673,25 @@ public void writeCSVData(final File localSrc) throws IOException { @Test public void testPartialDelete() throws Throwable { describe("delete with part of the child tree read only; multidelete"); - executePartialDelete(createAssumedRoleConfig()); + executePartialDelete(createAssumedRoleConfig(), false); } @Test public void testPartialDeleteSingleDelete() throws Throwable { describe("delete with part of the child tree read only"); - Configuration conf = createAssumedRoleConfig(); - conf.setBoolean(ENABLE_MULTI_DELETE, false); - executePartialDelete(conf); + executePartialDelete(createAssumedRoleConfig(), true); } /** * Have a directory with full R/W permissions, but then remove * write access underneath, and try to delete it. * @param conf FS configuration + * @param singleDelete flag to indicate this is a single delete operation */ - public void executePartialDelete(final Configuration conf) + public void executePartialDelete(final Configuration conf, + final boolean singleDelete) throws Exception { + conf.setBoolean(ENABLE_MULTI_DELETE, !singleDelete); Path destDir = methodPath(); Path readOnlyDir = new Path(destDir, "readonlyDir"); @@ -888,25 +722,4 @@ public void executePartialDelete(final Configuration conf) roleFS.delete(pathWhichDoesntExist, true)); } - /** - * Assert that the number of files in a destination matches that expected. - * @param text text to use in the message - * @param fs filesystem - * @param path path to list (recursively) - * @param expected expected count - * @throws IOException IO problem - */ - private static void assertFileCount(String text, FileSystem fs, - Path path, long expected) - throws IOException { - List files = new ArrayList<>(); - applyLocatedFiles(fs.listFiles(path, true), - (status) -> files.add(status.getPath().toString())); - long actual = files.size(); - if (actual != expected) { - String ls = files.stream().collect(Collectors.joining("\n")); - fail(text + ": expected " + expected + " files in " + path - + " but got " + actual + "\n" + ls); - } - } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java index dbbaee5f8a9d1..8942d70daa698 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/RoleTestUtils.java @@ -18,8 +18,12 @@ package org.apache.hadoop.fs.s3a.auth; +import java.io.IOException; import java.nio.file.AccessDeniedException; +import java.util.List; import java.util.concurrent.Callable; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import com.fasterxml.jackson.core.JsonProcessingException; import org.junit.Assume; @@ -160,18 +164,35 @@ public static Configuration newAssumedRoleConfig( /** * Assert that an operation is forbidden. + * @param type of closure * @param contained contained text, may be null * @param eval closure to evaluate + * @return the access denied exception + * @throws Exception any other exception + */ + public static AccessDeniedException forbidden( + final String contained, + final Callable eval) + throws Exception { + return forbidden("", contained, eval); + } + + /** + * Assert that an operation is forbidden. * @param type of closure + * @param message error message + * @param contained contained text, may be null + * @param eval closure to evaluate * @return the access denied exception * @throws Exception any other exception */ public static AccessDeniedException forbidden( - String contained, - Callable eval) + final String message, + final String contained, + final Callable eval) throws Exception { return intercept(AccessDeniedException.class, - contained, eval); + contained, message, eval); } /** @@ -209,4 +230,23 @@ public static void assertCredentialsEqual(final String message, actual.getSessionToken()); } + + /** + * Parallel-touch a set of files in the destination directory. + * @param fs filesystem + * @param destDir destination + * @param range range 1..range inclusive of files to create. + * @return the list of paths created. + */ + public static List touchFiles(final FileSystem fs, + final Path destDir, + final int range) throws IOException { + List paths = IntStream.rangeClosed(1, range) + .mapToObj((i) -> new Path(destDir, "file-" + i)) + .collect(Collectors.toList()); + for (Path path : paths) { + touch(fs, path); + } + return paths; + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java index 2886a998b0373..56adab75808f3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java @@ -21,9 +21,11 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import com.amazonaws.services.s3.model.PartETag; +import com.google.common.collect.Lists; import org.junit.Assume; import org.junit.Test; import org.slf4j.Logger; @@ -36,6 +38,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker; import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitter; @@ -332,13 +335,21 @@ private void commit(String filename, validateIntermediateAndFinalPaths(magicFile, destFile); SinglePendingCommit commit = SinglePendingCommit.load(getFileSystem(), validatePendingCommitData(filename, magicFile)); - CommitOperations actions = newCommitOperations(); setThrottling(throttle, failures); - actions.commitOrFail(commit); + commitOrFail(destFile, commit, newCommitOperations()); resetFailures(); verifyCommitExists(commit); } + private void commitOrFail(final Path destFile, + final SinglePendingCommit commit, final CommitOperations actions) + throws IOException { + try (CommitOperations.CommitContext commitContext + = actions.initiateCommitOperation(destFile)) { + commitContext.commitOrFail(commit); + } + } + /** * Perform any validation of paths. * @param magicFilePath path to magic file @@ -439,7 +450,7 @@ public void testUploadEmptyFile() throws Throwable { resetFailures(); assertPathDoesNotExist("pending commit", dest); fullThrottle(); - actions.commitOrFail(pendingCommit); + commitOrFail(dest, pendingCommit, actions); resetFailures(); FileStatus status = verifyPathExists(fs, "uploaded file commit", dest); @@ -462,7 +473,7 @@ public void testUploadSmallFile() throws Throwable { resetFailures(); assertPathDoesNotExist("pending commit", dest); fullThrottle(); - actions.commitOrFail(pendingCommit); + commitOrFail(dest, pendingCommit, actions); resetFailures(); String s = readUTF8(fs, dest, -1); assertEquals(text, s); @@ -544,4 +555,80 @@ public void testWriteNormalStream() throws Throwable { assertTrue("Empty marker file: " + status, status.getLen() > 0); } + /** + * Creates a bulk commit and commits multiple files. + * If the DDB metastore is in use, use the instrumentation to + * verify that the write count is as expected. + * This is done without actually looking into the store -just monitoring + * changes in the filesystem's instrumentation counters. + */ + @Test + public void testBulkCommitFiles() throws Throwable { + describe("verify bulk commit including metastore update count"); + File localFile = File.createTempFile("commit", ".txt"); + CommitOperations actions = newCommitOperations(); + Path destDir = methodPath("testBulkCommitFiles"); + S3AFileSystem fs = getFileSystem(); + fs.delete(destDir, false); + fullThrottle(); + + Path destFile1 = new Path(destDir, "file1"); + // this subdir will only be created in the commit of file 2 + Path subdir = new Path(destDir, "subdir"); + // file 2 + Path destFile2 = new Path(subdir, "file2"); + Path destFile3 = new Path(subdir, "file3"); + List destinations = Lists.newArrayList(destFile1, destFile2, + destFile3); + List commits = new ArrayList<>(3); + + for (Path destination : destinations) { + SinglePendingCommit commit1 = + actions.uploadFileToPendingCommit(localFile, + destination, null, + DEFAULT_MULTIPART_SIZE); + commits.add(commit1); + } + resetFailures(); + assertPathDoesNotExist("destination dir", destDir); + assertPathDoesNotExist("subdirectory", subdir); + // how many records have been written + try (CommitOperations.CommitContext commitContext + = actions.initiateCommitOperation(destDir)) { + MetricDiff writes = new MetricDiff(fs, + Statistic.S3GUARD_METADATASTORE_RECORD_WRITES); + commitContext.commitOrFail(commits.get(0)); + long writesOnFirstCommit = writes.diff(); + writes.reset(); + assertPathExists("destFile1", destFile1); + assertPathExists("destination dir", destDir); + + commitContext.commitOrFail(commits.get(1)); + assertPathExists("subdirectory", subdir); + assertPathExists("destFile2", destFile2); + if (writesOnFirstCommit != 0) { + // S3Guard is in use against DDB, so the metrics can be checked + // to see how many records were updated. + // there should only be two new entries: one for the file and + // one for the parent. + writes.assertDiffEquals( + "Number of records written after second commit; " + + "first commit had " + writesOnFirstCommit, + 2); + } + writes.reset(); + commitContext.commitOrFail(commits.get(2)); + assertPathExists("destFile3", destFile3); + if (writesOnFirstCommit != 0) { + // this file is in the same dir as destFile2, so only its entry + // is added + writes.assertDiffEquals( + "Number of records written after third commit; " + + "first commit had " + writesOnFirstCommit, + 1); + } + } + resetFailures(); + } + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitMRJob.java index a9b9c2cbe1e1d..f3c4174577235 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitMRJob.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitMRJob.java @@ -18,17 +18,20 @@ package org.apache.hadoop.fs.s3a.commit.magic; +import java.io.FileNotFoundException; import java.io.IOException; import org.junit.AfterClass; import org.junit.BeforeClass; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.commit.AbstractITCommitMRJob; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; import org.apache.hadoop.mapred.JobConf; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Full integration test for the Magic Committer. @@ -93,6 +96,13 @@ protected void applyCustomConfigOptions(JobConf conf) { @Override protected void customPostExecutionValidation(Path destPath, SuccessData successData) throws Exception { - assertPathDoesNotExist("No cleanup", new Path(destPath, MAGIC)); + Path magicDir = new Path(destPath, MAGIC); + + // if an FNFE isn't raised on getFileStatus, list out the directory + // contents + intercept(FileNotFoundException.class, () -> { + getFileSystem().getFileStatus(magicDir); + return ContractTestUtils.ls(getFileSystem(), magicDir); + }); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java index 072295962ce36..9a2ad0ee9da10 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java @@ -143,9 +143,12 @@ public void test_030_postCreationAssertions() throws Throwable { assertNotNull("jobDir", jobDir); Pair>> results = operations.loadSinglePendingCommits(jobDir, false); - for (SinglePendingCommit singlePendingCommit : - results.getKey().getCommits()) { - operations.commitOrFail(singlePendingCommit); + try(CommitOperations.CommitContext commitContext + = operations.initiateCommitOperation(jobDir)) { + for (SinglePendingCommit singlePendingCommit : + results.getKey().getCommits()) { + commitContext.commitOrFail(singlePendingCommit); + } } timer.end("time to commit %s", pendingDataFile); // upload is no longer pending 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 new file mode 100644 index 0000000000000..bec8e1fa054b9 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java @@ -0,0 +1,807 @@ +/* + * 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.impl; + +import java.io.IOException; +import java.nio.file.AccessDeniedException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import com.amazonaws.services.s3.model.MultiObjectDeleteException; +import com.google.common.base.Charsets; +import com.google.common.util.concurrent.ListeningExecutorService; +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +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.FileSystem; +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.S3AUtils; +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; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.*; +import static org.apache.hadoop.fs.s3a.Constants.*; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; +import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles; +import static org.apache.hadoop.fs.s3a.Statistic.FILES_DELETE_REJECTED; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUESTS; +import static org.apache.hadoop.fs.s3a.auth.RoleModel.Effects; +import static org.apache.hadoop.fs.s3a.auth.RoleModel.Statement; +import static org.apache.hadoop.fs.s3a.auth.RoleModel.directory; +import static org.apache.hadoop.fs.s3a.auth.RoleModel.statement; +import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*; +import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.bindRolePolicyStatements; +import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden; +import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.newAssumedRoleConfig; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion; +import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.extractUndeletedPaths; +import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.removeUndeletedPaths; +import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.assertFileCount; +import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.extractCause; +import static org.apache.hadoop.test.LambdaTestUtils.eval; + +/** + * Test partial failures of delete and rename operations, especially + * that the S3Guard tables are consistent with the state of + * the filesystem. + * + * All these test have a unique path for each run, with a roleFS having + * full RW access to part of it, and R/O access to a restricted subdirectory + * + * The tests are parameterized to single/multi delete, which control which + * of the two delete mechanisms are used. + * In multi delete, in a scale test run, a significantly larger set of files + * is created and then deleted. + * This isn't done in the single delete as it is much slower and it is not + * the situation we are trying to create. + * + * This test manages to create lots of load on the s3guard prune command + * when that is tested; too many tombstone files for the test to complete. + * An attempt is made in teardown to prune the test files. + */ +@SuppressWarnings("ThrowableNotThrown") +@RunWith(Parameterized.class) +public class ITestPartialRenamesDeletes extends AbstractS3ATestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestPartialRenamesDeletes.class); + + private static final Path ROOT = new Path("/"); + + private static final Statement STATEMENT_ALL_BUCKET_READ_ACCESS + = statement(true, S3_ALL_BUCKETS, S3_BUCKET_READ_OPERATIONS); + + /** Many threads for scale performance: {@value}. */ + public static final int EXECUTOR_THREAD_COUNT = 64; + + /** + * For submitting work. + */ + private static final ListeningExecutorService EXECUTOR = + BlockingThreadPoolExecutorService.newInstance( + EXECUTOR_THREAD_COUNT, + EXECUTOR_THREAD_COUNT * 2, + 30, TimeUnit.SECONDS, + "test-operations"); + + public static final int FILE_COUNT_NON_SCALED = 2; + + /** + * The number of files for a scaled test. This is still + * less than half the amount which can be fitted into a delete + * request, so that even with this many R/W and R/O files, + * both can fit in the same request. + * Then, when a partial delete occurs, we can make assertions + * knowing that all R/W files should have been deleted and all + * R/O files rejected. + */ + public static final int FILE_COUNT_SCALED = 10; + + public static final int DIR_COUNT = 2; + public static final int DIR_COUNT_SCALED = 4; + public static final int DEPTH = 2; + public static final int DEPTH_SCALED = 2; + + /** + * A role FS; if non-null it is closed in teardown. + */ + private S3AFileSystem roleFS; + + /** + * Base path for this test run. + */ + private Path basePath; + + private Path writableDir; + + private Path readOnlyChild; + + private Path readOnlyDir; + + private Path noReadDir; + + /** delete policy: single or multi? */ + private final boolean multiDelete; + + /** + * Configuration for the assume role FS. + */ + private Configuration assumedRoleConfig; + + private int fileCount; + private int dirCount; + private int dirDepth; + + private boolean scaleTest; + + /** + * Test array for parameterized test runs. + *
    + *
  • Run 0: single deletes
  • + *
  • Run 1: multi deletes
  • + *
+ * + * @return a list of parameter tuples. + */ + @Parameterized.Parameters(name = "bulk delete={0}") + public static Collection params() { + return Arrays.asList(new Object[][]{ + {false}, + {true}, + }); + } + + /** + * Constructor. + * @param multiDelete single vs multi delete in the role FS? + */ + public ITestPartialRenamesDeletes(final boolean multiDelete) { + this.multiDelete = multiDelete; + } + + /** + * This sets up a unique path for every test run, so as to guarantee isolation + * from previous runs. + * It creates a role policy which has read access to everything except + * the contents of {@link #noReadDir}, and with write access to + * {@link #writableDir}. + * @throws Exception failure + */ + @Override + public void setup() throws Exception { + super.setup(); + assumeRoleTests(); + basePath = uniquePath(); + readOnlyDir = new Path(basePath, "readonlyDir"); + writableDir = new Path(basePath, "writableDir"); + readOnlyChild = new Path(readOnlyDir, "child"); + noReadDir = new Path(basePath, "noReadDir"); + // the full FS + S3AFileSystem fs = getFileSystem(); + fs.delete(basePath, true); + fs.mkdirs(writableDir); + + // create the baseline assumed role + assumedRoleConfig = createAssumedRoleConfig(); + bindRolePolicyStatements(assumedRoleConfig, + STATEMENT_S3GUARD_CLIENT, + STATEMENT_ALL_BUCKET_READ_ACCESS, // root: r-x + new Statement(Effects.Allow) // dest: rwx + .addActions(S3_PATH_RW_OPERATIONS) + .addResources(directory(writableDir)), + new Statement(Effects.Deny) // noReadDir: --x + .addActions(S3_ALL_GET) + .addActions(S3_ALL_PUT) + .addActions(S3_ALL_DELETE) + .addResources(directory(noReadDir))); + // the role configured to that set of restrictions + roleFS = (S3AFileSystem) readOnlyDir.getFileSystem(assumedRoleConfig); + + // switch to the big set of files iff this is a multidelete run + // with -Dscale set. + // without that the DELETE calls become a key part of the bottleneck + scaleTest = multiDelete && getTestPropertyBool( + getConfiguration(), + KEY_SCALE_TESTS_ENABLED, + DEFAULT_SCALE_TESTS_ENABLED); + fileCount = scaleTest ? FILE_COUNT_SCALED : FILE_COUNT_NON_SCALED; + dirCount = scaleTest ? DIR_COUNT_SCALED : DIR_COUNT; + dirDepth = scaleTest ? DEPTH_SCALED : DEPTH; + } + + @Override + public void teardown() throws Exception { + S3AUtils.closeAll(LOG, roleFS); + 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()); + } + + private String getAssumedRoleARN() { + return getContract().getConf().getTrimmed(ASSUMED_ROLE_ARN, ""); + } + + /** + * Create the assumed role configuration. + * @return a config bonded to the ARN of the assumed role + */ + public Configuration createAssumedRoleConfig() { + return createAssumedRoleConfig(getAssumedRoleARN()); + } + + /** + * Create a config for an assumed role; it also disables FS caching + * and sets the multi delete option to that of the current mode. + * @param roleARN ARN of role + * @return the new configuration + */ + private Configuration createAssumedRoleConfig(String roleARN) { + Configuration conf = newAssumedRoleConfig(getContract().getConf(), + roleARN); + String bucketName = getTestBucketName(conf); + + removeBucketOverrides(bucketName, conf, ENABLE_MULTI_DELETE); + conf.setBoolean(ENABLE_MULTI_DELETE, multiDelete); + return conf; + } + + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + String bucketName = getTestBucketName(conf); + + // ramp up the number of connections we can have for maximum PUT + // performance + removeBucketOverrides(bucketName, conf, + MAX_THREADS, + MAXIMUM_CONNECTIONS, + S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY); + conf.setInt(MAX_THREADS, EXECUTOR_THREAD_COUNT); + conf.setInt(MAXIMUM_CONNECTIONS, EXECUTOR_THREAD_COUNT * 2); + // turn off prune delays, so as to stop scale tests creating + // so much cruft that future CLI prune commands take forever + conf.setInt(S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY, 0); + return conf; + } + + /** + * Create a unique path, which includes method name, + * multidelete flag and a random UUID. + * @return a string to use for paths. + * @throws IOException path creation failure. + */ + private Path uniquePath() throws IOException { + return path( + String.format("%s-%s-%04d", + getMethodName(), + multiDelete ? "multi" : "single", + System.currentTimeMillis() % 10000)); + } + + /** + * This is here to verify role and path setup. + */ + @Test + public void testCannotTouchUnderRODir() throws Throwable { + forbidden("touching the empty child " + readOnlyChild, + "", + () -> { + touch(roleFS, readOnlyChild); + return readOnlyChild; + }); + } + @Test + public void testCannotReadUnderNoReadDir() throws Throwable { + Path path = new Path(noReadDir, "unreadable.txt"); + createFile(getFileSystem(), path, true, "readonly".getBytes()); + forbidden("trying to read " + path, + "", + () -> readUTF8(roleFS, path, -1)); + } + + @Test + public void testMultiDeleteOptionPropagated() throws Throwable { + describe("Verify the test parameter propagates to the store context"); + StoreContext ctx = roleFS.createStoreContext(); + Assertions.assertThat(ctx.isMultiObjectDeleteEnabled()) + .as(ctx.toString()) + .isEqualTo(multiDelete); + } + + /** + * Execute a sequence of rename operations with access locked down. + */ + @Test + public void testRenameParentPathNotWriteable() throws Throwable { + describe("rename with parent paths not writeable; multi=%s", multiDelete); + final Configuration conf = createAssumedRoleConfig(); + bindRolePolicyStatements(conf, + STATEMENT_S3GUARD_CLIENT, + STATEMENT_ALLOW_SSE_KMS_RW, + STATEMENT_ALL_BUCKET_READ_ACCESS, + new Statement(Effects.Allow) + .addActions(S3_PATH_RW_OPERATIONS) + .addResources(directory(readOnlyDir)) + .addResources(directory(writableDir))); + roleFS = (S3AFileSystem) readOnlyDir.getFileSystem(conf); + + S3AFileSystem fs = getFileSystem(); + roleFS.getFileStatus(ROOT); + fs.mkdirs(readOnlyDir); + // you can create an adjacent child + touch(fs, readOnlyChild); + + fs.delete(writableDir, true); + // as dest doesn't exist, this will map child -> dest + assertRenameOutcome(roleFS, readOnlyChild, writableDir, true); + + assertIsFile(writableDir); + assertIsDirectory(readOnlyDir); + Path renamedDestPath = new Path(readOnlyDir, writableDir.getName()); + assertRenameOutcome(roleFS, writableDir, readOnlyDir, true); + assertIsFile(renamedDestPath); + roleFS.delete(readOnlyDir, true); + roleFS.delete(writableDir, true); + } + + @Test + public void testRenameSingleFileFailsInDelete() throws Throwable { + describe("rename with source read only; multi=%s", multiDelete); + Path readOnlyFile = readOnlyChild; + + // the full FS + S3AFileSystem fs = getFileSystem(); + fs.delete(basePath, true); + + // this file is readable by the roleFS, but cannot be deleted + touch(fs, readOnlyFile); + + roleFS.delete(writableDir, true); + roleFS.mkdirs(writableDir); + // rename will fail in the delete phase + expectRenameForbidden(readOnlyFile, writableDir); + + // and the source file is still there + assertIsFile(readOnlyFile); + + // and so is the copied version, because there's no attempt + // at rollback, or preflight checking on the delete permissions + Path renamedFile = new Path(writableDir, readOnlyFile.getName()); + + assertIsFile(renamedFile); + + ContractTestUtils.assertDeleted(roleFS, renamedFile, true); + assertFileCount("Empty Dest Dir", roleFS, + writableDir, 0); + } + + /** + * Execute a sequence of rename operations where the source + * data is read only to the client calling rename(). + * This will cause the inner delete() operations to fail, whose outcomes + * are explored. + * Multiple files are created (in parallel) for some renames, so the test + * explores the outcome on bulk delete calls, including verifying that a + * MultiObjectDeleteException is translated to an AccessDeniedException. + *
    + *
  1. The exception raised is AccessDeniedException, + * from single and multi DELETE calls.
  2. + *
  3. It happens after the COPY. Not ideal, but, well, we can't pretend + * it's a filesystem forever.
  4. + *
+ */ + @Test + public void testRenameDirFailsInDelete() throws Throwable { + describe("rename with source read only; multi=%s", multiDelete); + + // the full FS + S3AFileSystem fs = getFileSystem(); + + roleFS.mkdirs(writableDir); + + // create a set of files + // this is done in parallel as it is 10x faster on a long-haul test run. + List createdFiles = createFiles(fs, readOnlyDir, dirDepth, fileCount, + dirCount); + // are they all there? + int expectedFileCount = createdFiles.size(); + assertFileCount("files ready to rename", roleFS, + readOnlyDir, expectedFileCount); + + // try to rename the directory + LOG.info("Renaming readonly files {} to {}", readOnlyDir, writableDir); + + AccessDeniedException deniedException = expectRenameForbidden(readOnlyDir, + writableDir); + if (multiDelete) { + // look in that exception for a multidelete + MultiObjectDeleteException mde = extractCause( + MultiObjectDeleteException.class, deniedException); + final List undeleted + = extractUndeletedPaths(mde, fs::keyToQualifiedPath); + Assertions.assertThat(undeleted) + .as("files which could not be deleted") + .hasSize(expectedFileCount) + .containsAll(createdFiles) + .containsExactlyInAnyOrderElementsOf(createdFiles); + } + LOG.info("Result of renaming read-only files is as expected", + deniedException); + assertFileCount("files in the source directory", roleFS, + readOnlyDir, expectedFileCount); + // now lets look at the destination. + // even with S3Guard on, we expect the destination to match that of our + // the remote state. + // the test will exist + describe("Verify destination directory exists"); + FileStatus st = roleFS.getFileStatus(writableDir); + assertTrue("Not a directory: " + st, + st.isDirectory()); + assertFileCount("files in the dest directory", roleFS, + writableDir, expectedFileCount); + } + + @Test + public void testRenameFileFailsNoWrite() throws Throwable { + describe("Try to rename to a write-only destination fails with src" + + " & dest unchanged."); + roleFS.mkdirs(writableDir); + S3AFileSystem fs = getFileSystem(); + Path source = new Path(writableDir, "source"); + touch(fs, source); + fs.mkdirs(readOnlyDir); + Path dest = new Path(readOnlyDir, "dest"); + describe("Renaming files {} to {}", writableDir, dest); + // rename fails but doesn't raise an exception. Good or bad? + expectRenameForbidden(source, dest); + assertIsFile(source); + assertPathDoesNotExist("rename destination", dest); + } + + @Test + public void testCopyDirFailsToReadOnlyDir() throws Throwable { + describe("Try to copy to a read-only destination"); + roleFS.mkdirs(writableDir); + S3AFileSystem fs = getFileSystem(); + List files = createFiles(fs, writableDir, dirDepth, fileCount, + dirCount); + + fs.mkdirs(readOnlyDir); + Path dest = new Path(readOnlyDir, "dest"); + expectRenameForbidden(writableDir, dest); + assertPathDoesNotExist("rename destination", dest); + assertFileCount("files in the source directory", roleFS, + writableDir, files.size()); + } + + @Test + public void testCopyFileFailsOnSourceRead() throws Throwable { + describe("The source file isn't readable, so the COPY fails"); + Path source = new Path(noReadDir, "source"); + S3AFileSystem fs = getFileSystem(); + touch(fs, source); + fs.mkdirs(writableDir); + Path dest = new Path(writableDir, "dest"); + expectRenameForbidden(source, dest); + assertIsFile(source); + assertPathDoesNotExist("rename destination", dest); + } + + @Test + public void testCopyDirFailsOnSourceRead() throws Throwable { + describe("The source file isn't readable, so the COPY fails"); + S3AFileSystem fs = getFileSystem(); + List files = createFiles(fs, noReadDir, dirDepth, fileCount, + dirCount); + fs.mkdirs(writableDir); + Path dest = new Path(writableDir, "dest"); + expectRenameForbidden(noReadDir, dest); + assertFileCount("files in the source directory", fs, + noReadDir, files.size()); + } + + /** + * Have a directory with full R/W permissions, but then remove + * write access underneath, and try to delete it. + */ + @Test + public void testPartialDirDelete() throws Throwable { + describe("delete with part of the child tree read only;" + + " multidelete=%s", multiDelete); + + // the full FS + S3AFileSystem fs = getFileSystem(); + + List readOnlyFiles = createFiles(fs, readOnlyDir, + dirDepth, fileCount, dirCount); + List deletableFiles = createFiles(fs, + writableDir, dirDepth, fileCount, dirCount); + + // as a safety check, verify that one of the deletable files can be deleted + Path head = deletableFiles.remove(0); + assertTrue("delete " + head + " failed", + roleFS.delete(head, false)); + List allFiles = Stream.concat( + readOnlyFiles.stream(), + deletableFiles.stream()) + .collect(Collectors.toList()); + + // this set can be deleted by the role FS + MetricDiff rejectionCount = new MetricDiff(roleFS, FILES_DELETE_REJECTED); + MetricDiff deleteVerbCount = new MetricDiff(roleFS, OBJECT_DELETE_REQUESTS); + + describe("Trying to delete read only directory"); + AccessDeniedException ex = expectDeleteForbidden(readOnlyDir); + if (multiDelete) { + // multi-delete status checks + extractCause(MultiObjectDeleteException.class, ex); + rejectionCount.assertDiffEquals("Wrong rejection count", + readOnlyFiles.size()); + deleteVerbCount.assertDiffEquals("Wrong delete count", 1); + reset(rejectionCount, deleteVerbCount); + } + // all the files are still there? (avoid in scale test due to cost) + if (!scaleTest) { + readOnlyFiles.forEach(this::pathMustExist); + } + + describe("Trying to delete upper-level directory"); + ex = expectDeleteForbidden(basePath); + if (multiDelete) { + // multi-delete status checks + extractCause(MultiObjectDeleteException.class, ex); + deleteVerbCount.assertDiffEquals("Wrong delete count", 1); + MultiObjectDeleteException mde = extractCause( + MultiObjectDeleteException.class, ex); + final List undeleted + = removeUndeletedPaths(mde, allFiles, fs::keyToQualifiedPath); + Assertions.assertThat(undeleted) + .as("files which could not be deleted") + .containsExactlyInAnyOrderElementsOf(readOnlyFiles); + Assertions.assertThat(allFiles) + .as("files which were deleted") + .containsExactlyInAnyOrderElementsOf(deletableFiles); + rejectionCount.assertDiffEquals("Wrong rejection count", + readOnlyFiles.size()); + } + reset(rejectionCount, deleteVerbCount); + + // build the set of all paths under the directory tree through + // a directory listing (i.e. not getFileStatus()). + // small risk of observed inconsistency here on unguarded stores. + final Set readOnlyListing = listFilesUnderPath(readOnlyDir, true); + + String directoryList = readOnlyListing.stream() + .map(Path::toString) + .collect(Collectors.joining(", ", "[", "]")); + + Assertions.assertThat(readOnlyListing) + .as("ReadOnly directory " + directoryList) + .containsAll(readOnlyFiles); + } + + /** + * Expect the delete() call to fail. + * @param path path to delete. + * @return the expected exception. + * @throws Exception any other failure. + */ + private AccessDeniedException expectDeleteForbidden(Path path) + throws Exception { + try(DurationInfo ignored = + new DurationInfo(LOG, true, "delete %s", path)) { + return forbidden("Expected an error deleting " + path, + "", + () -> { + boolean r = roleFS.delete(path, true); + return " delete=" + r + " " + ls(path.getParent()); + }); + } + } + + /** + * Expect that a rename will fail with an exception using the roleFS. + * @param src source + * @param dest dest + * @return the exception raised. + * @throws Exception any other failure + */ + private AccessDeniedException expectRenameForbidden(Path src, Path dest) + throws Exception { + try(DurationInfo ignored = + new DurationInfo(LOG, true, + "rename(%s, %s)", src, dest)) { + return forbidden( + "Renaming " + src + " to " + dest, + "", + () -> { + boolean result = roleFS.rename(src, dest); + LOG.error("Rename should have been forbidden but returned {}", + result); + LOG.error("Source directory:\n{}", + ContractTestUtils.ls(getFileSystem(), src.getParent())); + LOG.error("Destination directory:\n{}", + ContractTestUtils.ls(getFileSystem(), src.getParent())); + return "Rename unexpectedly returned " + result; + }); + } + } + + /** + * Assert that a path must exist, map IOEs to RTEs for loops. + * @param p path. + */ + 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(System.currentTimeMillis(), + PathMetadataDynamoDBTranslation.pathToParentKey(fs.qualify(path))); + } + } + } + + /** + * List all files under a path. + * @param path path to list + * @param recursive recursive listing? + * @return an unordered set of the paths. + * @throws IOException failure + */ + private Set listFilesUnderPath(Path path, boolean recursive) + throws IOException { + Set files = new TreeSet<>(); + try (DurationInfo ignore = + new DurationInfo(LOG, "ls -R %s", path)) { + applyLocatedFiles(getFileSystem().listFiles(path, recursive), + (status) -> files.add(status.getPath())); + } + return files; + } + + /** + * Write the text to a file asynchronously. Logs the operation duration. + * @param fs filesystem + * @param path path + * @return future to the patch created. + */ + private static CompletableFuture put(FileSystem fs, + Path path, String text) { + return submit(EXECUTOR, () -> { + try (DurationInfo ignore = + new DurationInfo(LOG, false, "Creating %s", path)) { + createFile(fs, path, true, text.getBytes(Charsets.UTF_8)); + return path; + } + }); + } + + /** + * Parallel-touch a set of files in the destination directory. + * @param fs filesystem + * @param destDir destination + * @param depth file depth + * @param fileCount number of files to create. + * @param dirCount number of dirs to create at each level + * @return the list of files created. + */ + public static List createFiles(final FileSystem fs, + final Path destDir, + final int depth, + final int fileCount, + final int dirCount) throws IOException { + List> futures = new ArrayList<>(fileCount); + List paths = new ArrayList<>(fileCount); + List dirs = new ArrayList<>(fileCount); + buildPaths(paths, dirs, destDir, depth, fileCount, dirCount); + try(DurationInfo ignore = + new DurationInfo(LOG, "Creating %d files", fileCount)) { + for (Path path : paths) { + futures.add(put(fs, path, path.getName())); + } + waitForCompletion(futures); + return paths; + } + } + + /** + * Recursive method to build up lists of files and directories. + * @param filePaths list of file paths to add entries to. + * @param dirPaths list of directory paths to add entries to. + * @param destDir destination directory. + * @param depth depth of directories + * @param fileCount number of files. + * @param dirCount number of directories. + */ + private static void buildPaths( + final List filePaths, + final List dirPaths, + final Path destDir, + final int depth, + final int fileCount, + final int dirCount) { + if (depth<=0) { + return; + } + // create the file paths + for (int i = 0; i < fileCount; i++) { + String name = "file-" + i; + Path p = new Path(destDir, name); + filePaths.add(p); + } + for (int i = 0; i < dirCount; i++) { + String name = "dir-" + i; + Path p = new Path(destDir, name); + dirPaths.add(p); + buildPaths(filePaths, dirPaths, p, depth - 1, fileCount, dirCount); + } + + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java new file mode 100644 index 0000000000000..ad7fd2cc3c459 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestPartialDeleteFailures.java @@ -0,0 +1,350 @@ +/* + * 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.impl; + +import javax.annotation.Nullable; +import java.io.IOException; +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.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.amazonaws.services.s3.model.MultiObjectDeleteException; +import com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; +import org.junit.Before; +import org.junit.Test; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.lang3.tuple.Triple; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.Constants; +import org.apache.hadoop.fs.s3a.Invoker; +import org.apache.hadoop.fs.s3a.S3AInputPolicy; +import org.apache.hadoop.fs.s3a.S3AInstrumentation; +import org.apache.hadoop.fs.s3a.S3AStorageStatistics; +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; +import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata; +import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; +import org.apache.hadoop.fs.s3a.s3guard.PathMetadata; +import org.apache.hadoop.fs.s3a.s3guard.RenameTracker; +import org.apache.hadoop.io.retry.RetryPolicies; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.BlockingThreadPoolExecutorService; + +import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.ACCESS_DENIED; +import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.removeUndeletedPaths; +import static org.junit.Assert.assertEquals; + +/** + * Unit test suite covering translation of AWS SDK exceptions to S3A exceptions, + * and retry/recovery policies. + */ +public class TestPartialDeleteFailures { + + private StoreContext context; + + private static Path qualify(String k) { + return new Path("s3a://bucket/" + k); + } + + @Before + public void setUp() throws Exception { + context = craateMockStoreContext(true, + new OperationTrackingStore()); + } + + @Test + public void testDeleteExtraction() { + List src = pathList("a", "a/b", "a/c"); + List rejected = pathList("a/b"); + MultiObjectDeleteException ex = createDeleteException(ACCESS_DENIED, + rejected); + List undeleted = removeUndeletedPaths(ex, src, + TestPartialDeleteFailures::qualify); + assertEquals(rejected, undeleted); + } + + @Test + public void testSplitKeysFromResults() throws Throwable { + List src = pathList("a", "a/b", "a/c"); + List rejected = pathList("a/b"); + List keys = keysToDelete(src); + MultiObjectDeleteException ex = createDeleteException(ACCESS_DENIED, + rejected); + Pair, List> pair = + new MultiObjectDeleteSupport(context) + .splitUndeletedKeys(ex, keys); + List undeleted = pair.getLeft(); + List deleted = pair.getRight(); + assertEquals(rejected, undeleted); + // now check the deleted list to verify that it is valid + src.remove(rejected.get(0)); + assertEquals(src, deleted); + } + + /** + * Build a list of qualified paths from vararg parameters. + * @param paths paths to qualify and then convert to a lst. + * @return same paths as a list. + */ + private List pathList(String... paths) { + return Arrays.stream(paths) + .map(TestPartialDeleteFailures::qualify) + .collect(Collectors.toList()); + } + + /** + * Build a delete exception containing all the rejected paths. + * The list of successful entries is empty. + * @param rejected the rejected paths. + * @return a new exception + */ + private MultiObjectDeleteException createDeleteException( + final String code, + final List rejected) { + List errors = rejected.stream() + .map((p) -> { + MultiObjectDeleteException.DeleteError e + = new MultiObjectDeleteException.DeleteError(); + e.setKey(p.toUri().getPath()); + e.setCode(code); + e.setMessage("forbidden"); + return e; + }).collect(Collectors.toList()); + return new MultiObjectDeleteException(errors, Collections.emptyList()); + } + + /** + * From a list of paths, build up the list of keys for a delete request. + * @param paths path list + * @return a key list suitable for a delete request. + */ + public static List keysToDelete( + List paths) { + return paths.stream() + .map((p) -> p.toUri().getPath()) + .map(DeleteObjectsRequest.KeyVersion::new) + .collect(Collectors.toList()); + } + + /** + * Verify that on a partial delete, the S3Guard tables are updated + * with deleted items. And only them. + */ + @Test + public void testProcessDeleteFailure() throws Throwable { + Path pathA = qualify("/a"); + Path pathAB = qualify("/a/b"); + Path pathAC = qualify("/a/c"); + List src = Lists.newArrayList(pathA, pathAB, pathAC); + List keyList = keysToDelete(src); + List deleteForbidden = Lists.newArrayList(pathAB); + final List deleteAllowed = Lists.newArrayList(pathA, pathAC); + MultiObjectDeleteException ex = createDeleteException(ACCESS_DENIED, + deleteForbidden); + OperationTrackingStore store + = new OperationTrackingStore(); + StoreContext storeContext = craateMockStoreContext(true, store); + MultiObjectDeleteSupport deleteSupport + = new MultiObjectDeleteSupport(storeContext); + Triple, List, List>> + triple = deleteSupport.processDeleteFailure(ex, keyList); + Assertions.assertThat(triple.getRight()) + .as("failure list") + .isEmpty(); + List undeleted = triple.getLeft(); + List deleted = triple.getMiddle(); + Assertions.assertThat(deleted). + as("deleted files") + .containsAll(deleteAllowed) + .doesNotContainAnyElementsOf(deleteForbidden); + Assertions.assertThat(undeleted). + as("undeleted store entries") + .containsAll(deleteForbidden) + .doesNotContainAnyElementsOf(deleteAllowed); + } + + + StoreContext craateMockStoreContext(boolean multiDelete, + OperationTrackingStore store) throws URISyntaxException, IOException { + URI name = new URI("s3a://bucket"); + return new StoreContext( + name, + "bucket", + new Configuration(), + "alice", + UserGroupInformation.getCurrentUser(), + BlockingThreadPoolExecutorService.newInstance( + 4, + 4, + 10, TimeUnit.SECONDS, + "s3a-transfer-shared"), + Constants.DEFAULT_EXECUTOR_CAPACITY, + new Invoker(RetryPolicies.TRY_ONCE_THEN_FAIL, Invoker.LOG_EVENT), + new S3AInstrumentation(name), + new S3AStorageStatistics(), + S3AInputPolicy.Normal, + ChangeDetectionPolicy.createPolicy(ChangeDetectionPolicy.Mode.None, + ChangeDetectionPolicy.Source.ETag, false), + multiDelete, + store, + TestPartialDeleteFailures::qualify, + "us-west", + false, + false, + null); + } + + /** + * MetadataStore which tracks what is deleted and added. + */ + private static class OperationTrackingStore implements MetadataStore { + + private final List deleted = new ArrayList<>(); + + private final List created = new ArrayList<>(); + + @Override + public void initialize(final FileSystem fs) { + } + + @Override + public void initialize(final Configuration conf) { + } + + @Override + public void delete(final Path path) { + deleted.add(path); + } + + @Override + public void forgetMetadata(final Path path) { + + } + + @Override + public void deleteSubtree(final Path path) { + + } + + @Override + public PathMetadata get(final Path path) { + return null; + } + + @Override + public PathMetadata get(final Path path, + final boolean wantEmptyDirectoryFlag) { + return null; + } + + @Override + public DirListingMetadata listChildren(final Path path) { + return null; + } + + @Override + public void move( + @Nullable final Collection pathsToDelete, + @Nullable final Collection pathsToCreate, + @Nullable final BulkOperationState moveState) { + + } + + @Override + public void put(final PathMetadata meta) throws IOException { + put(meta, null); + } + + @Override + public void put(final PathMetadata meta, + final BulkOperationState operationState) { + created.add(meta.getFileStatus().getPath()); + } + + @Override + public void put(final Collection metas, + final BulkOperationState operationState) { + metas.stream().forEach(meta -> put(meta, null)); + } + + @Override + public void put(final DirListingMetadata meta) { + created.add(meta.getPath()); + } + + @Override + public void destroy() { + + } + + @Override + public void prune(final long modTime) + throws IOException, UnsupportedOperationException { + + } + + @Override + public void prune(final long modTime, final String keyPrefix) + throws IOException, UnsupportedOperationException { + } + + @Override + public Map getDiagnostics() { + return null; + } + + @Override + public void updateParameters(final Map parameters) { + } + + @Override + public void close() { + } + + public List getDeleted() { + return deleted; + } + + public List getCreated() { + return created; + } + + @Override + public RenameTracker initiateRenameOperation( + final StoreContext storeContext, + final Path source, + final FileStatus sourceStatus, + final Path dest) throws IOException { + throw new UnsupportedOperationException("unsupported"); + } + } + +} 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 ad4691a6d962b..e263c99d4f0c4 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 @@ -59,7 +59,6 @@ import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_NAME_KEY; import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_METASTORE_NULL; import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestDynamoTablePrefix; import static org.apache.hadoop.fs.s3a.S3AUtils.clearBucketOption; import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.E_BAD_STATE; import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.SUCCESS; @@ -81,6 +80,16 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase { private MetadataStore ms; private S3AFileSystem rawFs; + /** + * 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; + } + protected static void expectResult(int expected, String message, S3GuardTool tool, @@ -188,19 +197,21 @@ protected void mkdirs(Path path, boolean onS3, boolean onMetadataStore) fs.mkdirs(path); } else if (onMetadataStore) { S3AFileStatus status = new S3AFileStatus(true, path, OWNER); - ms.put(new PathMetadata(status)); + ms.put(new PathMetadata(status), null); } } protected static void putFile(MetadataStore ms, S3AFileStatus f) throws IOException { assertNotNull(f); - ms.put(new PathMetadata(f)); - Path parent = f.getPath().getParent(); - while (parent != null) { - S3AFileStatus dir = new S3AFileStatus(false, parent, f.getOwner()); - ms.put(new PathMetadata(dir)); - parent = parent.getParent(); + try(BulkOperationState bulkWrite = ms.initiateBulkWrite(f.getPath())) { + ms.put(new PathMetadata(f), bulkWrite); + Path parent = f.getPath().getParent(); + while (parent != null) { + S3AFileStatus dir = new S3AFileStatus(false, parent, f.getOwner()); + ms.put(new PathMetadata(dir), bulkWrite); + parent = parent.getParent(); + } } } 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 158f13d3fc763..d2176d941ca20 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 @@ -36,16 +36,17 @@ import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription; import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException; import com.amazonaws.services.dynamodbv2.model.TableDescription; - import com.amazonaws.services.dynamodbv2.model.Tag; import com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; + import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.contract.s3a.S3AContract; import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.Tristate; - import org.apache.hadoop.io.IOUtils; + import org.junit.AfterClass; import org.junit.Assert; import org.junit.Assume; @@ -62,6 +63,7 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.security.UserGroupInformation; +import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.s3guard.PathMetadataDynamoDBTranslation.*; @@ -150,22 +152,24 @@ public static void beforeClassSetup() throws IOException { // be configured to use this test. testDynamoDBTableName = conf.get(S3GUARD_DDB_TEST_TABLE_NAME_KEY); String dynamoDbTableName = conf.getTrimmed(S3GUARD_DDB_TABLE_NAME_KEY); - Assume.assumeTrue("No DynamoDB table name configured", !StringUtils - .isEmpty(dynamoDbTableName)); + Assume.assumeTrue("No DynamoDB table name configured in " + + S3GUARD_DDB_TABLE_NAME_KEY, + !StringUtils.isEmpty(dynamoDbTableName)); // We should assert that the table name is configured, so the test should // fail if it's not configured. - assertTrue("Test DynamoDB table name '" + assertNotNull("Test DynamoDB table name '" + S3GUARD_DDB_TEST_TABLE_NAME_KEY + "' should be set to run " - + "integration tests.", testDynamoDBTableName != null); + + "integration tests.", testDynamoDBTableName); // We should assert that the test table is not the same as the production // table, as the test table could be modified and destroyed multiple // times during the test. - assertTrue("Test DynamoDB table name: '" + assertNotEquals("Test DynamoDB table name: '" + S3GUARD_DDB_TEST_TABLE_NAME_KEY + "' and production table name: '" + S3GUARD_DDB_TABLE_NAME_KEY + "' can not be the same.", - !conf.get(S3GUARD_DDB_TABLE_NAME_KEY).equals(testDynamoDBTableName)); + testDynamoDBTableName, + conf.get(S3GUARD_DDB_TABLE_NAME_KEY)); // We can use that table in the test if these assertions are valid conf.set(S3GUARD_DDB_TABLE_NAME_KEY, testDynamoDBTableName); @@ -357,8 +361,8 @@ private void doTestBatchWrite(int numDelete, int numPut, final Path newDir = new Path(root, "newDir"); LOG.info("doTestBatchWrite: oldDir={}, newDir={}", oldDir, newDir); - ms.put(new PathMetadata(basicFileStatus(oldDir, 0, true))); - ms.put(new PathMetadata(basicFileStatus(newDir, 0, true))); + ms.put(new PathMetadata(basicFileStatus(oldDir, 0, true)), null); + ms.put(new PathMetadata(basicFileStatus(newDir, 0, true)), null); final List oldMetas = numDelete < 0 ? null : new ArrayList<>(numDelete); @@ -388,11 +392,22 @@ private void doTestBatchWrite(int numDelete, int numPut, } // move the old paths to new paths and verify - ms.move(pathsToDelete, newMetas); - assertEquals(0, ms.listChildren(oldDir).withoutTombstones().numEntries()); + AncestorState state = checkNotNull(ms.initiateBulkWrite(newDir), + "No state from initiateBulkWrite()"); + assertEquals(newDir, state.getDest()); + + ms.move(pathsToDelete, newMetas, state); + assertEquals("Number of children in source directory", + 0, ms.listChildren(oldDir).withoutTombstones().numEntries()); if (newMetas != null) { - assertTrue(CollectionUtils - .isEqualCollection(newMetas, ms.listChildren(newDir).getListing())); + Assertions.assertThat(ms.listChildren(newDir).getListing()) + .describedAs("Directory listing") + .containsAll(newMetas); + if (!newMetas.isEmpty()) { + Assertions.assertThat(state.size()) + .describedAs("Size of ancestor state") + .isGreaterThan(newMetas.size()); + } } } @@ -511,7 +526,7 @@ public void testRootDirectory() throws IOException { ddbms.put(new PathMetadata(new S3AFileStatus(true, new Path(rootPath, "foo"), - UserGroupInformation.getCurrentUser().getShortUserName()))); + UserGroupInformation.getCurrentUser().getShortUserName())), null); verifyRootDirectory(ddbms.get(rootPath), false); } @@ -561,10 +576,13 @@ public void testMovePopulatesAncestors() throws IOException { final String srcRoot = testRoot + "/a/b/src"; final String destRoot = testRoot + "/c/d/e/dest"; + AncestorState bulkWrite = ddbms.initiateBulkWrite(null); final Path nestedPath1 = strToPath(srcRoot + "/file1.txt"); - ddbms.put(new PathMetadata(basicFileStatus(nestedPath1, 1024, false))); + ddbms.put(new PathMetadata(basicFileStatus(nestedPath1, 1024, false)), + bulkWrite); final Path nestedPath2 = strToPath(srcRoot + "/dir1/dir2"); - ddbms.put(new PathMetadata(basicFileStatus(nestedPath2, 0, true))); + ddbms.put(new PathMetadata(basicFileStatus(nestedPath2, 0, true)), + bulkWrite); // We don't put the destRoot path here, since put() would create ancestor // entries, and we want to ensure that move() does it, instead. @@ -574,8 +592,7 @@ public void testMovePopulatesAncestors() throws IOException { strToPath(srcRoot), strToPath(srcRoot + "/dir1"), strToPath(srcRoot + "/dir1/dir2"), - strToPath(srcRoot + "/file1.txt") - ); + strToPath(srcRoot + "/file1.txt")); final Collection pathsToCreate = Lists.newArrayList( new PathMetadata(basicFileStatus(strToPath(destRoot), 0, true)), @@ -587,8 +604,9 @@ public void testMovePopulatesAncestors() throws IOException { 1024, false)) ); - ddbms.move(fullSourcePaths, pathsToCreate); + ddbms.move(fullSourcePaths, pathsToCreate, bulkWrite); + bulkWrite.close(); // assert that all the ancestors should have been populated automatically assertCached(testRoot + "/c"); assertCached(testRoot + "/c/d"); 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 408c72eb8f521..615eadcc3e290 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 @@ -238,7 +238,7 @@ public void test_030_BatchedWrite() throws Exception { toCleanup.add(pm); } - ddbms.put(pm); + ddbms.put(pm, null); pruneItems++; @@ -274,7 +274,7 @@ public void test_040_get() throws Throwable { Path path = new Path("s3a://example.org/get"); S3AFileStatus status = new S3AFileStatus(true, path, "alice"); PathMetadata metadata = new PathMetadata(status); - ddbms.put(metadata); + ddbms.put(metadata, null); try { execute("get", OPERATIONS_PER_THREAD, @@ -318,7 +318,7 @@ public void test_060_list() throws Throwable { Path path = new Path("s3a://example.org/list"); S3AFileStatus status = new S3AFileStatus(true, path, "alice"); PathMetadata metadata = new PathMetadata(status); - ddbms.put(metadata); + ddbms.put(metadata, null); try { Path parent = path.getParent(); execute("list", @@ -337,7 +337,7 @@ public void test_070_putDirMarker() throws Throwable { Path path = new Path("s3a://example.org/putDirMarker"); S3AFileStatus status = new S3AFileStatus(true, path, "alice"); PathMetadata metadata = new PathMetadata(status); - ddbms.put(metadata); + ddbms.put(metadata, null); DirListingMetadata children = ddbms.listChildren(path.getParent()); try { execute("list", @@ -356,12 +356,14 @@ public void test_080_fullPathsToPut() throws Throwable { Path base = new Path("s3a://example.org/test_080_fullPathsToPut"); Path child = new Path(base, "child"); List pms = new ArrayList<>(); - ddbms.put(new PathMetadata(makeDirStatus(base))); - ddbms.put(new PathMetadata(makeDirStatus(child))); + BulkOperationState bulkUpdate + = ddbms.initiateBulkWrite(child); + ddbms.put(new PathMetadata(makeDirStatus(base)), bulkUpdate); + ddbms.put(new PathMetadata(makeDirStatus(child)), bulkUpdate); ddbms.getInvoker().retry("set up directory tree", base.toString(), true, - () -> ddbms.put(pms)); + () -> ddbms.put(pms, bulkUpdate)); try { DDBPathMetadata dirData = ddbms.get(child, true); execute("list", 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 98c1e998ed0aa..98936954e4dae 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 @@ -29,6 +29,7 @@ import com.amazonaws.services.dynamodbv2.document.DynamoDB; import com.amazonaws.services.dynamodbv2.document.Table; import com.amazonaws.services.dynamodbv2.model.ListTagsOfResourceRequest; +import com.amazonaws.services.dynamodbv2.model.ResourceInUseException; import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException; import com.amazonaws.services.dynamodbv2.model.Tag; import org.junit.Assert; @@ -270,7 +271,9 @@ public void testDynamoDBInitDestroyCycle() throws Throwable { try { table.delete(); table.waitForDelete(); - } catch (ResourceNotFoundException e) { /* Ignore */ } + } catch (ResourceNotFoundException | ResourceInUseException e) { + /* Ignore */ + } } } } 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 799c5a046bc22..7e49874ea54cd 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 @@ -43,7 +43,6 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.test.HadoopTestBase; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.isMetadataStoreAuthoritative; import static org.apache.hadoop.fs.s3a.S3ATestUtils.metadataStorePersistsAuthoritativeBit; /** @@ -151,7 +150,7 @@ private void doTestDescendantsIterator( final FileStatus status = pathStr.contains("file") ? basicFileStatus(strToPath(pathStr), 100, false) : basicFileStatus(strToPath(pathStr), 0, true); - ms.put(new PathMetadata(status)); + ms.put(new PathMetadata(status), null); } final PathMetadata rootMeta = new PathMetadata(makeDirStatus("/")); @@ -238,7 +237,7 @@ public void testPutNew() throws Exception { * containing directory. We only track direct children of the directory. * Thus this will not affect entry for /da1. */ - ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc1", 100))); + ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc1", 100)), null); assertEmptyDirs("/da2", "/da3"); assertDirectorySize("/da1/db1", 1); @@ -250,7 +249,7 @@ public void testPutNew() throws Exception { } /* This already exists, and should silently replace it. */ - ms.put(new PathMetadata(makeDirStatus("/da1/db1"))); + ms.put(new PathMetadata(makeDirStatus("/da1/db1")), null); /* If we had putNew(), and used it above, this would be empty again. */ assertDirectorySize("/da1", 1); @@ -258,8 +257,8 @@ public void testPutNew() throws Exception { assertEmptyDirs("/da2", "/da3"); /* Ensure new files update correct parent dirs. */ - ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc1", 100))); - ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc2", 200))); + ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc1", 100)), null); + ms.put(new PathMetadata(makeFileStatus("/da1/db1/fc2", 200)), null); assertDirectorySize("/da1", 1); assertDirectorySize("/da1/db1", 2); assertEmptyDirs("/da2", "/da3"); @@ -274,14 +273,15 @@ public void testPutNew() throws Exception { public void testPutOverwrite() throws Exception { final String filePath = "/a1/b1/c1/some_file"; final String dirPath = "/a1/b1/c1/d1"; - ms.put(new PathMetadata(makeFileStatus(filePath, 100))); - ms.put(new PathMetadata(makeDirStatus(dirPath))); + ms.put(new PathMetadata(makeFileStatus(filePath, 100)), null); + ms.put(new PathMetadata(makeDirStatus(dirPath)), null); PathMetadata meta = ms.get(strToPath(filePath)); if (!allowMissing() || meta != null) { verifyFileStatus(meta.getFileStatus(), 100); } - ms.put(new PathMetadata(basicFileStatus(strToPath(filePath), 9999, false))); + ms.put(new PathMetadata(basicFileStatus(strToPath(filePath), 9999, false)), + null); meta = ms.get(strToPath(filePath)); if (!allowMissing() || meta != null) { verifyFileStatus(meta.getFileStatus(), 9999); @@ -292,7 +292,7 @@ public void testPutOverwrite() throws Exception { public void testRootDirPutNew() throws Exception { Path rootPath = strToPath("/"); - ms.put(new PathMetadata(makeFileStatus("/file1", 100))); + ms.put(new PathMetadata(makeFileStatus("/file1", 100)), null); DirListingMetadata dir = ms.listChildren(rootPath); if (!allowMissing() || dir != null) { assertNotNull("Root dir cached", dir); @@ -332,7 +332,7 @@ private void deleteSubtreeHelper(String pathPrefix) throws Exception { setUpDeleteTest(p); createNewDirs(p + "/ADirectory1/db1/dc1", p + "/ADirectory1/db1/dc1/dd1"); ms.put(new PathMetadata( - makeFileStatus(p + "/ADirectory1/db1/dc1/dd1/deepFile", 100))); + makeFileStatus(p + "/ADirectory1/db1/dc1/dd1/deepFile", 100)), null); if (!allowMissing()) { assertCached(p + "/ADirectory1/db1"); } @@ -382,9 +382,9 @@ private void setUpDeleteTest(String prefix) throws IOException { createNewDirs(prefix + "/ADirectory1", prefix + "/ADirectory2", prefix + "/ADirectory1/db1"); ms.put(new PathMetadata(makeFileStatus(prefix + "/ADirectory1/db1/file1", - 100))); + 100)), null); ms.put(new PathMetadata(makeFileStatus(prefix + "/ADirectory1/db1/file2", - 100))); + 100)), null); PathMetadata meta = ms.get(strToPath(prefix + "/ADirectory1/db1/file2")); if (!allowMissing() || meta != null) { @@ -397,8 +397,8 @@ private void setUpDeleteTest(String prefix) throws IOException { public void testGet() throws Exception { final String filePath = "/a1/b1/c1/some_file"; final String dirPath = "/a1/b1/c1/d1"; - ms.put(new PathMetadata(makeFileStatus(filePath, 100))); - ms.put(new PathMetadata(makeDirStatus(dirPath))); + ms.put(new PathMetadata(makeFileStatus(filePath, 100)), null); + ms.put(new PathMetadata(makeDirStatus(dirPath)), null); PathMetadata meta = ms.get(strToPath(filePath)); if (!allowMissing() || meta != null) { assertNotNull("Get found file", meta); @@ -584,7 +584,7 @@ public void testMove() throws Exception { destMetas.add(new PathMetadata(makeDirStatus("/b1"))); destMetas.add(new PathMetadata(makeFileStatus("/b1/file1", 100))); destMetas.add(new PathMetadata(makeFileStatus("/b1/file2", 100))); - ms.move(srcPaths, destMetas); + ms.move(srcPaths, destMetas, null); // Assert src is no longer there dirMeta = ms.listChildren(strToPath("/a1")); @@ -628,7 +628,7 @@ public void testMultiBucketPaths() throws Exception { assertNull("Path2 should not be present yet.", meta); // Put p1, assert p2 doesn't match - ms.put(new PathMetadata(makeFileStatus(p1, 100))); + ms.put(new PathMetadata(makeFileStatus(p1, 100)), null); meta = ms.get(new Path(p2)); assertNull("Path 2 should not match path 1.", meta); @@ -648,7 +648,7 @@ public void testPruneFiles() throws Exception { long oldTime = getTime(); ms.put(new PathMetadata(makeFileStatus("/pruneFiles/old", 1, oldTime, - oldTime))); + oldTime)), null); DirListingMetadata ls2 = ms.listChildren(strToPath("/pruneFiles")); if (!allowMissing()) { assertListingsEqual(ls2.getListing(), "/pruneFiles/old"); @@ -660,7 +660,7 @@ public void testPruneFiles() throws Exception { long cutoff = System.currentTimeMillis(); long newTime = getTime(); ms.put(new PathMetadata(makeFileStatus("/pruneFiles/new", 1, newTime, - newTime))); + newTime)), null); DirListingMetadata ls; ls = ms.listChildren(strToPath("/pruneFiles")); @@ -691,7 +691,7 @@ public void testPruneDirs() throws Exception { long oldTime = getTime(); ms.put(new PathMetadata(makeFileStatus("/pruneDirs/dir/file", - 1, oldTime, oldTime))); + 1, oldTime, oldTime)), null); // It's possible for the Local implementation to get from the old // modification time to here in under 1ms, causing it to not get pruned @@ -716,10 +716,10 @@ public void testPruneUnsetsAuthoritative() throws Exception { long time = System.currentTimeMillis(); ms.put(new PathMetadata( new FileStatus(0, false, 0, 0, time - 1, strToPath(staleFile)), - Tristate.FALSE, false)); + Tristate.FALSE, false), null); ms.put(new PathMetadata( new FileStatus(0, false, 0, 0, time + 1, strToPath(freshFile)), - Tristate.FALSE, false)); + Tristate.FALSE, false), null); // set parent dir as authoritative if (!allowMissing()) { @@ -753,10 +753,10 @@ public void testPrunePreservesAuthoritative() throws Exception { long time = System.currentTimeMillis(); ms.put(new PathMetadata( new FileStatus(0, false, 0, 0, time + 1, strToPath(staleFile)), - Tristate.FALSE, false)); + Tristate.FALSE, false), null); ms.put(new PathMetadata( new FileStatus(0, false, 0, 0, time + 1, strToPath(freshFile)), - Tristate.FALSE, false)); + Tristate.FALSE, false), null); if (!allowMissing()) { // set parent dir as authoritative @@ -814,7 +814,7 @@ public void testPutRetainsIsDeletedInParentListing() throws Exception { final FileStatus fileStatus = basicFileStatus(path, 0, false); PathMetadata pm = new PathMetadata(fileStatus); pm.setIsDeleted(true); - ms.put(pm); + ms.put(pm, null); if(!allowMissing()) { final PathMetadata pathMetadata = ms.listChildren(path.getParent()).get(path); @@ -854,8 +854,8 @@ private void commonTestPutListStatus(final String parent) throws IOException { private void setupListStatus() throws IOException { createNewDirs("/a1", "/a2", "/a1/b1", "/a1/b2", "/a1/b1/c1", "/a1/b1/c1/d1"); - ms.put(new PathMetadata(makeFileStatus("/a1/b1/file1", 100))); - ms.put(new PathMetadata(makeFileStatus("/a1/b1/file2", 100))); + ms.put(new PathMetadata(makeFileStatus("/a1/b1/file1", 100)), null); + ms.put(new PathMetadata(makeFileStatus("/a1/b1/file2", 100)), null); } private void assertListingsEqual(Collection listing, @@ -886,7 +886,7 @@ private void putListStatusFiles(String dirPath, boolean authoritative, private void createNewDirs(String... dirs) throws IOException { for (String pathStr : dirs) { - ms.put(new PathMetadata(makeDirStatus(pathStr))); + ms.put(new PathMetadata(makeDirStatus(pathStr)), null); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathOrderComparators.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathOrderComparators.java new file mode 100644 index 0000000000000..9a3db1ab21a5d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathOrderComparators.java @@ -0,0 +1,197 @@ +/* + * 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.Comparator; +import java.util.List; + +import org.junit.Test; + +import org.apache.hadoop.fs.Path; + +import static com.google.common.collect.Lists.newArrayList; +import static org.apache.hadoop.fs.s3a.s3guard.PathOrderComparators.TOPMOST_PATH_FIRST; +import static org.apache.hadoop.fs.s3a.s3guard.PathOrderComparators.TOPMOST_PATH_LAST; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; + +/** + * Test ordering of paths with the comparator matches requirements. + */ +public class TestPathOrderComparators { + + private static final Path ROOT = new Path("s3a://bucket/"); + + public static final Path DIR_A = new Path(ROOT, "dirA"); + + public static final Path DIR_B = new Path(ROOT, "dirB"); + + public static final Path DIR_A_FILE_1 = new Path(DIR_A, "file1"); + + public static final Path DIR_A_FILE_2 = new Path(DIR_A, "file2"); + + public static final Path DIR_B_FILE_3 = new Path(DIR_B, "file3"); + + public static final Path DIR_B_FILE_4 = new Path(DIR_B, "file4"); + + @Test + public void testRootEqual() throws Throwable { + assertComparesEqual(ROOT, ROOT); + } + + @Test + public void testRootFirst() throws Throwable { + assertComparesTopmost(ROOT, DIR_A_FILE_1); + } + + @Test + public void testDirOrdering() throws Throwable { + assertComparesTopmost(DIR_A, DIR_B); + } + + @Test + public void testFilesEqual() throws Throwable { + assertComparesEqual(DIR_A_FILE_1, DIR_A_FILE_1); + } + + @Test + public void testFilesInSameDir() throws Throwable { + assertComparesTopmost(ROOT, DIR_A_FILE_1); + assertComparesTopmost(DIR_A, DIR_A_FILE_1); + assertComparesTopmost(DIR_A, DIR_A_FILE_2); + assertComparesTopmost(DIR_A_FILE_1, DIR_A_FILE_2); + } + + @Test + public void testReversedFiles() throws Throwable { + assertReverseOrder(DIR_A_FILE_1, ROOT); + assertReverseOrder(DIR_A_FILE_1, DIR_A); + assertReverseOrder(DIR_A_FILE_2, DIR_A); + assertReverseOrder(DIR_A_FILE_2, DIR_A_FILE_1); + } + + @Test + public void testFilesAndDifferentShallowDir() throws Throwable { + assertComparesTopmost(DIR_B, DIR_A_FILE_1); + assertComparesTopmost(DIR_A, DIR_B_FILE_3); + } + + @Test + public void testOrderRoot() throws Throwable { + verifySorted(ROOT); + } + + @Test + public void testOrderRootDirs() throws Throwable { + verifySorted(ROOT, DIR_A, DIR_B); + } + + @Test + public void testOrderRootDirsAndFiles() throws Throwable { + verifySorted(ROOT, DIR_A, DIR_B, DIR_A_FILE_1, DIR_A_FILE_2); + } + + @Test + public void testOrderRootDirsAndAllFiles() throws Throwable { + verifySorted(ROOT, DIR_A, DIR_B, + DIR_A_FILE_1, DIR_A_FILE_2, + DIR_B_FILE_3, DIR_B_FILE_4); + } + + @Test + public void testSortOrderConstant() throws Throwable { + List sort1 = verifySorted(ROOT, DIR_A, DIR_B, + DIR_A_FILE_1, DIR_A_FILE_2, + DIR_B_FILE_3, DIR_B_FILE_4); + List sort2 = newArrayList(sort1); + assertSortsTo(sort2, sort1, true); + } + + @Test + public void testSortReverse() throws Throwable { + List sort1 = newArrayList( + ROOT, + DIR_A, + DIR_B, + DIR_A_FILE_1, + DIR_A_FILE_2, + DIR_B_FILE_3, + DIR_B_FILE_4); + List expected = newArrayList( + DIR_B_FILE_4, + DIR_B_FILE_3, + DIR_A_FILE_2, + DIR_A_FILE_1, + DIR_B, + DIR_A, + ROOT); + assertSortsTo(expected, sort1, false); + } + + + private List verifySorted(Path... paths) { + List original = newArrayList(paths); + List sorted = newArrayList(paths); + assertSortsTo(original, sorted, true); + return sorted; + } + + private void assertSortsTo( + final List original, + final List sorted, + boolean topmost) { + sorted.sort(topmost ? TOPMOST_PATH_FIRST : TOPMOST_PATH_LAST); + assertThat(sorted) + .as("Sorted paths") + .containsExactlyElementsOf(original); + } + + private void assertComparesEqual(Path l, Path r) { + assertOrder(0, l, r); + } + + private void assertComparesTopmost(Path l, Path r) { + assertOrder(-1, l, r); + assertOrder(1, r, l); + } + + private void assertReverseOrder(Path l, Path r) { + assertComparesTo(-1, TOPMOST_PATH_LAST, l, r); + assertComparesTo(1, TOPMOST_PATH_LAST, r, l); + } + + private void assertOrder(int res, + Path l, Path r) { + assertComparesTo(res, TOPMOST_PATH_FIRST, l, r); + } + + private void assertComparesTo(final int expected, + final Comparator comparator, + final Path l, final Path r) { + int actual = comparator.compare(l, r); + if (actual < -1) { + actual = -1; + } + if (actual > 1) { + actual = 1; + } + assertEquals("Comparing " + l + " to " + r, + expected, actual); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestProgressiveRenameTracker.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestProgressiveRenameTracker.java new file mode 100644 index 0000000000000..ff681a616f899 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestProgressiveRenameTracker.java @@ -0,0 +1,25 @@ +/* + * 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; + +/** + * Unit tests for {@link ProgressiveRenameTracker}. + */ +public class TestProgressiveRenameTracker { +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java index 0e6a1d8d09245..57c543ffc9905 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractITestS3AMetadataStoreScale.java @@ -20,6 +20,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.fs.s3a.s3guard.PathMetadata; @@ -129,7 +130,7 @@ public void test_020_Moves() throws Throwable { toDelete = movedPaths; toCreate = origMetas; } - ms.move(toDelete, toCreate); + ms.move(toDelete, toCreate, null); } moveTimer.end(); printTiming(LOG, "move", moveTimer, operations); @@ -180,9 +181,11 @@ private long populateMetadataStore(Collection paths, long count = 0; NanoTimer putTimer = new NanoTimer(); describe("Inserting into MetadataStore"); - for (PathMetadata p : paths) { - ms.put(p); - count++; + try(BulkOperationState operationState = ms.initiateBulkWrite(BUCKET_ROOT)) { + for (PathMetadata p : paths) { + ms.put(p, operationState); + count++; + } } putTimer.end(); printTiming(LOG, "put", putTimer, count); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.java new file mode 100644 index 0000000000000..2b93d72face4e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/ExtraAssertions.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.test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +import org.junit.Assert; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.util.DurationInfo; + +import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles; +import static org.junit.Assert.assertTrue; + +/** + * Some extra assertions for tests. + */ +@InterfaceAudience.Private +public final class ExtraAssertions { + + private static final Logger LOG = LoggerFactory.getLogger( + ExtraAssertions.class); + + private ExtraAssertions() { + } + + /** + * Assert that the number of files in a destination matches that expected. + * @param message text to use in the message + * @param fs filesystem + * @param path path to list (recursively) + * @param expected expected count + * @throws IOException IO problem + */ + public static void assertFileCount(final String message, + final FileSystem fs, + final Path path, + final long expected) + throws IOException { + List files = new ArrayList<>(); + try (DurationInfo ignored = new DurationInfo(LOG, false, + "Counting files in %s", path)) { + applyLocatedFiles(fs.listFiles(path, true), + (status) -> files.add(status.getPath().toString())); + } + long actual = files.size(); + if (actual != expected) { + String ls = files.stream().collect(Collectors.joining("\n")); + Assert.fail(message + ": expected " + expected + " files in " + path + + " but got " + actual + "\n" + ls); + } + } + + /** + * Assert that a string contains a piece of text. + * @param text text to can. + * @param contained text to look for. + */ + public static void assertTextContains(String text, String contained) { + assertTrue("string \"" + contained + "\" not found in \"" + text + "\"", + text != null && text.contains(contained)); + } + + /** + * If the condition is met, throw an AssertionError with the message + * and any nested exception. + * @param condition condition + * @param message text to use in the exception + * @param cause a (possibly null) throwable to init the cause with + * @throws AssertionError with the text and throwable if condition == true. + */ + public static void failIf(boolean condition, + String message, + Throwable cause) { + if (condition) { + ContractTestUtils.fail(message, cause); + } + } + + /** + * If the condition is met, throw an AssertionError with the message + * and any nested exception. + * @param condition condition + * @param message text to use in the exception + * @param cause a (possibly null) throwable to init the cause with + * @throws AssertionError with the text and throwable if condition == true. + */ + public static void failUnless(boolean condition, + String message, + Throwable cause) { + failIf(!condition, message, cause); + } + + /** + * Extract the inner cause of an exception. + * @param expected expected class of the cause + * @param thrown thrown exception. + * @param type of the cause + * @return the extracted exception. + * @throws AssertionError with the text and throwable if the cause is not as + * expected + */ + public static T extractCause(Class expected, + Throwable thrown) { + Throwable cause = thrown.getCause(); + failIf(cause == null, + "No inner cause", + thrown); + failUnless(cause.getClass().equals(expected), + "Inner cause is of wrong type : expected " + expected, + thrown); + return (T)cause; + } +}