From a0c71dcc33ca7c5539d0ab61c4a276c4f39e5744 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 15 Jan 2018 11:33:32 +0000 Subject: [PATCH] HADOOP-15079. ITestS3AFileOperationCost#testFakeDirectoryDeletion failing after OutputCommitter patch. Contributed by Steve Loughran --- .../org/apache/hadoop/fs/s3a/Listing.java | 7 + .../org/apache/hadoop/fs/s3a/Retries.java | 21 ++ .../apache/hadoop/fs/s3a/S3AFileSystem.java | 193 +++++++++--------- .../apache/hadoop/fs/s3a/s3guard/S3Guard.java | 12 +- .../fs/s3a/ITestS3AFileOperationCost.java | 171 +++++++++++++--- 5 files changed, 276 insertions(+), 128 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java index eb87705a38c..11b2e472d64 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java @@ -108,6 +108,7 @@ public class Listing { * @return the iterator * @throws IOException IO Problems */ + @Retries.RetryRaw FileStatusListingIterator createFileStatusListingIterator( Path listPath, S3ListRequest request, @@ -330,6 +331,7 @@ public class Listing { * items that are not listed from source. * @throws IOException IO Problems */ + @Retries.RetryTranslated FileStatusListingIterator(ObjectListingIterator source, PathFilter filter, FileStatusAcceptor acceptor, @@ -361,10 +363,12 @@ public class Listing { * @throws IOException */ @Override + @Retries.RetryTranslated public boolean hasNext() throws IOException { return sourceHasNext() || providedStatusIterator.hasNext(); } + @Retries.RetryTranslated private boolean sourceHasNext() throws IOException { if (statusBatchIterator.hasNext() || requestNextBatch()) { return true; @@ -379,6 +383,7 @@ public class Listing { } @Override + @Retries.RetryTranslated public FileStatus next() throws IOException { final FileStatus status; if (sourceHasNext()) { @@ -408,6 +413,7 @@ public class Listing { * @return true if a new batch was created. * @throws IOException IO problems */ + @Retries.RetryTranslated private boolean requestNextBatch() throws IOException { // look for more object listing batches being available while (source.hasNext()) { @@ -543,6 +549,7 @@ public class Listing { * @param request initial request to make * @throws IOException if listObjects raises one. */ + @Retries.RetryRaw ObjectListingIterator( Path listPath, S3ListRequest request) throws IOException { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Retries.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Retries.java index 80ecf0cb9a4..e46a5a46937 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Retries.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Retries.java @@ -89,4 +89,25 @@ public class Retries { String value() default ""; } + + /** + * Retried, Exceptions are swallowed. + */ + @Documented + @Retention(RetentionPolicy.SOURCE) + public @interface RetryExceptionsSwallowed { + + String value() default ""; + } + + /** + * One attempt, Exceptions are swallowed. + */ + @Documented + @Retention(RetentionPolicy.SOURCE) + public @interface OnceExceptionsSwallowed { + + String value() 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 62b97d69b46..138fc5f354b 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 @@ -74,7 +74,6 @@ import com.amazonaws.services.s3.transfer.TransferManagerConfiguration; import com.amazonaws.services.s3.transfer.Upload; import com.amazonaws.services.s3.transfer.model.UploadResult; import com.amazonaws.event.ProgressListener; -import com.amazonaws.event.ProgressEvent; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.util.concurrent.ListeningExecutorService; @@ -865,7 +864,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { * The inner rename operation. See {@link #rename(Path, Path)} for * the description of the operation. * This operation throws an exception on any failure which needs to be - * reported and downgraded to a failure. That is: if a rename + * reported and downgraded to a failure. + * Retries: retry translated, assuming all operations it is called do + * so. For safely, consider catch and handle AmazonClientException + * because this is such a complex method there's a risk it could surface. * @param source path to be renamed * @param dest new path after rename * @throws RenameFailedException if some criteria for a state changing @@ -876,6 +878,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { * @throws IOException on IO failure. * @throws AmazonClientException on failures inside the AWS SDK */ + @Retries.RetryMixed private boolean innerRename(Path source, Path dest) throws RenameFailedException, FileNotFoundException, IOException, AmazonClientException { @@ -969,10 +972,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { LOG.debug("rename: renaming file {} to {}", src, dst); long length = srcStatus.getLen(); if (dstStatus != null && dstStatus.isDirectory()) { - String newDstKey = dstKey; - if (!newDstKey.endsWith("/")) { - newDstKey = newDstKey + "/"; - } + String newDstKey = maybeAddTrailingSlash(dstKey); String filename = srcKey.substring(pathToKey(src.getParent()).length()+1); newDstKey = newDstKey + filename; @@ -990,13 +990,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { LOG.debug("rename: renaming directory {} to {}", src, dst); // This is a directory to directory copy - if (!dstKey.endsWith("/")) { - dstKey = dstKey + "/"; - } - - if (!srcKey.endsWith("/")) { - srcKey = srcKey + "/"; - } + dstKey = maybeAddTrailingSlash(dstKey); + srcKey = maybeAddTrailingSlash(srcKey); //Verify dest is not a child of the source directory if (dstKey.startsWith(srcKey)) { @@ -1065,7 +1060,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { metadataStore.move(srcPaths, dstMetas); - if (src.getParent() != dst.getParent()) { + if (!src.getParent().equals(dst.getParent())) { + LOG.debug("source & dest parents are different; fix up dir markers"); deleteUnnecessaryFakeDirectories(dst.getParent()); maybeCreateFakeParentDirectory(src); } @@ -1321,6 +1317,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { * does not update the metastore. * Increments the {@code OBJECT_DELETE_REQUESTS} and write * operation statistics. + * This call does not create any mock parent entries. * * Retry policy: retry untranslated; delete considered idempotent. * @param key key to blob to delete. @@ -1515,7 +1512,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { * @return the upload initiated * @throws AmazonClientException on problems */ - @Retries.OnceRaw + @Retries.OnceRaw("For PUT; post-PUT actions are RetriesExceptionsSwallowed") PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest) throws AmazonClientException { long len = getPutRequestLength(putObjectRequest); @@ -1685,7 +1682,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { public boolean delete(Path f, boolean recursive) throws IOException { try { entryPoint(INVOCATION_DELETE); - return innerDelete(innerGetFileStatus(f, true), recursive); + boolean outcome = innerDelete(innerGetFileStatus(f, true), recursive); + if (outcome) { + maybeCreateFakeParentDirectory(f); + } + return outcome; } catch (FileNotFoundException e) { LOG.debug("Couldn't delete {} - does not exist", f); instrumentation.errorIgnored(); @@ -1697,7 +1698,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { /** * Delete an object. See {@link #delete(Path, boolean)}. - * + * This call does not create any fake parent directory; that is + * left to the caller. * @param status fileStatus object * @param recursive if path is a directory and set to * true, the directory is deleted else throws an exception. In @@ -1771,7 +1773,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { deleteObjectAtPath(f, key, true); } - maybeCreateFakeParentDirectory(f); return true; } @@ -2049,11 +2050,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { fPart = fPart.getParent(); } String key = pathToKey(f); + // this will create the marker file, delete the parent entries + // and update S3Guard createFakeDirectory(key); - S3Guard.makeDirsOrdered(metadataStore, metadataStoreDirs, username, true); - // this is complicated because getParent(a/b/c/) returns a/b/c, but - // we want a/b. See HADOOP-14428 for more details. - deleteUnnecessaryFakeDirectories(new Path(f.toString()).getParent()); return true; } } @@ -2389,7 +2388,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { * @return the upload result * @throws InterruptedIOException if the blocking was interrupted. */ - @Retries.OnceRaw + @Retries.OnceRaw("For PUT; post-PUT actions are RetriesExceptionsSwallowed") UploadResult executePut(PutObjectRequest putObjectRequest, Progressable progress) throws InterruptedIOException { @@ -2483,6 +2482,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { /** * Copy a single object in the bucket via a COPY operation. + * There's no update of metadata, directory markers, etc. + * Callers must implement. * @param srcKey source object path * @param dstKey destination object path * @param size object size @@ -2490,46 +2491,42 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { * @throws InterruptedIOException the operation was interrupted * @throws IOException Other IO problems */ + @Retries.RetryMixed private void copyFile(String srcKey, String dstKey, long size) - throws IOException, InterruptedIOException, AmazonClientException { + throws IOException, InterruptedIOException { LOG.debug("copyFile {} -> {} ", srcKey, dstKey); - try { - ObjectMetadata srcom = getObjectMetadata(srcKey); - ObjectMetadata dstom = cloneObjectMetadata(srcom); - setOptionalObjectMetadata(dstom); - CopyObjectRequest copyObjectRequest = - new CopyObjectRequest(bucket, srcKey, bucket, dstKey); - setOptionalCopyObjectRequestParameters(copyObjectRequest); - copyObjectRequest.setCannedAccessControlList(cannedACL); - copyObjectRequest.setNewObjectMetadata(dstom); - - ProgressListener progressListener = new ProgressListener() { - public void progressChanged(ProgressEvent progressEvent) { - switch (progressEvent.getEventType()) { - case TRANSFER_PART_COMPLETED_EVENT: - incrementWriteOperations(); - break; - default: - break; - } - } - }; - - Copy copy = transfers.copy(copyObjectRequest); - copy.addProgressListener(progressListener); - try { - copy.waitForCopyResult(); + ProgressListener progressListener = progressEvent -> { + switch (progressEvent.getEventType()) { + case TRANSFER_PART_COMPLETED_EVENT: incrementWriteOperations(); - instrumentation.filesCopied(1, size); - } catch (InterruptedException e) { - throw new InterruptedIOException("Interrupted copying " + srcKey - + " to " + dstKey + ", cancelling"); + break; + default: + break; } - } catch (AmazonClientException e) { - throw translateException("copyFile("+ srcKey+ ", " + dstKey + ")", - srcKey, e); - } + }; + + once("copyFile(" + srcKey + ", " + dstKey + ")", srcKey, + () -> { + ObjectMetadata srcom = getObjectMetadata(srcKey); + ObjectMetadata dstom = cloneObjectMetadata(srcom); + setOptionalObjectMetadata(dstom); + CopyObjectRequest copyObjectRequest = + new CopyObjectRequest(bucket, srcKey, bucket, dstKey); + setOptionalCopyObjectRequestParameters(copyObjectRequest); + copyObjectRequest.setCannedAccessControlList(cannedACL); + copyObjectRequest.setNewObjectMetadata(dstom); + Copy copy = transfers.copy(copyObjectRequest); + copy.addProgressListener(progressListener); + try { + copy.waitForCopyResult(); + incrementWriteOperations(); + instrumentation.filesCopied(1, size); + } catch (InterruptedException e) { + throw new InterruptedIOException("Interrupted copying " + srcKey + + " to " + dstKey + ", cancelling"); + } + }); } protected void setOptionalMultipartUploadRequestParameters( @@ -2626,9 +2623,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { /** * Perform post-write actions. + * Calls {@link #deleteUnnecessaryFakeDirectories(Path)} and then + * {@link S3Guard#addAncestors(MetadataStore, Path, String)}}. * This operation MUST be called after any PUT/multipart PUT completes * successfully. - * This includes + * + * The operations actions include *
    *
  1. Calling {@link #deleteUnnecessaryFakeDirectories(Path)}
  2. *
  3. Updating any metadata store with details on the newly created @@ -2638,12 +2638,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { * @param length total length of file written */ @InterfaceAudience.Private - @Retries.RetryTranslated("Exceptions are swallowed") + @Retries.RetryExceptionsSwallowed void finishedWrite(String key, long length) { LOG.debug("Finished write to {}, len {}", key, length); Path p = keyToQualifiedPath(key); - deleteUnnecessaryFakeDirectories(p.getParent()); Preconditions.checkArgument(length >= 0, "content length is negative"); + deleteUnnecessaryFakeDirectories(p.getParent()); // See note about failure semantics in S3Guard documentation try { @@ -2666,7 +2666,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { * Retry policy: retrying; exceptions swallowed. * @param path path */ - @Retries.RetryRaw("Exceptions are swallowed") + @Retries.RetryExceptionsSwallowed private void deleteUnnecessaryFakeDirectories(Path path) { List keysToRemove = new ArrayList<>(); while (!path.isRoot()) { @@ -2960,7 +2960,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { * @throws IOException IO failure * @see Common Response Headers */ - + @Override + @Retries.RetryTranslated public EtagChecksum getFileChecksum(Path f, final long length) throws IOException { Preconditions.checkArgument(length >= 0); @@ -3002,18 +3003,21 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { * @throws IOException if any I/O error occurred */ @Override + @Retries.OnceTranslated public RemoteIterator listFiles(Path f, boolean recursive) throws FileNotFoundException, IOException { return innerListFiles(f, recursive, new Listing.AcceptFilesOnly(qualify(f))); } + @Retries.OnceTranslated public RemoteIterator listFilesAndEmptyDirectories(Path f, boolean recursive) throws IOException { return innerListFiles(f, recursive, new Listing.AcceptAllButS3nDirs()); } + @Retries.OnceTranslated private RemoteIterator innerListFiles(Path f, boolean recursive, Listing.FileStatusAcceptor acceptor) throws IOException { entryPoint(INVOCATION_LIST_FILES); @@ -3097,42 +3101,43 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { * @throws IOException if any I/O error occurred */ @Override - @Retries.OnceTranslated + @Retries.OnceTranslated("s3guard not retrying") public RemoteIterator listLocatedStatus(final Path f, final PathFilter filter) throws FileNotFoundException, IOException { entryPoint(INVOCATION_LIST_LOCATED_STATUS); Path path = qualify(f); LOG.debug("listLocatedStatus({}, {}", path, filter); - try { - // lookup dir triggers existence check - final FileStatus fileStatus = getFileStatus(path); - if (fileStatus.isFile()) { - // simple case: File - LOG.debug("Path is a file"); - return new Listing.SingleStatusRemoteIterator( - filter.accept(path) ? toLocatedFileStatus(fileStatus) : null); - } else { - // directory: trigger a lookup - final String key = maybeAddTrailingSlash(pathToKey(path)); - final Listing.FileStatusAcceptor acceptor = - new Listing.AcceptAllButSelfAndS3nDirs(path); - DirListingMetadata meta = metadataStore.listChildren(path); - final RemoteIterator cachedFileStatusIterator = - listing.createProvidedFileStatusIterator( - S3Guard.dirMetaToStatuses(meta), filter, acceptor); - return (allowAuthoritative && meta != null && meta.isAuthoritative()) - ? listing.createLocatedFileStatusIterator(cachedFileStatusIterator) - : listing.createLocatedFileStatusIterator( - listing.createFileStatusListingIterator(path, - createListObjectsRequest(key, "/"), - filter, - acceptor, - cachedFileStatusIterator)); - } - } catch (AmazonClientException e) { - throw translateException("listLocatedStatus", path, e); - } + return once("listLocatedStatus", path.toString(), + () -> { + // lookup dir triggers existence check + final FileStatus fileStatus = getFileStatus(path); + if (fileStatus.isFile()) { + // simple case: File + LOG.debug("Path is a file"); + return new Listing.SingleStatusRemoteIterator( + filter.accept(path) ? toLocatedFileStatus(fileStatus) : null); + } else { + // directory: trigger a lookup + final String key = maybeAddTrailingSlash(pathToKey(path)); + final Listing.FileStatusAcceptor acceptor = + new Listing.AcceptAllButSelfAndS3nDirs(path); + DirListingMetadata meta = metadataStore.listChildren(path); + final RemoteIterator cachedFileStatusIterator = + listing.createProvidedFileStatusIterator( + S3Guard.dirMetaToStatuses(meta), filter, acceptor); + return (allowAuthoritative && meta != null + && meta.isAuthoritative()) + ? listing.createLocatedFileStatusIterator( + cachedFileStatusIterator) + : listing.createLocatedFileStatusIterator( + listing.createFileStatusListingIterator(path, + createListObjectsRequest(key, "/"), + filter, + acceptor, + cachedFileStatusIterator)); + } + }); } /** @@ -3159,6 +3164,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities { * @return Iterator over multipart uploads. * @throws IOException on failure */ + @InterfaceAudience.Private + @Retries.RetryTranslated public MultipartUtils.UploadIterator listUploads(@Nullable String prefix) throws IOException { return MultipartUtils.listMultipartUploads(s3, invoker, bucket, maxKeys, 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 a56b0558bcb..19cfe1b34f1 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 @@ -257,7 +257,15 @@ public final class S3Guard { * Update MetadataStore to reflect creation of the given directories. * * If an IOException is raised while trying to update the entry, this - * operation catches the exception and returns. + * operation catches the exception, swallows it and returns. + * + * @deprecated this is no longer called by {@code S3AFilesystem.innerMkDirs}. + * See: HADOOP-15079 (January 2018). + * It is currently retained because of its discussion in the method on + * atomicity and in case we need to reinstate it or adapt the current + * process of directory marker creation. + * But it is not being tested and so may age with time...consider + * deleting it in future if it's clear there's no need for it. * @param ms MetadataStore to update. * @param dirs null, or an ordered list of directories from leaf to root. * E.g. if /a/ exists, and mkdirs(/a/b/c/d) is called, this @@ -267,6 +275,8 @@ public final class S3Guard { * @param owner Hadoop user name. * @param authoritative Whether to mark new directories as authoritative. */ + @Deprecated + @Retries.OnceExceptionsSwallowed public static void makeDirsOrdered(MetadataStore ms, List dirs, String owner, boolean authoritative) { if (dirs == null) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java index e56fdf8a6f0..279ec9cd6a8 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java @@ -29,12 +29,14 @@ import org.slf4j.LoggerFactory; import java.io.File; import java.io.FileNotFoundException; import java.net.URI; +import java.util.UUID; +import java.util.concurrent.Callable; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.test.GenericTestUtils.getTestDir; -import static org.junit.Assume.assumeFalse; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Use metrics to assert about the cost of file status queries. @@ -54,6 +56,7 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase { S3AFileSystem fs = getFileSystem(); metadataRequests = new MetricDiff(fs, OBJECT_METADATA_REQUESTS); listRequests = new MetricDiff(fs, OBJECT_LIST_REQUESTS); + skipDuringFaultInjection(fs); } @Test @@ -83,8 +86,8 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase { fs.mkdirs(dir); resetMetricDiffs(); S3AFileStatus status = fs.innerGetFileStatus(dir, true); - assertTrue("not empty: " + status, - status.isEmptyDirectory() == Tristate.TRUE); + assertSame("not empty: " + status, status.isEmptyDirectory(), + Tristate.TRUE); if (!fs.hasMetadataStore()) { metadataRequests.assertDiffEquals(2); @@ -98,12 +101,8 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase { S3AFileSystem fs = getFileSystem(); Path path = path("missing"); resetMetricDiffs(); - try { - FileStatus status = fs.getFileStatus(path); - fail("Got a status back from a missing file path " + status); - } catch (FileNotFoundException expected) { - // expected - } + intercept(FileNotFoundException.class, + () -> fs.getFileStatus(path)); metadataRequests.assertDiffEquals(2); listRequests.assertDiffEquals(1); } @@ -114,12 +113,8 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase { S3AFileSystem fs = getFileSystem(); Path path = path("missingdir/missingpath"); resetMetricDiffs(); - try { - FileStatus status = fs.getFileStatus(path); - fail("Got a status back from a missing file path " + status); - } catch (FileNotFoundException expected) { - // expected - } + intercept(FileNotFoundException.class, + () -> fs.getFileStatus(path)); metadataRequests.assertDiffEquals(2); listRequests.assertDiffEquals(1); } @@ -184,10 +179,11 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase { } } - private void reset(MetricDiff... diffs) { + private boolean reset(MetricDiff... diffs) { for (MetricDiff diff : diffs) { diff.reset(); } + return true; } @Test @@ -201,9 +197,7 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase { // operations, it depends on side effects happening internally. With // metadata store enabled, it is brittle to change. We disable this test // before the internal behavior w/ or w/o metadata store. - assumeFalse(fs.hasMetadataStore()); - - skipDuringFaultInjection(fs); +// assumeFalse(fs.hasMetadataStore()); Path srcBaseDir = path("src"); mkdirs(srcBaseDir); @@ -216,62 +210,171 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase { MetricDiff directoriesCreated = new MetricDiff(fs, Statistic.DIRECTORIES_CREATED); + // when you call toString() on this, you get the stats + // so it gets auto-evaluated in log calls. + Object summary = new Object() { + @Override + public String toString() { + return String.format("[%s, %s, %s, %s]", + directoriesCreated, directoriesDeleted, + deleteRequests, fakeDirectoriesDeleted); + } + }; + + // reset operation to invoke + Callable reset = () -> + reset(deleteRequests, directoriesCreated, directoriesDeleted, + fakeDirectoriesDeleted); + Path srcDir = new Path(srcBaseDir, "1/2/3/4/5/6"); int srcDirDepth = directoriesInPath(srcDir); // one dir created, one removed mkdirs(srcDir); - String state = "after mkdir(srcDir)"; + String state = "after mkdir(srcDir) " + summary; directoriesCreated.assertDiffEquals(state, 1); deleteRequests.assertDiffEquals(state, 1); directoriesDeleted.assertDiffEquals(state, 0); // HADOOP-14255 deletes unnecessary fake directory objects in mkdirs() fakeDirectoriesDeleted.assertDiffEquals(state, srcDirDepth - 1); - reset(deleteRequests, directoriesCreated, directoriesDeleted, - fakeDirectoriesDeleted); + reset.call(); // creating a file should trigger demise of the src dir final Path srcFilePath = new Path(srcDir, "source.txt"); touch(fs, srcFilePath); - state = "after touch(fs, srcFilePath)"; + state = "after touch(fs, srcFilePath) " + summary; deleteRequests.assertDiffEquals(state, 1); directoriesCreated.assertDiffEquals(state, 0); directoriesDeleted.assertDiffEquals(state, 0); fakeDirectoriesDeleted.assertDiffEquals(state, srcDirDepth); - reset(deleteRequests, directoriesCreated, directoriesDeleted, - fakeDirectoriesDeleted); + reset.call(); + // create a directory tree, expect the dir to be created and + // a request to delete all parent directories made. Path destBaseDir = path("dest"); Path destDir = new Path(destBaseDir, "1/2/3/4/5/6"); Path destFilePath = new Path(destDir, "dest.txt"); mkdirs(destDir); - state = "after mkdir(destDir)"; + state = "after mkdir(destDir) " + summary; int destDirDepth = directoriesInPath(destDir); directoriesCreated.assertDiffEquals(state, 1); deleteRequests.assertDiffEquals(state, 1); directoriesDeleted.assertDiffEquals(state, 0); fakeDirectoriesDeleted.assertDiffEquals(state, destDirDepth - 1); - reset(deleteRequests, directoriesCreated, directoriesDeleted, - fakeDirectoriesDeleted); + // create a new source file. + // Explicitly use a new path object to guarantee that the parent paths + // are different object instances + final Path srcFile2 = new Path(srcDir.toUri() + "/source2.txt"); + touch(fs, srcFile2); + + reset.call(); + + // rename the source file to the destination file. + // this tests the file rename path, not the dir rename path + // as srcFile2 exists, the parent dir of srcFilePath must not be created. fs.rename(srcFilePath, destFilePath); - state = "after rename(srcFilePath, destFilePath)"; - directoriesCreated.assertDiffEquals(state, 1); - // one for the renamed file, one for the parent + state = String.format("after rename(srcFilePath, destFilePath)" + + " %s dest dir depth=%d", + summary, + destDirDepth); + + directoriesCreated.assertDiffEquals(state, 0); + // one for the renamed file, one for the parent of the dest dir deleteRequests.assertDiffEquals(state, 2); directoriesDeleted.assertDiffEquals(state, 0); fakeDirectoriesDeleted.assertDiffEquals(state, destDirDepth); - reset(deleteRequests, directoriesCreated, directoriesDeleted, - fakeDirectoriesDeleted); - + // these asserts come after the checks on iop counts, so they don't + // interfere assertIsFile(destFilePath); assertIsDirectory(srcDir); + assertPathDoesNotExist("should have gone in the rename", srcFilePath); + reset.call(); + + // rename the source file2 to the (no longer existing + // this tests the file rename path, not the dir rename path + // as srcFile2 exists, the parent dir of srcFilePath must not be created. + fs.rename(srcFile2, srcFilePath); + state = String.format("after rename(%s, %s) %s dest dir depth=%d", + srcFile2, srcFilePath, + summary, + destDirDepth); + + // here we expect there to be no fake directories + directoriesCreated.assertDiffEquals(state, 0); + // one for the renamed file only + deleteRequests.assertDiffEquals(state, 1); + directoriesDeleted.assertDiffEquals(state, 0); + fakeDirectoriesDeleted.assertDiffEquals(state, 0); } private int directoriesInPath(Path path) { return path.isRoot() ? 0 : 1 + directoriesInPath(path.getParent()); } + @Test + public void testCostOfRootRename() throws Throwable { + describe("assert that a root directory rename doesn't" + + " do much in terms of parent dir operations"); + S3AFileSystem fs = getFileSystem(); + + // unique name, so that even when run in parallel tests, there's no conflict + String uuid = UUID.randomUUID().toString(); + Path src = new Path("/src-" + uuid); + Path dest = new Path("/dest-" + uuid); + + try { + MetricDiff deleteRequests = + new MetricDiff(fs, Statistic.OBJECT_DELETE_REQUESTS); + MetricDiff directoriesDeleted = + new MetricDiff(fs, Statistic.DIRECTORIES_DELETED); + MetricDiff fakeDirectoriesDeleted = + new MetricDiff(fs, Statistic.FAKE_DIRECTORIES_DELETED); + MetricDiff directoriesCreated = + new MetricDiff(fs, Statistic.DIRECTORIES_CREATED); + touch(fs, src); + fs.rename(src, dest); + Object summary = new Object() { + @Override + public String toString() { + return String.format("[%s, %s, %s, %s]", + directoriesCreated, directoriesDeleted, + deleteRequests, fakeDirectoriesDeleted); + } + }; + + String state = String.format("after touch(%s) %s", + src, summary); + touch(fs, src); + fs.rename(src, dest); + directoriesCreated.assertDiffEquals(state, 0); + + + state = String.format("after rename(%s, %s) %s", + src, dest, summary); + // here we expect there to be no fake directories + directoriesCreated.assertDiffEquals(state, 0); + // one for the renamed file only + deleteRequests.assertDiffEquals(state, 1); + directoriesDeleted.assertDiffEquals(state, 0); + fakeDirectoriesDeleted.assertDiffEquals(state, 0); + + // delete that destination file, assert only the file delete was issued + reset(deleteRequests, directoriesCreated, directoriesDeleted, + fakeDirectoriesDeleted); + + fs.delete(dest, false); + // here we expect there to be no fake directories + directoriesCreated.assertDiffEquals(state, 0); + // one for the deleted file + deleteRequests.assertDiffEquals(state, 1); + directoriesDeleted.assertDiffEquals(state, 0); + fakeDirectoriesDeleted.assertDiffEquals(state, 0); + } finally { + fs.delete(src, false); + fs.delete(dest, false); + } + } }