From 66cac818651b6676a046001e527e9658b38a733e Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 14 Jul 2020 16:57:07 +0100 Subject: [PATCH 01/20] HADOOP-13230. Dir Marker patch: reapply everything from previous PR Change-Id: I93097ff39f7254f18f8d382ad891002502c7112d --- .../apache/hadoop/test/AssertExtensions.java | 74 ++ hadoop-tools/hadoop-aws/pom.xml | 43 + .../org/apache/hadoop/fs/s3a/Constants.java | 36 + .../org/apache/hadoop/fs/s3a/Listing.java | 17 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 437 +++--- .../apache/hadoop/fs/s3a/S3ListRequest.java | 24 +- .../apache/hadoop/fs/s3a/S3ListResult.java | 112 +- .../hadoop/fs/s3a/impl/DirMarkerTracker.java | 154 +++ .../hadoop/fs/s3a/impl/DirectoryPolicy.java | 54 + .../fs/s3a/impl/DirectoryPolicyImpl.java | 118 ++ .../fs/s3a/impl/OperationCallbacks.java | 18 + .../hadoop/fs/s3a/impl/RenameOperation.java | 95 +- .../hadoop/fs/s3a/impl/StatusProbeEnum.java | 29 +- .../fs/s3a/s3guard/DirListingMetadata.java | 25 +- .../fs/s3a/s3guard/ITtlTimeProvider.java | 13 + .../apache/hadoop/fs/s3a/s3guard/S3Guard.java | 68 +- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 14 +- .../hadoop/fs/s3a/tools/MarkerTool.java | 261 ++++ .../hadoop/fs/s3a/tools/package-info.java | 27 + .../fs/s3a/ITestS3ABucketExistence.java | 5 +- .../hadoop/fs/s3a/ITestS3AEncryptionSSEC.java | 212 +-- .../fs/s3a/ITestS3AFileOperationCost.java | 1167 ++++++++++++----- .../fs/s3a/ITestS3ARemoteFileChanged.java | 2 +- .../s3a/ITestS3GuardOutOfBandOperations.java | 13 +- .../apache/hadoop/fs/s3a/S3ATestUtils.java | 9 +- .../hadoop/fs/s3a/TestS3AGetFileStatus.java | 24 +- .../s3a/auth/ITestRestrictedReadAccess.java | 72 +- .../s3a/impl/TestDirectoryMarkerPolicy.java | 120 ++ .../ITestS3GuardDDBRootOperations.java | 9 +- .../s3a/s3guard/TestDirListingMetadata.java | 11 +- .../hadoop/fs/s3a/test/HeadListCosts.java | 82 ++ .../fs/s3a/test/OperationCostValidator.java | 327 +++++ 32 files changed, 2930 insertions(+), 742 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/AssertExtensions.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/package-info.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/HeadListCosts.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/OperationCostValidator.java diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/AssertExtensions.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/AssertExtensions.java new file mode 100644 index 0000000000000..8c5e553f71ee3 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/AssertExtensions.java @@ -0,0 +1,74 @@ +/* + * 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.test; + +import java.util.concurrent.Callable; + +import org.assertj.core.description.Description; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Extra classes to work with AssertJ. + * These are kept separate from {@link LambdaTestUtils} so there's + * no requirement for AssertJ to be on the classpath in that broadly + * used class. + */ +public final class AssertExtensions { + + private static final Logger LOG = + LoggerFactory.getLogger(AssertExtensions.class); + + private AssertExtensions() { + } + + /** + * A description for AssertJ "describedAs" clauses which evaluates the + * lambda-expression only on failure. That must return a string + * or null/"" to be skipped. + * @param eval lambda expression to invoke + * @return a description for AssertJ + */ + public static Description dynamicDescription(Callable eval) { + return new DynamicDescription(eval); + } + + private static final class DynamicDescription extends Description { + private final Callable eval; + + private DynamicDescription(final Callable eval) { + this.eval = eval; + } + + @Override + public String value() { + try { + return eval.call(); + } catch (Exception e) { + LOG.warn("Failed to evaluate description: " + e); + LOG.debug("Evaluation failure", e); + // return null so that the description evaluation chain + // will skip this one + return null; + } + } + } + + +} diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 5cdaf26007f7a..a887e80fb124d 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -52,6 +52,7 @@ 200000 + @@ -123,6 +124,7 @@ ${fs.s3a.scale.test.huge.filesize} ${fs.s3a.scale.test.huge.partitionsize} ${fs.s3a.scale.test.timeout} + ${fs.s3a.directory.marker} @@ -163,6 +165,7 @@ ${fs.s3a.s3guard.test.enabled} ${fs.s3a.s3guard.test.authoritative} ${fs.s3a.s3guard.test.implementation} + ${fs.s3a.directory.marker} ${test.integration.timeout} @@ -215,6 +218,7 @@ ${fs.s3a.s3guard.test.enabled} ${fs.s3a.s3guard.test.implementation} ${fs.s3a.s3guard.test.authoritative} + ${fs.s3a.directory.marker} @@ -269,6 +273,7 @@ ${fs.s3a.s3guard.test.enabled} ${fs.s3a.s3guard.test.implementation} ${fs.s3a.s3guard.test.authoritative} + ${fs.s3a.directory.marker} ${fs.s3a.scale.test.timeout} @@ -332,6 +337,44 @@ + + + keep-markers + + + markers + keep + + + + keep + + + + delete-markers + + + markers + delete + + + + delete + + + + auth-markers + + + markers + authoritative + + + + authoritative + + + 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 22a0b45f1c7a5..a738fe2465880 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 @@ -953,4 +953,40 @@ private Constants() { * Value: {@value} seconds. */ public static final int THREAD_POOL_SHUTDOWN_DELAY_SECONDS = 30; + + /** + * Policy for directory markers. + * This is a new feature of HADOOP-13230 which addresses + * some scale, performance and permissions issues -but + * at the risk of backwards compatibility. + */ + public static final String DIRECTORY_MARKER_POLICY = + "fs.s3a.directory.markers"; + + /** + * Retain directory markers. + * Value: {@value}. + */ + public static final String DIRECTORY_MARKER_POLICY_KEEP = + "keep"; + + /** + * Delete directory markers. This is the backwards compatible option. + * Value: {@value}. + */ + public static final String DIRECTORY_MARKER_POLICY_DELETE = + "delete"; + + /** + * Retain directory markers in authoritative directory trees only. + * Value: {@value}. + */ + public static final String DIRECTORY_MARKER_POLICY_AUTHORITATIVE = + "authoritative"; + + /** + * Default retention policy. + */ + public static final String DEFAULT_DIRECTORY_MARKER_POLICY = + DIRECTORY_MARKER_POLICY_DELETE; } 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 fcb492857e617..2db74fd74423b 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 @@ -142,12 +142,27 @@ public FileStatusListingIterator createFileStatusListingIterator( Listing.FileStatusAcceptor acceptor, RemoteIterator providedStatus) throws IOException { return new FileStatusListingIterator( - new ObjectListingIterator(listPath, request), + createObjectListingIterator(listPath, request), filter, acceptor, providedStatus); } + /** + * Create an object listing iterator against a path, with a given + * list object request. + * @param listPath path of the listing + * @param request initial request to make + * @return the iterator + * @throws IOException IO Problems + */ + @Retries.RetryRaw + public ObjectListingIterator createObjectListingIterator( + final Path listPath, + final S3ListRequest request) throws IOException { + return new ObjectListingIterator(listPath, request); + } + /** * Create a located status iterator over a file status iterator. * @param statusIterator an iterator over the remote status entries 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 2cd23255c4b26..1069972cfafbc 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 @@ -67,7 +67,6 @@ import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.PutObjectRequest; import com.amazonaws.services.s3.model.PutObjectResult; -import com.amazonaws.services.s3.model.S3ObjectSummary; import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; import com.amazonaws.services.s3.model.SSECustomerKey; import com.amazonaws.services.s3.model.UploadPartRequest; @@ -104,6 +103,8 @@ import org.apache.hadoop.fs.s3a.impl.ContextAccessors; import org.apache.hadoop.fs.s3a.impl.CopyOutcome; import org.apache.hadoop.fs.s3a.impl.DeleteOperation; +import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; +import org.apache.hadoop.fs.s3a.impl.DirectoryPolicyImpl; import org.apache.hadoop.fs.s3a.impl.InternalConstants; import org.apache.hadoop.fs.s3a.impl.ListingOperationCallbacks; import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport; @@ -295,6 +296,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private final ListingOperationCallbacks listingOperationCallbacks = new ListingOperationCallbacksImpl(); + /** + * Directory policy. + */ + private DirectoryPolicy directoryPolicy; + + /** + * Context accessors for re-use. + */ + private final ContextAccessors contextAccessors = new ContextAccessorsImpl(); /** Add any deprecated keys. */ @SuppressWarnings("deprecation") @@ -452,6 +462,12 @@ public void initialize(URI name, Configuration originalConf) DEFAULT_S3GUARD_DISABLED_WARN_LEVEL); S3Guard.logS3GuardDisabled(LOG, warnLevel, bucket); } + // directory policy, which will look at authoritative paths + // if needed + directoryPolicy = new DirectoryPolicyImpl(conf, + this::allowAuthoritative); + LOG.debug("Directory marker retention policy is {}", + directoryPolicy); initMultipartUploads(conf); @@ -1285,7 +1301,7 @@ public WriteOperationHelper getWriteOperationHelper() { * is not a directory. */ @Override - public FSDataOutputStream createNonRecursive(Path path, + public FSDataOutputStream createNonRecursive(Path p, FsPermission permission, EnumSet flags, int bufferSize, @@ -1293,10 +1309,22 @@ public FSDataOutputStream createNonRecursive(Path path, long blockSize, Progressable progress) throws IOException { entryPoint(INVOCATION_CREATE_NON_RECURSIVE); + final Path path = makeQualified(p); Path parent = path.getParent(); - if (parent != null) { - // expect this to raise an exception if there is no parent - if (!getFileStatus(parent).isDirectory()) { + // expect this to raise an exception if there is no parent dir + if (parent != null && !parent.isRoot()) { + S3AFileStatus status; + try { + // optimize for the directory existing: Call list first + status = innerGetFileStatus(parent, false, + StatusProbeEnum.DIRECTORIES); + } catch (FileNotFoundException e) { + // no dir, fall back to looking for a file + // (failure condition if true) + status = innerGetFileStatus(parent, false, + StatusProbeEnum.HEAD_ONLY); + } + if (!status.isDirectory()) { throw new FileAlreadyExistsException("Not a directory: " + parent); } } @@ -1431,10 +1459,13 @@ private Pair initiateRename( LOG.debug("rename: destination path {} not found", dst); // Parent must exist Path parent = dst.getParent(); - if (!pathToKey(parent).isEmpty()) { + if (!pathToKey(parent).isEmpty() + && !parent.equals(src.getParent()) ) { try { - S3AFileStatus dstParentStatus = innerGetFileStatus(dst.getParent(), - false, StatusProbeEnum.ALL); + // only look against S3 for directories; saves + // a HEAD request on all normal operations. + S3AFileStatus dstParentStatus = innerGetFileStatus(parent, + false, StatusProbeEnum.DIRECTORIES); if (!dstParentStatus.isDirectory()) { throw new RenameFailedException(src, dst, "destination parent is not a directory"); @@ -1535,11 +1566,22 @@ public void deleteObjectAtPath(final Path path, final boolean isFile, final BulkOperationState operationState) throws IOException { - once("delete", key, () -> + once("delete", path.toString(), () -> S3AFileSystem.this.deleteObjectAtPath(path, key, isFile, operationState)); } + @Override + @Retries.RetryTranslated + public void deleteDirectoryMarkers(final Path path, + final String key, + final BulkOperationState operationState) + throws IOException { + if (!keepDirectoryMarkers(path)) { + deleteUnnecessaryFakeDirectories(path, operationState); + } + } + @Override @Retries.RetryTranslated public RemoteIterator listFilesAndEmptyDirectories( @@ -1585,7 +1627,9 @@ public void finishRename(final Path sourceRenamed, final Path destCreated) Path destParent = destCreated.getParent(); if (!sourceRenamed.getParent().equals(destParent)) { LOG.debug("source & dest parents are different; fix up dir markers"); - deleteUnnecessaryFakeDirectories(destParent); + if (!keepDirectoryMarkers(destCreated)) { + deleteUnnecessaryFakeDirectories(destParent, null); + } maybeCreateFakeParentDirectory(sourceRenamed); } } @@ -1940,6 +1984,7 @@ protected ObjectMetadata getObjectMetadata(String key, protected S3ListResult listObjects(S3ListRequest request) throws IOException { incrementReadOperations(); incrementStatistic(OBJECT_LIST_REQUESTS); + LOG.debug("LIST {}", request); validateListArguments(request); try(DurationInfo ignored = new DurationInfo(LOG, false, "LIST")) { @@ -2573,7 +2618,7 @@ private void createFakeDirectoryIfNecessary(Path f) // we only make the LIST call; the codepaths to get here should not // be reached if there is an empty dir marker -and if they do, it // is mostly harmless to create a new one. - if (!key.isEmpty() && !s3Exists(f, EnumSet.of(StatusProbeEnum.List))) { + if (!key.isEmpty() && !s3Exists(f, StatusProbeEnum.DIRECTORIES)) { LOG.debug("Creating new fake directory at {}", f); createFakeDirectory(key); } @@ -2589,7 +2634,7 @@ private void createFakeDirectoryIfNecessary(Path f) void maybeCreateFakeParentDirectory(Path path) throws IOException, AmazonClientException { Path parent = path.getParent(); - if (parent != null) { + if (parent != null && !parent.isRoot()) { createFakeDirectoryIfNecessary(parent); } } @@ -2618,7 +2663,7 @@ public FileStatus[] listStatus(Path f) throws FileNotFoundException, * @throws IOException due to an IO problem. * @throws AmazonClientException on failures inside the AWS SDK */ - public FileStatus[] innerListStatus(Path f) throws FileNotFoundException, + private S3AFileStatus[] innerListStatus(Path f) throws FileNotFoundException, IOException, AmazonClientException { Path path = qualify(f); String key = pathToKey(path); @@ -2626,7 +2671,9 @@ public FileStatus[] innerListStatus(Path f) throws FileNotFoundException, entryPoint(INVOCATION_LIST_STATUS); List result; - final FileStatus fileStatus = getFileStatus(path); + entryPoint(INVOCATION_GET_FILE_STATUS); + final S3AFileStatus fileStatus = innerGetFileStatus(path, false, + StatusProbeEnum.ALL); if (fileStatus.isDirectory()) { if (!key.isEmpty()) { @@ -2658,7 +2705,7 @@ public FileStatus[] innerListStatus(Path f) throws FileNotFoundException, allowAuthoritative, ttlTimeProvider); } else { LOG.debug("Adding: rd (not a dir): {}", path); - FileStatus[] stats = new FileStatus[1]; + S3AFileStatus[] stats = new S3AFileStatus[1]; stats[0]= fileStatus; return stats; } @@ -2769,9 +2816,10 @@ public UserGroupInformation getOwner() { public boolean mkdirs(Path path, FsPermission permission) throws IOException, FileAlreadyExistsException { try { + entryPoint(INVOCATION_MKDIRS); return innerMkdirs(path, permission); } catch (AmazonClientException e) { - throw translateException("innerMkdirs", path, e); + throw translateException("mkdirs", path, e); } } @@ -2791,11 +2839,15 @@ private boolean innerMkdirs(Path p, FsPermission permission) throws IOException, FileAlreadyExistsException, AmazonClientException { Path f = qualify(p); LOG.debug("Making directory: {}", f); - entryPoint(INVOCATION_MKDIRS); + if (p.isRoot()) { + // fast exit for root. + return true; + } FileStatus fileStatus; try { - fileStatus = getFileStatus(f); + fileStatus = innerGetFileStatus(f, false, + StatusProbeEnum.ALL); if (fileStatus.isDirectory()) { return true; @@ -2805,7 +2857,7 @@ private boolean innerMkdirs(Path p, FsPermission permission) } catch (FileNotFoundException e) { // Walk path to root, ensuring closest ancestor is a directory, not file Path fPart = f.getParent(); - while (fPart != null) { + while (fPart != null && !fPart.isRoot()) { try { fileStatus = getFileStatus(fPart); if (fileStatus.isDirectory()) { @@ -2866,7 +2918,8 @@ S3AFileStatus innerGetFileStatus(final Path f, final Set probes) throws IOException { final Path path = qualify(f); String key = pathToKey(path); - LOG.debug("Getting path status for {} ({})", path, key); + LOG.debug("Getting path status for {} ({}); needEmptyDirectory={}", + path, key, needEmptyDirectoryFlag); boolean allowAuthoritative = allowAuthoritative(path); // Check MetadataStore, if any. @@ -2877,9 +2930,10 @@ S3AFileStatus innerGetFileStatus(final Path f, } Set tombstones = Collections.emptySet(); if (pm != null) { + S3AFileStatus msStatus = pm.getFileStatus(); if (pm.isDeleted()) { OffsetDateTime deletedAt = OffsetDateTime.ofInstant( - Instant.ofEpochMilli(pm.getFileStatus().getModificationTime()), + Instant.ofEpochMilli(msStatus.getModificationTime()), ZoneOffset.UTC); throw new FileNotFoundException("Path " + path + " is recorded as " + "deleted by S3Guard at " + deletedAt); @@ -2890,72 +2944,96 @@ S3AFileStatus innerGetFileStatus(final Path f, // Skip going to s3 if the file checked is a directory. Because if the // dest is also a directory, there's no difference. - if (!pm.getFileStatus().isDirectory() && + if (!msStatus.isDirectory() && !allowAuthoritative && probes.contains(StatusProbeEnum.Head)) { // a file has been found in a non-auth path and the caller has not said // they only care about directories LOG.debug("Metadata for {} found in the non-auth metastore.", path); - final long msModTime = pm.getFileStatus().getModificationTime(); - - S3AFileStatus s3AFileStatus; - try { - s3AFileStatus = s3GetFileStatus(path, key, probes, tombstones); - } catch (FileNotFoundException fne) { - s3AFileStatus = null; - } - if (s3AFileStatus == null) { - LOG.warn("Failed to find file {}. Either it is not yet visible, or " - + "it has been deleted.", path); - } else { - final long s3ModTime = s3AFileStatus.getModificationTime(); - - if(s3ModTime > msModTime) { - LOG.debug("S3Guard metadata for {} is outdated;" - + " s3modtime={}; msModTime={} updating metastore", - path, s3ModTime, msModTime); - return S3Guard.putAndReturn(metadataStore, s3AFileStatus, - ttlTimeProvider); + // TODO: if the timestamp of the pm is close to "now", we don't need to + // TODO: bother with a check of S3. that means: + // TODO: one of : status modtime is close to now, + // TODO: or pm.getLastUpdated() == now + long validTime = ttlTimeProvider.getNow() - ttlTimeProvider.getMetadataTtl(); + final long msModTime = msStatus.getModificationTime(); + + if (msModTime < validTime) { + LOG.debug("Metastore entry is out of date, probing S3"); + try { + S3AFileStatus s3AFileStatus = s3GetFileStatus(path, key, probes, + tombstones, needEmptyDirectoryFlag); + final long s3ModTime = s3AFileStatus.getModificationTime(); + + if (s3ModTime > msModTime) { + // there's new data in S3 + LOG.debug("S3Guard metadata for {} is outdated;" + + " s3modtime={}; msModTime={} updating metastore", + path, s3ModTime, msModTime); + // add to S3Guard and return the value + // note that the checks for empty dir status below can be skipped + // because the call to s3GetFileStatus include the checks there + return S3Guard.putAndReturn(metadataStore, s3AFileStatus, + ttlTimeProvider); + } else { + // the modtime of the data is the same as/older than the s3guard + // value either an old object has been found, or the existing one + // was retrieved in both cases -refresh the S3Guard entry so the + // record's TTL is updated. + S3Guard.refreshEntry(metadataStore, pm, s3AFileStatus, + ttlTimeProvider); + } + } catch (FileNotFoundException fne) { + // the attempt to refresh the record failed because there was + // no entry. Either it is a new file not visible, or it + // has been deleted (and therefore S3Guard is out of sync with S3) + LOG.warn("Failed to find file {}. Either it is not yet visible, or " + + "it has been deleted.", path); } } } - S3AFileStatus msStatus = pm.getFileStatus(); if (needEmptyDirectoryFlag && msStatus.isDirectory()) { + // the caller needs to know if a directory is empty, + // and that this is a directory. if (pm.isEmptyDirectory() != Tristate.UNKNOWN) { // We have a definitive true / false from MetadataStore, we are done. return msStatus; } else { + // execute a S3Guard listChildren command to list tombstones under the + // path. + // This list will be used in the forthcoming s3GetFileStatus call. DirListingMetadata children = S3Guard.listChildrenWithTtl(metadataStore, path, ttlTimeProvider, allowAuthoritative); if (children != null) { tombstones = children.listTombstones(); } - LOG.debug("MetadataStore doesn't know if dir is empty, using S3."); + LOG.debug("MetadataStore doesn't know if {} is empty, using S3.", + path); } } else { // Either this is not a directory, or we don't care if it is empty return msStatus; } - // If the metadata store has no children for it and it's not listed in - // S3 yet, we'll assume the empty directory is true; - S3AFileStatus s3FileStatus; + // now issue the S3 getFileStatus call. try { - s3FileStatus = s3GetFileStatus(path, key, probes, tombstones); + S3AFileStatus s3FileStatus = s3GetFileStatus(path, key, probes, tombstones, + true); + // entry was found, so save in S3Guard and return the final value. + return S3Guard.putAndReturn(metadataStore, s3FileStatus, + ttlTimeProvider); } catch (FileNotFoundException e) { + // If the metadata store has no children for it and it's not listed in + // S3 yet, we'll conclude that it is an empty directory return S3AFileStatus.fromFileStatus(msStatus, Tristate.TRUE, null, null); } - // entry was found, save in S3Guard - return S3Guard.putAndReturn(metadataStore, s3FileStatus, - ttlTimeProvider); } else { // there was no entry in S3Guard // retrieve the data and update the metadata store in the process. return S3Guard.putAndReturn(metadataStore, - s3GetFileStatus(path, key, probes, tombstones), + s3GetFileStatus(path, key, probes, tombstones, needEmptyDirectoryFlag), ttlTimeProvider); } } @@ -3010,6 +3088,8 @@ S3AFileStatus innerGetFileStatus(final Path f, * @param key Key string for the path * @param probes probes to make * @param tombstones tombstones to filter + * @param needEmptyDirectoryFlag if true, implementation will calculate + * a TRUE or FALSE value for {@link S3AFileStatus#isEmptyDirectory()} * @return Status * @throws FileNotFoundException the supplied probes failed. * @throws IOException on other problems. @@ -3019,88 +3099,85 @@ S3AFileStatus innerGetFileStatus(final Path f, S3AFileStatus s3GetFileStatus(final Path path, final String key, final Set probes, - @Nullable Set tombstones) throws IOException { - if (!key.isEmpty()) { - if (probes.contains(StatusProbeEnum.Head) && !key.endsWith("/")) { - try { - // look for the simple file - ObjectMetadata meta = getObjectMetadata(key); - LOG.debug("Found exact file: normal file {}", key); - return new S3AFileStatus(meta.getContentLength(), - dateToLong(meta.getLastModified()), - path, - getDefaultBlockSize(path), - username, - meta.getETag(), - meta.getVersionId()); - } catch (AmazonServiceException e) { - // if the response is a 404 error, it just means that there is - // no file at that path...the remaining checks will be needed. - if (e.getStatusCode() != SC_404 || isUnknownBucket(e)) { - throw translateException("getFileStatus", path, e); - } - } catch (AmazonClientException e) { + @Nullable final Set tombstones, + final boolean needEmptyDirectoryFlag) throws IOException { + LOG.debug("S3GetFileStatus {}", path); + Preconditions.checkArgument(!needEmptyDirectoryFlag + || probes.contains(StatusProbeEnum.List), + "s3GetFileStatus(%s) wants to know if a directory is empty but" + + " does not request a list probe", path); + + if (!key.isEmpty() && !key.endsWith("/") + && probes.contains(StatusProbeEnum.Head)) { + try { + // look for the simple file + ObjectMetadata meta = getObjectMetadata(key); + LOG.debug("Found exact file: normal file {}", key); + return new S3AFileStatus(meta.getContentLength(), + dateToLong(meta.getLastModified()), + path, + getDefaultBlockSize(path), + username, + meta.getETag(), + meta.getVersionId()); + } catch (AmazonServiceException e) { + // if the response is a 404 error, it just means that there is + // no file at that path...the remaining checks will be needed. + if (e.getStatusCode() != SC_404 || isUnknownBucket(e)) { throw translateException("getFileStatus", path, e); } - } - - // Either a normal file was not found or the probe was skipped. - // because the key ended in "/" or it was not in the set of probes. - // Look for the dir marker - if (probes.contains(StatusProbeEnum.DirMarker)) { - String newKey = maybeAddTrailingSlash(key); - try { - ObjectMetadata meta = getObjectMetadata(newKey); - - if (objectRepresentsDirectory(newKey, meta.getContentLength())) { - LOG.debug("Found file (with /): fake directory"); - return new S3AFileStatus(Tristate.TRUE, path, username); - } else { - LOG.warn("Found file (with /): real file? should not happen: {}", - key); - - return new S3AFileStatus(meta.getContentLength(), - dateToLong(meta.getLastModified()), - path, - getDefaultBlockSize(path), - username, - meta.getETag(), - meta.getVersionId()); - } - } catch (AmazonServiceException e) { - if (e.getStatusCode() != SC_404 || isUnknownBucket(e)) { - throw translateException("getFileStatus", newKey, e); - } - } catch (AmazonClientException e) { - throw translateException("getFileStatus", newKey, e); - } + } catch (AmazonClientException e) { + throw translateException("getFileStatus", path, e); } } // execute the list if (probes.contains(StatusProbeEnum.List)) { try { + // this will find a marker dir / as well as an entry. + // When making a simple "is this a dir check" all is good. + // but when looking for an empty dir, we need to verify there are no + // children, so ask for two entries, so as to find + // a child String dirKey = maybeAddTrailingSlash(key); - S3ListRequest request = createListObjectsRequest(dirKey, "/", 1); + // list size is dir marker + at least one non-tombstone entry + // there's a corner case: more tombstones than you have in a + // single page list. We assume that if you have been deleting + // that many files, then the AWS listing will have purged some + // by the time of listing so that the response includes some + // which have not. + + int listSize; + if (tombstones == null) { + // no tombstones so look for a marker and at least one child. + listSize = 2; + } else { + // build a listing > tombstones. If the caller has many thousands + // of tombstones this won't work properly, which is why pruning + // of expired tombstones matters. + listSize = Math.min(2 + tombstones.size(), Math.max(2, maxKeys)); + } + S3ListRequest request = createListObjectsRequest(dirKey, "/", + listSize); + // execute the request + S3ListResult listResult = listObjects(request); - S3ListResult objects = listObjects(request); - Collection prefixes = objects.getCommonPrefixes(); - Collection summaries = objects.getObjectSummaries(); - if (!isEmptyOfKeys(prefixes, tombstones) || - !isEmptyOfObjects(summaries, tombstones)) { + if (listResult.hasPrefixesOrObjects(contextAccessors, tombstones)) { if (LOG.isDebugEnabled()) { - LOG.debug("Found path as directory (with /): {}/{}", - prefixes.size(), summaries.size()); - - for (S3ObjectSummary summary : summaries) { - LOG.debug("Summary: {} {}", summary.getKey(), summary.getSize()); - } - for (String prefix : prefixes) { - LOG.debug("Prefix: {}", prefix); - } + LOG.debug("Found path as directory (with /)"); + listResult.logAtDebug(LOG); } - + // At least one entry has been found. + // If looking for an empty directory, the marker must exist but no children. + // So the listing must contain the marker entry only. + if (needEmptyDirectoryFlag + && listResult.representsEmptyDirectory( + contextAccessors, dirKey, tombstones)) { + return new S3AFileStatus(Tristate.TRUE, path, username); + } + // either an empty directory is not needed, or the + // listing does not meet the requirements. return new S3AFileStatus(Tristate.FALSE, path, username); } else if (key.isEmpty()) { LOG.debug("Found root directory"); @@ -3119,48 +3196,6 @@ S3AFileStatus s3GetFileStatus(final Path path, throw new FileNotFoundException("No such file or directory: " + path); } - /** - * Helper function to determine if a collection of paths is empty - * after accounting for tombstone markers (if provided). - * @param keys Collection of path (prefixes / directories or keys). - * @param tombstones Set of tombstone markers, or null if not applicable. - * @return false if summaries contains objects not accounted for by - * tombstones. - */ - private boolean isEmptyOfKeys(Collection keys, Set - tombstones) { - if (tombstones == null) { - return keys.isEmpty(); - } - for (String key : keys) { - Path qualified = keyToQualifiedPath(key); - if (!tombstones.contains(qualified)) { - return false; - } - } - return true; - } - - /** - * Helper function to determine if a collection of object summaries is empty - * after accounting for tombstone markers (if provided). - * @param summaries Collection of objects as returned by listObjects. - * @param tombstones Set of tombstone markers, or null if not applicable. - * @return false if summaries contains objects not accounted for by - * tombstones. - */ - private boolean isEmptyOfObjects(Collection summaries, - Set tombstones) { - if (tombstones == null) { - return summaries.isEmpty(); - } - Collection stringCollection = new ArrayList<>(summaries.size()); - for (S3ObjectSummary summary : summaries) { - stringCollection.add(summary.getKey()); - } - return isEmptyOfKeys(stringCollection, tombstones); - } - /** * Raw version of {@link FileSystem#exists(Path)} which uses S3 only: * S3Guard MetadataStore, if any, will be skipped. @@ -3175,7 +3210,7 @@ private boolean s3Exists(final Path path, final Set probes) throws IOException { String key = pathToKey(path); try { - s3GetFileStatus(path, key, probes, null); + s3GetFileStatus(path, key, probes, null, false); return true; } catch (FileNotFoundException e) { return false; @@ -3578,6 +3613,7 @@ private CopyResult copyFile(String srcKey, String dstKey, long size, copyObjectRequest.setNewObjectMetadata(dstom); Optional.ofNullable(srcom.getStorageClass()) .ifPresent(copyObjectRequest::setStorageClass); + incrementStatistic(OBJECT_COPY_REQUESTS); Copy copy = transfers.copy(copyObjectRequest); copy.addProgressListener(progressListener); CopyOutcome copyOutcome = CopyOutcome.waitForCopy(copy); @@ -3711,14 +3747,14 @@ private Optional generateSSECustomerKey() { /** * Perform post-write actions. - * Calls {@link #deleteUnnecessaryFakeDirectories(Path)} and then + * Calls {@link #deleteUnnecessaryFakeDirectories(Path, BulkOperationState)} and then * updates any metastore. * This operation MUST be called after any PUT/multipart PUT completes * successfully. * * The operations actions include *
    - *
  1. Calling {@link #deleteUnnecessaryFakeDirectories(Path)}
  2. + *
  3. Calling {@link #deleteUnnecessaryFakeDirectories(Path, BulkOperationState)}
  4. *
  5. Updating any metadata store with details on the newly created * object.
  6. *
@@ -3743,12 +3779,19 @@ void finishedWrite(String key, long length, String eTag, String versionId, Preconditions.checkArgument(length >= 0, "content length is negative"); final boolean isDir = objectRepresentsDirectory(key, length); // kick off an async delete - final CompletableFuture deletion = submit( - unboundedThreadPool, - () -> { - deleteUnnecessaryFakeDirectories(p.getParent()); - return null; - }); + CompletableFuture deletion; + if (!keepDirectoryMarkers(p)) { + deletion = submit( + unboundedThreadPool, + () -> { + deleteUnnecessaryFakeDirectories( + p.getParent(), + operationState); + return null; + }); + } else { + deletion = null; + } // this is only set if there is a metastore to update and the // operationState parameter passed in was null. BulkOperationState stateToClose = null; @@ -3807,13 +3850,26 @@ void finishedWrite(String key, long length, String eTag, String versionId, } } + /** + * Should we keep directory markers under the path being created + * by mkdir/file creation/rename? + * @param path path to probe + * @return true if the markers MAY be retained, + * false if they MUST be deleted + */ + private boolean keepDirectoryMarkers(Path path) { + return directoryPolicy.keepDirectoryMarkers(path); + } + /** * Delete mock parent directories which are no longer needed. * Retry policy: retrying; exceptions swallowed. * @param path path + * @param operationState (nullable) operational state for a bulk update */ @Retries.RetryExceptionsSwallowed - private void deleteUnnecessaryFakeDirectories(Path path) { + private void deleteUnnecessaryFakeDirectories(Path path, + final BulkOperationState operationState) { List keysToRemove = new ArrayList<>(); while (!path.isRoot()) { String key = pathToKey(path); @@ -3823,7 +3879,7 @@ private void deleteUnnecessaryFakeDirectories(Path path) { path = path.getParent(); } try { - removeKeys(keysToRemove, true, null); + removeKeys(keysToRemove, true, operationState); } catch(AmazonClientException | IOException e) { instrumentation.errorIgnored(); if (LOG.isDebugEnabled()) { @@ -4086,25 +4142,41 @@ public boolean exists(Path f) throws IOException { } /** - * Override superclass so as to add statistic collection. + * Optimized probe for a path referencing a dir. + * Even though it is optimized to a single HEAD, applications + * should not over-use this method...it is all too common. * {@inheritDoc} */ @Override @SuppressWarnings("deprecation") public boolean isDirectory(Path f) throws IOException { entryPoint(INVOCATION_IS_DIRECTORY); - return super.isDirectory(f); + try { + return innerGetFileStatus(f, false, StatusProbeEnum.DIRECTORIES) + .isDirectory(); + } catch (FileNotFoundException e) { + // not found or it is a file. + return false; + } } /** - * Override superclass so as to add statistic collection. + * Optimized probe for a path referencing a file. + * Even though it is optimized to a single HEAD, applications + * should not over-use this method...it is all too common. * {@inheritDoc} */ @Override @SuppressWarnings("deprecation") public boolean isFile(Path f) throws IOException { entryPoint(INVOCATION_IS_FILE); - return super.isFile(f); + try { + return innerGetFileStatus(f, false, StatusProbeEnum.HEAD_ONLY) + .isFile(); + } catch (FileNotFoundException e) { + // not found or it is a dir. + return false; + } } /** @@ -4800,6 +4872,15 @@ public StoreContext createStoreContext() { .build(); } + /** + * Get the operation callbacks for this FS. + * @return callbacks for operations. + */ + @InterfaceAudience.Private + public OperationCallbacks getOperationCallbacks() { + return operationCallbacks; + } + /** * The implementation of context accessors. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java index 1a0d2c3378ca6..abce86d410272 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java @@ -24,11 +24,15 @@ /** * API version-independent container for S3 List requests. */ -public class S3ListRequest { - private ListObjectsRequest v1Request; - private ListObjectsV2Request v2Request; +public final class S3ListRequest { - protected S3ListRequest(ListObjectsRequest v1, ListObjectsV2Request v2) { + private static final String DESCRIPTION + = "List %s:/%s delimiter=%s keys=%d requester pays=%s"; + + private final ListObjectsRequest v1Request; + private final ListObjectsV2Request v2Request; + + private S3ListRequest(ListObjectsRequest v1, ListObjectsV2Request v2) { v1Request = v1; v2Request = v2; } @@ -70,11 +74,15 @@ public ListObjectsV2Request getV2() { @Override public String toString() { if (isV1()) { - return String.format("List %s:/%s", - v1Request.getBucketName(), v1Request.getPrefix()); + return String.format(DESCRIPTION, + v1Request.getBucketName(), v1Request.getPrefix(), + v1Request.getDelimiter(), v1Request.getMaxKeys(), + v1Request.isRequesterPays()); } else { - return String.format("List %s:/%s", - v2Request.getBucketName(), v2Request.getPrefix()); + return String.format(DESCRIPTION, + v2Request.getBucketName(), v2Request.getPrefix(), + v2Request.getDelimiter(), v2Request.getMaxKeys(), + v2Request.isRequesterPays()); } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java index e8aff329070ef..551a8c4375151 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java @@ -18,11 +18,18 @@ package org.apache.hadoop.fs.s3a; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + import com.amazonaws.services.s3.model.ListObjectsV2Result; import com.amazonaws.services.s3.model.ObjectListing; import com.amazonaws.services.s3.model.S3ObjectSummary; +import org.slf4j.Logger; -import java.util.List; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.impl.ContextAccessors; /** * API version-independent container for S3 List responses. @@ -92,6 +99,109 @@ public List getCommonPrefixes() { } else { return v2Result.getCommonPrefixes(); } + } + + /** + * Is the list of object summaries empty + * after accounting for tombstone markers (if provided)? + * @param accessors callback for key to path mapping. + * @param tombstones Set of tombstone markers, or null if not applicable. + * @return false if summaries contains objects not accounted for by + * tombstones. + */ + public boolean isEmptyOfObjects( + final ContextAccessors accessors, + final Set tombstones) { + if (tombstones == null) { + return getObjectSummaries().isEmpty(); + } + return isEmptyOfKeys(accessors, + objectSummaryKeys(), + tombstones); + } + + /** + * Get the list of keys in the object summary. + * @return a possibly empty list + */ + private List objectSummaryKeys() { + return getObjectSummaries().stream() + .map(S3ObjectSummary::getKey) + .collect(Collectors.toList()); + } + + /** + * Does this listing have prefixes or objects? + * @param accessors callback for key to path mapping. + * @param tombstones Set of tombstone markers, or null if not applicable. + * @return true if the reconciled list is non-empty + */ + public boolean hasPrefixesOrObjects( + final ContextAccessors accessors, + final Set tombstones) { + + return !isEmptyOfKeys(accessors, getCommonPrefixes(), tombstones) + || !isEmptyOfObjects(accessors, tombstones); + } + + /** + * Helper function to determine if a collection of keys is empty + * after accounting for tombstone markers (if provided). + * @param accessors callback for key to path mapping. + * @param keys Collection of path (prefixes / directories or keys). + * @param tombstones Set of tombstone markers, or null if not applicable. + * @return true if the list is considered empty. + */ + public boolean isEmptyOfKeys( + final ContextAccessors accessors, + final Collection keys, + final Set tombstones) { + if (tombstones == null) { + return keys.isEmpty(); + } + for (String key : keys) { + Path qualified = accessors.keyToPath(key); + if (!tombstones.contains(qualified)) { + return false; + } + } + return true; + } + /** + * Does this listing represent an empty directory? + * @param contextAccessors callback for key to path mapping. + * @param dirKey directory key + * @param tombstones Set of tombstone markers, or null if not applicable. + * @return true if the list is considered empty. + */ + public boolean representsEmptyDirectory( + final ContextAccessors contextAccessors, + final String dirKey, + final Set tombstones) { + // If looking for an empty directory, the marker must exist but + // no children. + // So the listing must contain the marker entry only as an object, + // and prefixes is null + List keys = objectSummaryKeys(); + return keys.size() == 1 && keys.contains(dirKey) + && getCommonPrefixes().isEmpty(); + } + + /** + * dump the result at debug level only. + * @param log log to use + */ + public void logAtDebug(Logger log) { + Collection prefixes = getCommonPrefixes(); + Collection summaries = getObjectSummaries(); + log.debug("Prefix count = {}; object count={}", + prefixes.size(), summaries.size()); + for (S3ObjectSummary summary : summaries) { + log.debug("Summary: {} {}", summary.getKey(), summary.getSize()); + } + for (String prefix : prefixes) { + log.debug("Prefix: {}", prefix); + } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java new file mode 100644 index 0000000000000..af785eb9b268e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java @@ -0,0 +1,154 @@ +/* + * 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.util.Map; +import java.util.TreeMap; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; + +/** + * Tracks directory markers which have been reported in object listings. + * This is needed for auditing and cleanup, including during rename + * operations. + *

+ * Designed to be used while scanning through the results of listObject + * calls, where are we assume the results come in alphanumeric sort order + * and parent entries before children. + *

+ * This lets as assume that we can identify all leaf markers as those + * markers which were added to set of leaf markers and not subsequently + * removed as a child entries were discovered. + *

+ * To avoid scanning datastructures excessively, the path of the parent + * directory of the last file added is cached. This allows for a + * quick bailout when many children of the same directory are + * returned in a listing. + */ +public class DirMarkerTracker { + + /** + * all leaf markers. + */ + private final Map> leafMarkers + = new TreeMap<>(); + + /** + * all surplus markers. + */ + private final Map> surplusMarkers + = new TreeMap<>(); + + /** + * last parent directory checked. + */ + private Path lastDirChecked; + + /** + * Count of scans; used for test assertions. + */ + private int scanCount; + + /** + * A marker has been found; this may or may not be a leaf. + * Trigger a move of all markers above it into the surplus map. + * @param path marker path + * @param key object key + * @param source listing source + * @return the number of surplus markers found. + */ + public int markerFound(Path path, + final String key, + final S3ALocatedFileStatus source) { + leafMarkers.put(path, Pair.of(key, source)); + return fileFound(path, key, source); + } + + /** + * A file has been found. Trigger a move of all + * markers above it into the surplus map. + * @param path marker path + * @param key object key + * @param source listing source + * @return the number of surplus markers found. + */ + public int fileFound(Path path, + final String key, + final S3ALocatedFileStatus source) { + // all parent entries are superfluous + final Path parent = path.getParent(); + if (parent == null || parent.equals(lastDirChecked)) { + // short cut exit + return 0; + } + final int markers = removeParentMarkers(parent); + lastDirChecked = parent; + return markers; + } + + /** + * Remove all markers from the path and its parents. + * @param path path to start at + * @return number of markers removed. + */ + private int removeParentMarkers(final Path path) { + if (path == null || path.isRoot()) { + return 0; + } + scanCount++; + int parents = removeParentMarkers(path.getParent()); + final Pair value = leafMarkers.remove(path); + if (value != null) { + // marker is surplus + surplusMarkers.put(path, value); + parents++; + } + return parents; + } + + /** + * get the map of leaf markers. + * @return all leaf markers. + */ + public Map> getLeafMarkers() { + return leafMarkers; + } + + /** + * get the map of surplus markers. + * @return all surplus markers. + */ + public Map> getSurplusMarkers() { + return surplusMarkers; + } + + @VisibleForTesting + public Path getLastDirChecked() { + return lastDirChecked; + } + + @VisibleForTesting + public int getScanCount() { + return scanCount; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java new file mode 100644 index 0000000000000..e81577f5486d2 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.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 org.apache.hadoop.fs.Path; + +public interface DirectoryPolicy { + + /** + * Should a directory marker be retained? + * @param path path a file/directory is being created with. + * @return true if the marker MAY be kept, false if it MUST be deleted. + */ + boolean keepDirectoryMarkers(Path path); + + /** + * Supported retention policies. + */ + enum MarkerPolicy { + /** + * Keep markers. + * This is Not backwards compatible. + */ + Keep, + + /** + * Delete markers. + * This is what has been done since S3A was released. */ + Delete, + + /** + * Keep markers in authoritative paths only. + * This is Not backwards compatible within the + * auth paths, but is outside these. + */ + Authoritative + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java new file mode 100644 index 0000000000000..e3170f4f10eb3 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java @@ -0,0 +1,118 @@ +/* + * 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.util.Locale; +import java.util.function.Predicate; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; + +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_DIRECTORY_MARKER_POLICY; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_AUTHORITATIVE; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP; + +/** + * Implementation of directory policy. + */ +public final class DirectoryPolicyImpl + implements DirectoryPolicy { + + /** + * Error string when unable to parse the marker policy option. + */ + public static final String UNKNOWN_MARKER_POLICY = "Unknown value of " + + DIRECTORY_MARKER_POLICY + ": "; + + private static final Logger LOG = LoggerFactory.getLogger( + DirectoryPolicyImpl.class); + + /** + * Chosen marker policy. + */ + private final MarkerPolicy markerPolicy; + + /** + * Callback to evaluate authoritativeness of a + * path. + */ + private final Predicate authoritativeness; + + /** + * Constructor. + * @param conf config + * @param authoritativeness Callback to evaluate authoritativeness of a + * path. + */ + public DirectoryPolicyImpl( + final Configuration conf, + final Predicate authoritativeness) { + this.authoritativeness = authoritativeness; + String option = conf.getTrimmed(DIRECTORY_MARKER_POLICY, + DEFAULT_DIRECTORY_MARKER_POLICY); + MarkerPolicy p; + switch (option.toLowerCase(Locale.ENGLISH)) { + case DIRECTORY_MARKER_POLICY_DELETE: + // backwards compatible. + p = MarkerPolicy.Delete; + LOG.debug("Directory markers will be deleted"); + break; + case DIRECTORY_MARKER_POLICY_KEEP: + p = MarkerPolicy.Keep; + LOG.info("Directory markers will be kept"); + break; + case DIRECTORY_MARKER_POLICY_AUTHORITATIVE: + p = MarkerPolicy.Authoritative; + LOG.info("Directory markers will be kept on authoritative" + + " paths"); + break; + default: + throw new IllegalArgumentException(UNKNOWN_MARKER_POLICY + option); + } + this.markerPolicy = p; + } + + @Override + public boolean keepDirectoryMarkers(final Path path) { + switch (markerPolicy) { + case Keep: + return true; + case Authoritative: + return authoritativeness.test(path); + case Delete: + default: // which cannot happen + return false; + } + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "DirectoryMarkerRetention{"); + sb.append("policy='").append(markerPolicy).append('\''); + sb.append('}'); + return sb.toString(); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java index 0fcf6454c11fb..7e74a08e8f99e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java @@ -104,6 +104,24 @@ void deleteObjectAtPath(Path path, BulkOperationState operationState) throws IOException; + /** + * Delete a directory marker also updating the metastore. + * If the marker retention policy is to keep markers under this + * path, the marker is not deleted. + * This call does not create any mock parent entries. + * Retry policy: retry untranslated; delete considered idempotent. + * @param path path to delete + * @param key key of entry + * @param operationState (nullable) operational state for a bulk update + * @throws AmazonClientException problems working with S3 + * @throws IOException IO failure in the metastore + */ + @Retries.RetryTranslated + void deleteDirectoryMarkers(final Path path, + final String key, + final BulkOperationState operationState) + throws IOException; + /** * Recursive list of files and empty directories. * diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java index 750aebf500a4b..c6b14655c7891 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; @@ -31,6 +32,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.s3a.RenameFailedException; @@ -299,8 +301,9 @@ protected void renameFileToDest() throws IOException { /** * Execute a full recursive rename. - * The source is a file: rename it to the destination. - * @throws IOException failure + * There is a special handling of directly markers here -only leaf markers + * are copied. This reduces incompatibility "regions" across versions. +Are * @throws IOException failure */ protected void recursiveDirectoryRename() throws IOException { final StoreContext storeContext = getStoreContext(); @@ -325,8 +328,10 @@ protected void recursiveDirectoryRename() throws IOException { // marker. LOG.debug("Deleting fake directory marker at destination {}", destStatus.getPath()); + // TODO: dir marker policy doesn't always need to do this. callbacks.deleteObjectAtPath(destStatus.getPath(), dstKey, false, null); } + DirMarkerTracker dirMarkerTracker = new DirMarkerTracker(); Path parentPath = storeContext.keyToPath(srcKey); final RemoteIterator iterator = @@ -350,33 +355,57 @@ protected void recursiveDirectoryRename() throws IOException { // mark for deletion on a successful copy. queueToDelete(childSourcePath, key); - // the destination key is that of the key under the source tree, - // remapped under the new destination path. - String newDestKey = - dstKey + key.substring(srcKey.length()); - Path childDestPath = storeContext.keyToPath(newDestKey); - // now begin the single copy - CompletableFuture copy = initiateCopy(child, key, - childSourcePath, newDestKey, childDestPath); - activeCopies.add(copy); - bytesCopied.addAndGet(sourceStatus.getLen()); - - if (activeCopies.size() == RENAME_PARALLEL_LIMIT) { - // the limit of active copies has been reached; - // wait for completion or errors to surface. - LOG.debug("Waiting for active copies to complete"); - completeActiveCopies("batch threshold reached"); - } - if (keysToDelete.size() == pageSize) { - // 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("paged delete"); + + boolean isMarker = key.endsWith("/"); + if (isMarker) { + // markers are not replicated until we know + // that they are leaf markers. + dirMarkerTracker.markerFound(childSourcePath, key, child); + } else { + // its a file, note + dirMarkerTracker.fileFound(childSourcePath, key, child); + // the destination key is that of the key under the source tree, + // remapped under the new destination path. + String newDestKey = + dstKey + key.substring(srcKey.length()); + Path childDestPath = storeContext.keyToPath(newDestKey); + // now begin the single copy + activeCopies.add(initiateCopy(child, key, + childSourcePath, newDestKey, childDestPath)); + bytesCopied.addAndGet(sourceStatus.getLen()); } + endOfLoopActions(); } // end of iteration through the list + // directory marker work. + // for all leaf markers: copy the original + Map> leafMarkers = + dirMarkerTracker.getLeafMarkers(); + Map> surplus = + dirMarkerTracker.getSurplusMarkers(); + LOG.debug("copying {} leaf markers; {} surplus", + leafMarkers.size(), surplus.size()); + for (Map.Entry> entry : + leafMarkers.entrySet()) { + Path source = entry.getKey(); + String key = entry.getValue().getLeft(); + S3ALocatedFileStatus stat = entry.getValue().getRight(); + String newDestKey = + dstKey + key.substring(srcKey.length()); + Path childDestPath = storeContext.keyToPath(newDestKey); + LOG.debug("copying dir marker from {} to {}", key, newDestKey); + activeCopies.add(initiateCopy(stat, key, + source, newDestKey, childDestPath)); + endOfLoopActions(); + } + // the surplus ones are also explicitly deleted + for (Map.Entry> entry : + surplus.entrySet()) { + queueToDelete(entry.getKey(), entry.getValue().getLeft()); + endOfLoopActions(); + } + // await the final set of copies and their deletion // This will notify the renameTracker that these objects // have been deleted. @@ -387,6 +416,22 @@ protected void recursiveDirectoryRename() throws IOException { renameTracker.moveSourceDirectory(); } + private void endOfLoopActions() throws IOException { + if (activeCopies.size() == RENAME_PARALLEL_LIMIT) { + // the limit of active copies has been reached; + // wait for completion or errors to surface. + LOG.debug("Waiting for active copies to complete"); + completeActiveCopies("batch threshold reached"); + } + if (keysToDelete.size() == pageSize) { + // 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("paged delete"); + } + } + /** * Initiate a copy operation in the executor. * @param source status of the source object. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StatusProbeEnum.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StatusProbeEnum.java index f843b20ab28b0..f9749a14a3d7d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StatusProbeEnum.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StatusProbeEnum.java @@ -33,28 +33,25 @@ public enum StatusProbeEnum { /** LIST under the path. */ List; - /** All probes. */ - public static final Set ALL = EnumSet.allOf( - StatusProbeEnum.class); - - /** Skip the HEAD and only look for directories. */ - public static final Set DIRECTORIES = - EnumSet.of(DirMarker, List); - - /** We only want the HEAD or dir marker. */ - public static final Set HEAD_OR_DIR_MARKER = - EnumSet.of(Head, DirMarker); + /** Look for files and directories. */ + public static final Set ALL = + EnumSet.of(Head, List); /** We only want the HEAD. */ public static final Set HEAD_ONLY = EnumSet.of(Head); - /** We only want the dir marker. */ - public static final Set DIR_MARKER_ONLY = - EnumSet.of(DirMarker); - - /** We only want the dir marker. */ + /** List operation only. */ public static final Set LIST_ONLY = EnumSet.of(List); + /** Look for files and directories. */ + public static final Set FILE = + HEAD_ONLY; + + /** Skip the HEAD and only look for directories. */ + public static final Set DIRECTORIES = + LIST_ONLY; + + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java index 213ffdc983718..5f033fa11f834 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DirListingMetadata.java @@ -25,6 +25,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -118,6 +119,10 @@ public Collection getListing() { return Collections.unmodifiableCollection(listMap.values()); } + /** + * List all tombstones. + * @return all tombstones in the listing. + */ public Set listTombstones() { Set tombstones = new HashSet<>(); for (PathMetadata meta : listMap.values()) { @@ -128,6 +133,12 @@ public Set listTombstones() { return tombstones; } + /** + * Get the directory listing excluding tombstones. + * Returns a new DirListingMetadata instances, without the tombstones -the + * lastUpdated field is copied from this instance. + * @return a new DirListingMetadata without the tombstones. + */ public DirListingMetadata withoutTombstones() { Collection filteredList = new ArrayList<>(); for (PathMetadata meta : listMap.values()) { @@ -143,6 +154,7 @@ public DirListingMetadata withoutTombstones() { * @return number of entries tracked. This is not the same as the number * of entries in the actual directory unless {@link #isAuthoritative()} is * true. + * It will also include any tombstones. */ public int numEntries() { return listMap.size(); @@ -251,19 +263,24 @@ public String toString() { * Remove expired entries from the listing based on TTL. * @param ttl the ttl time * @param now the current time + * @return the expired values. */ - public synchronized void removeExpiredEntriesFromListing(long ttl, - long now) { + public synchronized List removeExpiredEntriesFromListing( + long ttl, long now) { + List expired = new ArrayList<>(); final Iterator> iterator = listMap.entrySet().iterator(); while (iterator.hasNext()) { final Map.Entry entry = iterator.next(); // we filter iff the lastupdated is not 0 and the entry is expired - if (entry.getValue().getLastUpdated() != 0 - && (entry.getValue().getLastUpdated() + ttl) <= now) { + PathMetadata metadata = entry.getValue(); + if (metadata.getLastUpdated() != 0 + && (metadata.getLastUpdated() + ttl) <= now) { + expired.add(metadata); iterator.remove(); } } + return expired; } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ITtlTimeProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ITtlTimeProvider.java index daee6211b41d9..aa7fc4721b483 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ITtlTimeProvider.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/ITtlTimeProvider.java @@ -29,6 +29,19 @@ * Time is measured in milliseconds, */ public interface ITtlTimeProvider { + + /** + * The current time in milliseconds. + * Assuming this calls System.currentTimeMillis(), this is a native iO call + * and so should be invoked sparingly (i.e. evaluate before any loop, rather + * than inside). + * @return the current time. + */ long getNow(); + + /** + * The TTL of the metadata. + * @return time in millis after which metadata is considered out of date. + */ long getMetadataTtl(); } 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 05ebe671662ea..ae5c293d639ff 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 @@ -159,6 +159,54 @@ static Class getMetadataStoreClass( } + /** + * We update the metastore for the specific case of S3 value == S3Guard value + * so as to place a more recent modtime in the store. + * because if not, we will continue to probe S3 whenever we look for this + * object, even we only do this if confident the S3 status is the same + * as the one in the store (i.e. it is not an older version) + * @param metadataStore MetadataStore to {@code put()} into. + * @param pm current data + * @param s3AFileStatus status to store + * @param timeProvider Time provider to use when writing entries + * @return true if the entry was updated. + * @throws IOException if metadata store update failed + */ + @RetryTranslated + public static boolean refreshEntry( + MetadataStore metadataStore, + PathMetadata pm, + S3AFileStatus s3AFileStatus, + ITtlTimeProvider timeProvider) throws IOException { + // the modtime of the data is the same as/older than the s3guard value + // either an old object has been found, or the existing one was retrieved + // in both cases -return s3guard value + S3AFileStatus msStatus = pm.getFileStatus(); + + // first check: size + boolean sizeMatch = msStatus.getLen() == s3AFileStatus.getLen(); + + // etags are expected on all objects, but handle the situation + // that a third party store doesn't serve them. + String s3Etag = s3AFileStatus.getETag(); + String pmEtag = msStatus.getETag(); + boolean etagsMatch = s3Etag != null && s3Etag.equals(pmEtag); + + // version ID: only in some stores, and will be missing in the metastore + // if the entry was created through a list operation. + String s3VersionId = s3AFileStatus.getVersionId(); + String pmVersionId = msStatus.getVersionId(); + boolean versionsMatchOrMissingInMetastore = + pmVersionId == null || pmVersionId.equals(s3VersionId); + if (sizeMatch && etagsMatch && versionsMatchOrMissingInMetastore) { + // update the store, return the new value + LOG.debug("Refreshing the metastore entry/timestamp"); + putAndReturn(metadataStore, s3AFileStatus, timeProvider); + return true; + } + return false; + } + /** * Helper function which puts a given S3AFileStatus into the MetadataStore and * returns the same S3AFileStatus. Instrumentation monitors the put operation. @@ -314,14 +362,14 @@ public static S3AFileStatus[] dirMetaToStatuses(DirListingMetadata dirMeta) { * @return Final result of directory listing. * @throws IOException if metadata store update failed */ - public static FileStatus[] dirListingUnion(MetadataStore ms, Path path, + public static S3AFileStatus[] dirListingUnion(MetadataStore ms, Path path, List backingStatuses, DirListingMetadata dirMeta, boolean isAuthoritative, ITtlTimeProvider timeProvider) throws IOException { // Fast-path for NullMetadataStore if (isNullMetadataStore(ms)) { - return backingStatuses.toArray(new FileStatus[backingStatuses.size()]); + return backingStatuses.toArray(new S3AFileStatus[backingStatuses.size()]); } assertQualified(path); @@ -927,8 +975,10 @@ public static PathMetadata getWithTtl(MetadataStore ms, Path path, if (!pathMetadata.isExpired(ttl, timeProvider.getNow())) { return pathMetadata; } else { - LOG.debug("PathMetadata TTl for {} is expired in metadata store.", - path); + LOG.debug("PathMetadata TTl for {} is expired in metadata store" + + " -removing entry", path); + // delete the tombstone + ms.forgetMetadata(path); return null; } } @@ -940,6 +990,8 @@ public static PathMetadata getWithTtl(MetadataStore ms, Path path, * List children; mark the result as non-auth if the TTL has expired. * If the allowAuthoritative flag is true, return without filtering or * checking for TTL expiry. + * If false: the expiry scan takes place and the + * TODO: should we always purge tombstones? Even in auth? * @param ms metastore * @param path path to look up. * @param timeProvider nullable time provider @@ -968,9 +1020,15 @@ public static DirListingMetadata listChildrenWithTtl(MetadataStore ms, // filter expired entries if (dlm != null) { - dlm.removeExpiredEntriesFromListing( + List expired = dlm.removeExpiredEntriesFromListing( timeProvider.getMetadataTtl(), timeProvider.getNow()); + // now purge the tombstones + for (PathMetadata metadata : expired) { + if (metadata.isDeleted()) { + ms.forgetMetadata(metadata.getFileStatus().getPath()); + } + } } return dlm; 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 6e89d0cd2dadb..9a906439a7449 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 @@ -59,6 +59,7 @@ import org.apache.hadoop.fs.s3a.commit.CommitConstants; import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants; import org.apache.hadoop.fs.s3a.select.SelectTool; +import org.apache.hadoop.fs.s3a.tools.MarkerTool; import org.apache.hadoop.fs.shell.CommandFormat; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.security.UserGroupInformation; @@ -98,15 +99,17 @@ public abstract class S3GuardTool extends Configured implements Tool, "Commands: \n" + "\t" + Init.NAME + " - " + Init.PURPOSE + "\n" + "\t" + Destroy.NAME + " - " + Destroy.PURPOSE + "\n" + - "\t" + Import.NAME + " - " + Import.PURPOSE + "\n" + + "\t" + Authoritative.NAME + " - " + Authoritative.PURPOSE + "\n" + "\t" + BucketInfo.NAME + " - " + BucketInfo.PURPOSE + "\n" + - "\t" + Uploads.NAME + " - " + Uploads.PURPOSE + "\n" + "\t" + Diff.NAME + " - " + Diff.PURPOSE + "\n" + + "\t" + Fsck.NAME + " - " + Fsck.PURPOSE + "\n" + + "\t" + Import.NAME + " - " + Import.PURPOSE + "\n" + + "\t" + MarkerTool.NAME + " - " + MarkerTool.PURPOSE + "\n" + "\t" + Prune.NAME + " - " + Prune.PURPOSE + "\n" + "\t" + SetCapacity.NAME + " - " + SetCapacity.PURPOSE + "\n" + "\t" + SelectTool.NAME + " - " + SelectTool.PURPOSE + "\n" + - "\t" + Fsck.NAME + " - " + Fsck.PURPOSE + "\n" + - "\t" + Authoritative.NAME + " - " + Authoritative.PURPOSE + "\n"; + "\t" + Uploads.NAME + " - " + Uploads.PURPOSE + "\n" + ; private static final String DATA_IN_S3_IS_PRESERVED = "(all data in S3 is preserved)"; @@ -1991,6 +1994,9 @@ public static int run(Configuration conf, String...args) throws case Diff.NAME: command = new Diff(conf); break; + case MarkerTool.NAME: + command = new MarkerTool(conf); + break; case Prune.NAME: command = new Prune(conf); break; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java new file mode 100644 index 0000000000000..b8a4b5c47ece4 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -0,0 +1,261 @@ +/* + * 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.tools; + +import java.io.IOException; +import java.io.PrintStream; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.StorageStatistics; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; +import org.apache.hadoop.fs.s3a.impl.DirMarkerTracker; +import org.apache.hadoop.fs.s3a.impl.OperationCallbacks; +import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; +import org.apache.hadoop.fs.shell.CommandFormat; +import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.ExitUtil; + +import static org.apache.hadoop.fs.s3a.Invoker.once; +import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE; +import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_SUCCESS; +import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_USAGE; + +/** + * Handle directory-related command-line options in the + * s3guard tool. + *
+ *   scan: scan for markers
+ *   clean: clean up marker entries.
+ * 
+ * This tool does not go anywhere near S3Guard; its scan bypasses any + * metastore as we are explicitly looking for marker objects. + * + */ +public final class MarkerTool extends S3GuardTool { + + private static final Logger LOG = + LoggerFactory.getLogger(MarkerTool.class); + + public static final String NAME = "markers"; + + public static final String PURPOSE = + "view and manipulate S3 directory markers"; + + private static final String USAGE = NAME + + " [OPTIONS]" + + " (audit || report || clean)" +// + " [-out ]" + + " [-" + VERBOSE + "]" + + " \n" + + "\t" + PURPOSE + "\n\n"; + + public static final String OPT_EXPECTED = "expected"; + + public static final String OPT_AUDIT = "audit"; + + public static final String OPT_CLEAN = "clean"; + + public static final String OPT_REPORT = "report"; + + public static final String OPT_OUTPUT = "output"; + + public static final String OPT_VERBOSE = "verbose"; + + static final String TOO_FEW_ARGUMENTS = "Too few arguments"; + + private PrintStream out; + + public MarkerTool(final Configuration conf) { + super(conf, OPT_VERBOSE); + getCommandFormat().addOptionWithValue(OPT_EXPECTED); +// getCommandFormat().addOptionWithValue(OPT_OUTPUT); + } + + @Override + public String getUsage() { + return USAGE; + } + + @Override + public String getName() { + return NAME; + } + + @Override + public int run(final String[] args, final PrintStream stream) + throws ExitUtil.ExitException, Exception { + this.out = stream; + final List parsedArgs; + try { + parsedArgs = parseArgs(args); + } catch (CommandFormat.UnknownOptionException e) { + errorln(getUsage()); + throw new ExitUtil.ExitException(EXIT_USAGE, e.getMessage(), e); + } + if (parsedArgs.size() < 2) { + errorln(getUsage()); + throw new ExitUtil.ExitException(EXIT_USAGE, TOO_FEW_ARGUMENTS); + } + // read arguments + CommandFormat commandFormat = getCommandFormat(); + boolean verbose = commandFormat.getOpt(VERBOSE); + + boolean purge = false; + int expected = 0; + String action = parsedArgs.get(0); + switch (action) { + case OPT_AUDIT: + purge = false; + expected = 0; + break; + case OPT_CLEAN: + purge = true; + expected = -1; + break; + case OPT_REPORT: + purge = false; + expected = -1; + break; + default: + errorln(getUsage()); + throw new ExitUtil.ExitException(EXIT_USAGE, "Unknown action: " + action); + } + + final String file = parsedArgs.get(1); + final Path path = new Path(file); + S3AFileSystem fs = bindFilesystem(path.getFileSystem(getConf())); + final StoreContext context = fs.createStoreContext(); + final OperationCallbacks operations = fs.getOperationCallbacks(); + + + boolean finalPurge = purge; + int finalExpected = expected; + int result = once("action", path.toString(), + () -> scan(path, finalPurge, finalExpected, verbose, context, + operations)); + if (verbose) { + dumpFileSystemStatistics(fs); + } + return result; + } + + private int scan(final Path path, + final boolean purge, + final int expected, + final boolean verbose, + final StoreContext context, + final OperationCallbacks operations) + throws IOException, ExitUtil.ExitException { + DirMarkerTracker tracker = new DirMarkerTracker(); + try (DurationInfo ignored = + new DurationInfo(LOG, "marker scan %s", path)) { + scanDirectoryTree(path, expected, context, operations, tracker); + } + // scan done. what have we got? + Map> surplusMarkers + = tracker.getSurplusMarkers(); + Map> leafMarkers + = tracker.getLeafMarkers(); + int size = surplusMarkers.size(); + if (size == 0) { + println(out, "No surplus directory markers were found under %s", path); + } else { + println(out, "Found %d surplus directory marker%s under %s", + size, + suffix(size), + path); + + for (Path markers : surplusMarkers.keySet()) { + println(out, " %s", markers); + } + + if (size > expected) { + // failure + println(out, "Expected %d marker%s", expected, suffix(size)); + return EXIT_NOT_ACCEPTABLE; + } + + } + return EXIT_SUCCESS; + } + + private String suffix(final int size) { + return size == 1 ? "" : "s"; + } + + private void scanDirectoryTree(final Path path, + final int expected, + final StoreContext context, + final OperationCallbacks operations, + final DirMarkerTracker tracker) throws IOException { + RemoteIterator listing = operations + .listObjects(path, + context.pathToKey(path)); + while (listing.hasNext()) { + S3AFileStatus status = listing.next(); + Path p = status.getPath(); + S3ALocatedFileStatus lfs = new S3ALocatedFileStatus( + status, null); + String key = context.pathToKey(p); + if (status.isDirectory()) { + LOG.info("{}", key); + tracker.markerFound(p, + key + "/", + lfs); + } else { + tracker.fileFound(p, + key, + lfs); + } + + + } + } + + /** + * Dump the filesystem Storage Statistics. + * @param fs filesystem; can be null + */ + private void dumpFileSystemStatistics(FileSystem fs) { + if (fs == null) { + return; + } + println(out, "Storage Statistics"); + StorageStatistics st = fs.getStorageStatistics(); + Iterator it + = st.getLongStatistics(); + while (it.hasNext()) { + StorageStatistics.LongStatistic next = it.next(); + println(out, "%s\t%s", next.getName(), next.getValue()); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/package-info.java new file mode 100644 index 0000000000000..ee0b56a548229 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/package-info.java @@ -0,0 +1,27 @@ +/* + * 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. + */ + +/** + * Command line tools. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.tools; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java index 6be9003e4ec38..f6c824e6d23bf 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java @@ -75,7 +75,10 @@ public void testNoBucketProbing() throws Exception { // the exception must not be caught and marked down to an FNFE expectUnknownStore(() -> fs.exists(src)); - expectUnknownStore(() -> fs.isFile(src)); + // now that isFile() only does a HEAD, it will get a 404 without + // the no-such-bucket error (really) + assertFalse("isFile(" + src + " was expected to complete by returning false", + fs.isFile(src)); expectUnknownStore(() -> fs.isDirectory(src)); expectUnknownStore(() -> fs.mkdirs(src)); expectUnknownStore(() -> fs.delete(src)); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java index f086a08201cd7..1c395b2adcfc4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java @@ -20,8 +20,13 @@ import java.io.IOException; import java.nio.file.AccessDeniedException; +import java.util.Arrays; +import java.util.Collection; +import org.assertj.core.api.Assertions; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -31,13 +36,26 @@ import org.apache.hadoop.io.IOUtils; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; +import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP; +import static org.apache.hadoop.fs.s3a.Constants.ETAG_CHECKSUM_ENABLED; +import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; +import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM; +import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_KEY; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Concrete class that extends {@link AbstractTestS3AEncryption} * and tests SSE-C encryption. + * HEAD requests against SSE-C-encrypted data will fail if the wrong key + * is presented, so the tests are very brittle to S3Guard being on vs. off. + * Equally "vexing" has been the optimizations of getFileStatus(), wherein + * LIST comes before HEAD path + / */ +@RunWith(Parameterized.class) public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption { private static final String SERVICE_AMAZON_S3_STATUS_CODE_403 @@ -52,18 +70,67 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption { = "msdo3VvvZznp66Gth58a91Hxe/UpExMkwU9BHkIjfW8="; private static final int TEST_FILE_LEN = 2048; + /** + * Parameterization. + */ + @Parameterized.Parameters(name = "{0}") + public static Collection params() { + return Arrays.asList(new Object[][]{ + {"raw-keep-markers", false, true}, + {"raw-delete-markers", false, false}, + {"guarded-keep-markers", true, true}, + {"guarded-delete-markers", true, false} + }); + } + + /** + * Parameter: should the stores be guarded? + */ + private final boolean s3guard; + + /** + * Parameter: should directory markers be retained? + */ + private final boolean keepMarkers; + /** * Filesystem created with a different key. */ - private FileSystem fsKeyB; + private S3AFileSystem fsKeyB; + + public ITestS3AEncryptionSSEC(final String name, + final boolean s3guard, + final boolean keepMarkers) { + this.s3guard = s3guard; + this.keepMarkers = keepMarkers; + } @Override protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); disableFilesystemCaching(conf); - conf.set(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM, + String bucketName = getTestBucketName(conf); + removeBucketOverrides(bucketName, conf, + S3_METADATA_STORE_IMPL); + if (!s3guard) { + // in a raw run remove all s3guard settings + removeBaseAndBucketOverrides(bucketName, conf, + S3_METADATA_STORE_IMPL); + } + // directory marker options + removeBaseAndBucketOverrides(bucketName, conf, + DIRECTORY_MARKER_POLICY, + ETAG_CHECKSUM_ENABLED, + SERVER_SIDE_ENCRYPTION_ALGORITHM, + SERVER_SIDE_ENCRYPTION_KEY); + conf.set(DIRECTORY_MARKER_POLICY, + keepMarkers + ? DIRECTORY_MARKER_POLICY_KEEP + : DIRECTORY_MARKER_POLICY_DELETE); + conf.set(SERVER_SIDE_ENCRYPTION_ALGORITHM, getSSEAlgorithm().getMethod()); - conf.set(Constants.SERVER_SIDE_ENCRYPTION_KEY, KEY_1); + conf.set(SERVER_SIDE_ENCRYPTION_KEY, KEY_1); + conf.setBoolean(ETAG_CHECKSUM_ENABLED, true); return conf; } @@ -109,31 +176,19 @@ public void testCreateFileAndReadWithDifferentEncryptionKey() throws } /** - * While each object has its own key and should be distinct, this verifies - * that hadoop treats object keys as a filesystem path. So if a top level - * dir is encrypted with keyA, a sublevel dir cannot be accessed with a - * different keyB. - * - * This is expected AWS S3 SSE-C behavior. * + * You can use a different key under a sub directory, even if you + * do not have permissions to read the marker. * @throws Exception */ @Test public void testCreateSubdirWithDifferentKey() throws Exception { - requireUnguardedFilesystem(); - - intercept(AccessDeniedException.class, - SERVICE_AMAZON_S3_STATUS_CODE_403, - () -> { - Path base = path("testCreateSubdirWithDifferentKey"); - Path nestedDirectory = new Path(base, "nestedDir"); - fsKeyB = createNewFileSystemWithSSECKey( - KEY_2); - getFileSystem().mkdirs(base); - fsKeyB.mkdirs(nestedDirectory); - // expected to fail - return fsKeyB.getFileStatus(nestedDirectory); - }); + Path base = path("testCreateSubdirWithDifferentKey"); + Path nestedDirectory = new Path(base, "nestedDir"); + fsKeyB = createNewFileSystemWithSSECKey( + KEY_2); + getFileSystem().mkdirs(base); + fsKeyB.mkdirs(nestedDirectory); } /** @@ -176,14 +231,11 @@ public void testRenameFile() throws Exception { } /** - * It is possible to list the contents of a directory up to the actual - * end of the nested directories. This is due to how S3A mocks the - * directories and how prefixes work in S3. + * Directory listings always work. * @throws Exception */ @Test public void testListEncryptedDir() throws Exception { - requireUnguardedFilesystem(); Path pathABC = path("testListEncryptedDir/a/b/c/"); Path pathAB = pathABC.getParent(); @@ -196,17 +248,11 @@ public void testListEncryptedDir() throws Exception { fsKeyB.listFiles(pathA, true); fsKeyB.listFiles(pathAB, true); - - //Until this point, no exception is thrown about access - intercept(AccessDeniedException.class, - SERVICE_AMAZON_S3_STATUS_CODE_403, - () -> { - fsKeyB.listFiles(pathABC, false); - }); + fsKeyB.listFiles(pathABC, false); Configuration conf = this.createConfiguration(); - conf.unset(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM); - conf.unset(Constants.SERVER_SIDE_ENCRYPTION_KEY); + conf.unset(SERVER_SIDE_ENCRYPTION_ALGORITHM); + conf.unset(SERVER_SIDE_ENCRYPTION_KEY); S3AContract contract = (S3AContract) createContract(conf); contract.init(); @@ -215,20 +261,14 @@ public void testListEncryptedDir() throws Exception { //unencrypted can access until the final directory unencryptedFileSystem.listFiles(pathA, true); unencryptedFileSystem.listFiles(pathAB, true); - AWSBadRequestException ex = intercept(AWSBadRequestException.class, - () -> { - unencryptedFileSystem.listFiles(pathABC, false); - }); + unencryptedFileSystem.listFiles(pathABC, false); } /** - * Much like the above list encrypted directory test, you cannot get the - * metadata of an object without the correct encryption key. - * @throws Exception + * listStatus also works with encrypted directories and key mismatch. */ @Test public void testListStatusEncryptedDir() throws Exception { - requireUnguardedFilesystem(); Path pathABC = path("testListStatusEncryptedDir/a/b/c/"); Path pathAB = pathABC.getParent(); @@ -240,17 +280,14 @@ public void testListStatusEncryptedDir() throws Exception { fsKeyB.listStatus(pathA); fsKeyB.listStatus(pathAB); - //Until this point, no exception is thrown about access - intercept(AccessDeniedException.class, - SERVICE_AMAZON_S3_STATUS_CODE_403, - () -> { - fsKeyB.listStatus(pathABC); - }); + // this used to raise 403, but with LIST before HEAD, + // no longer true. + fsKeyB.listStatus(pathABC); //Now try it with an unencrypted filesystem. Configuration conf = createConfiguration(); - conf.unset(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM); - conf.unset(Constants.SERVER_SIDE_ENCRYPTION_KEY); + conf.unset(SERVER_SIDE_ENCRYPTION_ALGORITHM); + conf.unset(SERVER_SIDE_ENCRYPTION_KEY); S3AContract contract = (S3AContract) createContract(conf); contract.init(); @@ -259,21 +296,15 @@ public void testListStatusEncryptedDir() throws Exception { //unencrypted can access until the final directory unencryptedFileSystem.listStatus(pathA); unencryptedFileSystem.listStatus(pathAB); - - intercept(AWSBadRequestException.class, - () -> { - unencryptedFileSystem.listStatus(pathABC); - }); + unencryptedFileSystem.listStatus(pathABC); } /** - * Much like trying to access a encrypted directory, an encrypted file cannot - * have its metadata read, since both are technically an object. + * An encrypted file cannot have its metadata read. * @throws Exception */ @Test public void testListStatusEncryptedFile() throws Exception { - requireUnguardedFilesystem(); Path pathABC = path("testListStatusEncryptedFile/a/b/c/"); assertTrue("mkdirs failed", getFileSystem().mkdirs(pathABC)); @@ -283,23 +314,15 @@ public void testListStatusEncryptedFile() throws Exception { fsKeyB = createNewFileSystemWithSSECKey(KEY_4); //Until this point, no exception is thrown about access - intercept(AccessDeniedException.class, - SERVICE_AMAZON_S3_STATUS_CODE_403, - () -> { - fsKeyB.listStatus(fileToStat); - }); + if (!fsKeyB.hasMetadataStore()) { + intercept(AccessDeniedException.class, + SERVICE_AMAZON_S3_STATUS_CODE_403, + () -> fsKeyB.listStatus(fileToStat)); + } else { + fsKeyB.listStatus(fileToStat); + } } - /** - * Skip the test case if S3Guard is enabled; generally this is because - * list and GetFileStatus calls can succeed even with different keys. - */ - protected void requireUnguardedFilesystem() { - assume("Filesystem has a metastore", - !getFileSystem().hasMetadataStore()); - } - - /** * It is possible to delete directories without the proper encryption key and * the hierarchy above it. @@ -308,7 +331,7 @@ protected void requireUnguardedFilesystem() { */ @Test public void testDeleteEncryptedObjectWithDifferentKey() throws Exception { - requireUnguardedFilesystem(); + //requireUnguardedFilesystem(); Path pathABC = path("testDeleteEncryptedObjectWithDifferentKey/a/b/c/"); Path pathAB = pathABC.getParent(); @@ -317,12 +340,13 @@ public void testDeleteEncryptedObjectWithDifferentKey() throws Exception { Path fileToDelete = new Path(pathABC, "filetobedeleted.txt"); writeThenReadFile(fileToDelete, TEST_FILE_LEN); fsKeyB = createNewFileSystemWithSSECKey(KEY_4); - intercept(AccessDeniedException.class, - SERVICE_AMAZON_S3_STATUS_CODE_403, - () -> { - fsKeyB.delete(fileToDelete, false); - }); - + if (!fsKeyB.hasMetadataStore()) { + intercept(AccessDeniedException.class, + SERVICE_AMAZON_S3_STATUS_CODE_403, + () -> fsKeyB.delete(fileToDelete, false)); + } else { + fsKeyB.delete(fileToDelete, false); + } //This is possible fsKeyB.delete(pathABC, true); fsKeyB.delete(pathAB, true); @@ -330,15 +354,33 @@ public void testDeleteEncryptedObjectWithDifferentKey() throws Exception { assertPathDoesNotExist("expected recursive delete", fileToDelete); } - private FileSystem createNewFileSystemWithSSECKey(String sseCKey) throws + /** + * getFileChecksum always goes to S3, so when + * the caller lacks permissions, it fails irrespective + * of guard. + */ + @Test + public void testChecksumRequiresReadAccess() throws Throwable { + Path path = path("tagged-file"); + S3AFileSystem fs = getFileSystem(); + touch(fs, path); + Assertions.assertThat(fs.getFileChecksum(path)) + .isNotNull(); + fsKeyB = createNewFileSystemWithSSECKey(KEY_4); + intercept(AccessDeniedException.class, + SERVICE_AMAZON_S3_STATUS_CODE_403, + () -> fsKeyB.getFileChecksum(path)); + } + + private S3AFileSystem createNewFileSystemWithSSECKey(String sseCKey) throws IOException { Configuration conf = this.createConfiguration(); - conf.set(Constants.SERVER_SIDE_ENCRYPTION_KEY, sseCKey); + conf.set(SERVER_SIDE_ENCRYPTION_KEY, sseCKey); S3AContract contract = (S3AContract) createContract(conf); contract.init(); FileSystem fileSystem = contract.getTestFileSystem(); - return fileSystem; + return (S3AFileSystem) fileSystem; } @Override 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 e54fd97a6af1e..15bf9ecc2fe58 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 @@ -20,11 +20,14 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FSDataOutputStreamBuilder; +import org.apache.hadoop.fs.FileAlreadyExistsException; 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.impl.StatusProbeEnum; +import org.apache.hadoop.fs.s3a.test.HeadListCosts; +import org.apache.hadoop.fs.s3a.test.OperationCostValidator; import org.assertj.core.api.Assertions; import org.junit.Test; @@ -35,52 +38,85 @@ import java.io.File; import java.io.FileNotFoundException; +import java.io.IOException; import java.net.URI; import java.util.Arrays; import java.util.Collection; import java.util.EnumSet; +import java.util.Set; import java.util.UUID; import java.util.concurrent.Callable; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP; +import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE; import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; import static org.apache.hadoop.fs.s3a.Statistic.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; +import static org.apache.hadoop.fs.s3a.test.OperationCostValidator.probe; +import static org.apache.hadoop.fs.s3a.test.OperationCostValidator.probes; +import static org.apache.hadoop.test.AssertExtensions.dynamicDescription; import static org.apache.hadoop.test.GenericTestUtils.getTestDir; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Use metrics to assert about the cost of file status queries. * {@link S3AFileSystem#getFileStatus(Path)}. - * Parameterized on guarded vs raw. + * Parameterized on guarded vs raw. and directory marker keep vs delete */ @RunWith(Parameterized.class) public class ITestS3AFileOperationCost extends AbstractS3ATestBase { - private MetricDiff metadataRequests; - private MetricDiff listRequests; - private static final Logger LOG = LoggerFactory.getLogger(ITestS3AFileOperationCost.class); + private OperationCostValidator costValidator; + /** * Parameterization. */ @Parameterized.Parameters(name = "{0}") public static Collection params() { return Arrays.asList(new Object[][]{ - {"raw", false}, - {"guarded", true} + {"raw-keep-markers", false, true}, + {"raw-delete-markers", false, false}, + {"guarded-keep-markers", true, true}, + {"guarded-delete-markers", true, false} }); } - private final String name; - + /** + * Parameter: should the stores be guarded? + */ private final boolean s3guard; - public ITestS3AFileOperationCost(final String name, final boolean s3guard) { - this.name = name; + /** + * Parameter: should directory markers be retained? + */ + private final boolean keepMarkers; + + /** + * Is this an auth mode test run? + */ + private boolean authoritative; + + /* probe states calculated from the configuration options. */ + boolean isGuarded; + boolean isRaw; + + boolean isAuthoritative; + boolean isNonAuth; + + boolean isKeeping; + + boolean isDeleting; + + public ITestS3AFileOperationCost(final String name, + final boolean s3guard, + final boolean keepMarkers) { this.s3guard = s3guard; + this.keepMarkers = keepMarkers; } @Override @@ -89,83 +125,119 @@ public Configuration createConfiguration() { String bucketName = getTestBucketName(conf); removeBucketOverrides(bucketName, conf, S3_METADATA_STORE_IMPL); - if (!s3guard) { + if (!isGuarded()) { // in a raw run remove all s3guard settings removeBaseAndBucketOverrides(bucketName, conf, S3_METADATA_STORE_IMPL); } + // directory marker options + removeBaseAndBucketOverrides(bucketName, conf, + DIRECTORY_MARKER_POLICY); + conf.set(DIRECTORY_MARKER_POLICY, + keepMarkers + ? DIRECTORY_MARKER_POLICY_KEEP + : DIRECTORY_MARKER_POLICY_DELETE); + conf.setBoolean(METADATASTORE_AUTHORITATIVE, authoritative); disableFilesystemCaching(conf); return conf; } + @Override public void setup() throws Exception { super.setup(); - if (s3guard) { + if (isGuarded()) { // s3guard is required for those test runs where any of the // guard options are set assumeS3GuardState(true, getConfiguration()); } S3AFileSystem fs = getFileSystem(); - metadataRequests = new MetricDiff(fs, OBJECT_METADATA_REQUESTS); - listRequests = new MetricDiff(fs, OBJECT_LIST_REQUESTS); skipDuringFaultInjection(fs); + authoritative = fs.allowAuthoritative(new Path("/")); + + // build up the states + isGuarded = isGuarded(); + boolean isUnguarded = !isGuarded; + + boolean isAuthoritative = isGuarded && authoritative; + boolean isNonAuth = isGuarded && !authoritative; + + boolean isKeeping = isKeepingMarkers (); + + boolean isDeleting = !isKeeping; + + costValidator = OperationCostValidator.builder(getFileSystem()) + .withMetrics(INVOCATION_COPY_FROM_LOCAL_FILE, + OBJECT_COPY_REQUESTS, + OBJECT_DELETE_REQUESTS, + DIRECTORIES_CREATED, + DIRECTORIES_DELETED, + FAKE_DIRECTORIES_DELETED, + FILES_DELETED, + OBJECT_LIST_REQUESTS, + OBJECT_METADATA_REQUESTS, + OBJECT_PUT_BYTES, + OBJECT_PUT_REQUESTS) + .build(); + } + + public void assumeUnguarded() { + assume("Unguarded FS only", !isGuarded()); + } + + public boolean isAuthoritative() { + return authoritative; + } + + public boolean isGuarded() { + return s3guard; + } + + public boolean isKeepingMarkers() { + return keepMarkers; } @Test public void testCostOfLocatedFileStatusOnFile() throws Throwable { describe("performing listLocatedStatus on a file"); - Path file = path(getMethodName() + ".txt"); + Path file = file(methodPath()); S3AFileSystem fs = getFileSystem(); - touch(fs, file); - resetMetricDiffs(); - fs.listLocatedStatus(file); - if (!fs.hasMetadataStore()) { - // Unguarded FS. - metadataRequests.assertDiffEquals(1); - } - listRequests.assertDiffEquals(1); + verifyMetrics(() -> fs.listLocatedStatus(file), + raw(OBJECT_LIST_REQUESTS, HeadListCosts.LIST_LOCATED_STATUS_L), + nonauth(OBJECT_LIST_REQUESTS, HeadListCosts.LIST_LOCATED_STATUS_L), + raw(OBJECT_METADATA_REQUESTS, HeadListCosts.GFS_FILE_PROBE_H)); } @Test public void testCostOfListLocatedStatusOnEmptyDir() throws Throwable { describe("performing listLocatedStatus on an empty dir"); - Path dir = path(getMethodName()); + Path dir = dir(methodPath()); S3AFileSystem fs = getFileSystem(); - fs.mkdirs(dir); - resetMetricDiffs(); - fs.listLocatedStatus(dir); - if (!fs.hasMetadataStore()) { - // Unguarded FS. - verifyOperationCount(2, 1); - } else { - if (fs.allowAuthoritative(dir)) { - verifyOperationCount(0, 0); - } else { - verifyOperationCount(0, 1); - } - } + verifyMetrics(() -> + fs.listLocatedStatus(dir), + raw(OBJECT_METADATA_REQUESTS, + HeadListCosts.GFS_FILE_PROBE_H), + raw(OBJECT_LIST_REQUESTS, + HeadListCosts.LIST_LOCATED_STATUS_L + HeadListCosts.GFS_DIR_L), + guarded(OBJECT_METADATA_REQUESTS, 0), + authoritative(OBJECT_LIST_REQUESTS, 0), + nonauth(OBJECT_LIST_REQUESTS, + HeadListCosts.LIST_LOCATED_STATUS_L)); } @Test public void testCostOfListLocatedStatusOnNonEmptyDir() throws Throwable { describe("performing listLocatedStatus on a non empty dir"); - Path dir = path(getMethodName() + "dir"); + Path dir = dir(methodPath()); S3AFileSystem fs = getFileSystem(); - fs.mkdirs(dir); - Path file = new Path(dir, "file.txt"); - touch(fs, file); - resetMetricDiffs(); - fs.listLocatedStatus(dir); - if (!fs.hasMetadataStore()) { - // Unguarded FS. - verifyOperationCount(0, 1); - } else { - if(fs.allowAuthoritative(dir)) { - verifyOperationCount(0, 0); - } else { - verifyOperationCount(0, 1); - } - } + Path file = file(new Path(dir, "file.txt")); + verifyMetrics(() -> + fs.listLocatedStatus(dir), + always(OBJECT_METADATA_REQUESTS, 0), + raw(OBJECT_LIST_REQUESTS, + HeadListCosts.LIST_LOCATED_STATUS_L), + authoritative(OBJECT_LIST_REQUESTS, 0), + nonauth(OBJECT_LIST_REQUESTS, + HeadListCosts.LIST_LOCATED_STATUS_L)); } @Test @@ -174,7 +246,15 @@ public void testCostOfListFilesOnFile() throws Throwable { Path file = path(getMethodName() + ".txt"); S3AFileSystem fs = getFileSystem(); touch(fs, file); - resetMetricDiffs(); + verifyMetrics(() -> + fs.listFiles(file, true), + raw(OBJECT_METADATA_REQUESTS, 1), + raw(OBJECT_LIST_REQUESTS, + HeadListCosts.LIST_LOCATED_STATUS_L), + authoritative(OBJECT_LIST_REQUESTS, 0), + nonauth(OBJECT_LIST_REQUESTS, + 1)); +/* resetMetricDiffs(); fs.listFiles(file, true); if (!fs.hasMetadataStore()) { metadataRequests.assertDiffEquals(1); @@ -184,7 +264,7 @@ public void testCostOfListFilesOnFile() throws Throwable { } else { listRequests.assertDiffEquals(1); } - } + }*/ } @Test @@ -193,17 +273,11 @@ public void testCostOfListFilesOnEmptyDir() throws Throwable { Path dir = path(getMethodName()); S3AFileSystem fs = getFileSystem(); fs.mkdirs(dir); - resetMetricDiffs(); - fs.listFiles(dir, true); - if (!fs.hasMetadataStore()) { - verifyOperationCount(2, 1); - } else { - if (fs.allowAuthoritative(dir)) { - verifyOperationCount(0, 0); - } else { - verifyOperationCount(0, 1); - } - } + verifyMetrics(() -> + fs.listFiles(dir, true), + expectHeadList(isRaw, 2, 1), + expectHeadList(isAuthoritative, 0, 0), + expectHeadList(isNonAuth, 0, 1)); } @Test @@ -241,107 +315,114 @@ public void testCostOfListFilesOnNonExistingDir() throws Throwable { @Test public void testCostOfGetFileStatusOnFile() throws Throwable { describe("performing getFileStatus on a file"); - Path simpleFile = path("simple.txt"); - S3AFileSystem fs = getFileSystem(); - touch(fs, simpleFile); - resetMetricDiffs(); - FileStatus status = fs.getFileStatus(simpleFile); + Path simpleFile = file(methodPath()); + S3AFileStatus status = verifyRawGetFileStatus(simpleFile, true, + StatusProbeEnum.ALL, + HeadListCosts.GFS_SINGLE_FILE_H, + HeadListCosts.GFS_SINGLE_FILE_L); assertTrue("not a file: " + status, status.isFile()); - if (!fs.hasMetadataStore()) { - metadataRequests.assertDiffEquals(1); - } - listRequests.assertDiffEquals(0); } - private void resetMetricDiffs() { - reset(metadataRequests, listRequests); - } - - /** - * Verify that the head and list calls match expectations, - * then reset the counters ready for the next operation. - * @param head expected HEAD count - * @param list expected LIST count - */ - private void verifyOperationCount(int head, int list) { - metadataRequests.assertDiffEquals(head); - listRequests.assertDiffEquals(list); - metadataRequests.reset(); - listRequests.reset(); - } @Test public void testCostOfGetFileStatusOnEmptyDir() throws Throwable { describe("performing getFileStatus on an empty directory"); - S3AFileSystem fs = getFileSystem(); - Path dir = path("empty"); - fs.mkdirs(dir); - resetMetricDiffs(); - S3AFileStatus status = fs.innerGetFileStatus(dir, true, - StatusProbeEnum.ALL); + Path dir = dir(methodPath()); + S3AFileStatus status = verifyRawGetFileStatus(dir, true, + StatusProbeEnum.ALL, + HeadListCosts.GFS_MARKER_H, + HeadListCosts.GFS_EMPTY_DIR_L); assertSame("not empty: " + status, Tristate.TRUE, status.isEmptyDirectory()); - - if (!fs.hasMetadataStore()) { - metadataRequests.assertDiffEquals(2); - } - listRequests.assertDiffEquals(0); - // but now only ask for the directories and the file check is skipped. - resetMetricDiffs(); - fs.innerGetFileStatus(dir, false, - StatusProbeEnum.DIRECTORIES); - if (!fs.hasMetadataStore()) { - metadataRequests.assertDiffEquals(1); - } + verifyRawGetFileStatus(dir, false, + StatusProbeEnum.DIRECTORIES, + HeadListCosts.GFS_DIR_PROBE_H, + HeadListCosts.GFS_EMPTY_DIR_L); + + // now look at isFile/isDir against the same entry + isDir(dir, true, 0, + HeadListCosts.GFS_EMPTY_DIR_L); + isFile(dir, false, + HeadListCosts.GFS_SINGLE_FILE_H, HeadListCosts.GFS_SINGLE_FILE_L); } @Test public void testCostOfGetFileStatusOnMissingFile() throws Throwable { describe("performing getFileStatus on a missing file"); - S3AFileSystem fs = getFileSystem(); - Path path = path("missing"); - resetMetricDiffs(); - intercept(FileNotFoundException.class, - () -> fs.getFileStatus(path)); - metadataRequests.assertDiffEquals(2); - listRequests.assertDiffEquals(1); + verifyRawGetFileStatusFNFE(methodPath(), false, + StatusProbeEnum.ALL, + HeadListCosts.GFS_FNFE_H, + HeadListCosts.GFS_FNFE_L); } @Test - public void testCostOfGetFileStatusOnMissingSubPath() throws Throwable { - describe("performing getFileStatus on a missing file"); - S3AFileSystem fs = getFileSystem(); - Path path = path("missingdir/missingpath"); - resetMetricDiffs(); - intercept(FileNotFoundException.class, - () -> fs.getFileStatus(path)); - metadataRequests.assertDiffEquals(2); - listRequests.assertDiffEquals(1); + public void testIsDirIsFileMissingPath() throws Throwable { + describe("performing isDir and isFile on a missing file"); + Path path = methodPath(); + // now look at isFile/isDir against the same entry + isDir(path, false, + HeadListCosts.GFS_DIR_PROBE_H, + HeadListCosts.GFS_DIR_PROBE_L); + isFile(path, false, + HeadListCosts.GFS_FILE_PROBE_H, + HeadListCosts.GFS_FILE_PROBE_L); } @Test public void testCostOfGetFileStatusOnNonEmptyDir() throws Throwable { describe("performing getFileStatus on a non-empty directory"); + Path dir = dir(methodPath()); + Path simpleFile = file(new Path(dir, "simple.txt")); + S3AFileStatus status = verifyRawGetFileStatus(dir, true, + StatusProbeEnum.ALL, + HeadListCosts.GFS_DIR_H, HeadListCosts.GFS_DIR_L); + assertEmptyDirStatus(status, Tristate.FALSE); + } + + /** + * This creates a directory with a child and then deletes it. + * The parent dir must be found and declared as empty. + */ + @Test + public void testDeleteFile() throws Throwable { + describe("performing getFileStatus on newly emptied directory"); S3AFileSystem fs = getFileSystem(); - Path dir = path("empty"); - fs.mkdirs(dir); - Path simpleFile = new Path(dir, "simple.txt"); - touch(fs, simpleFile); - resetMetricDiffs(); - S3AFileStatus status = fs.innerGetFileStatus(dir, true, - StatusProbeEnum.ALL); - if (status.isEmptyDirectory() == Tristate.TRUE) { - // erroneous state - String fsState = fs.toString(); - fail("FileStatus says directory isempty: " + status - + "\n" + ContractTestUtils.ls(fs, dir) - + "\n" + fsState); - } - if (!fs.hasMetadataStore()) { - metadataRequests.assertDiffEquals(2); - listRequests.assertDiffEquals(1); - } + // creates the marker + Path dir = dir(methodPath()); + // file creation may have deleted that marker, but it may + // still be there + Path simpleFile = file(new Path(dir, "simple.txt")); + + verifyMetrics(() -> { + fs.delete(simpleFile, false); + return "after fs.delete(simpleFile) " + metricSummary; + }, + // delete file. For keeping: that's it + probe(isRaw && isKeeping, OBJECT_METADATA_REQUESTS, + HeadListCosts.GFS_FILE_PROBE_H), + // if deleting markers, look for the parent too + probe(isRaw && isDeleting, OBJECT_METADATA_REQUESTS, + HeadListCosts.GFS_FILE_PROBE_H + HeadListCosts.GFS_DIR_PROBE_H), + raw(OBJECT_LIST_REQUESTS, + HeadListCosts.GFS_FILE_PROBE_L + HeadListCosts.GFS_DIR_PROBE_L), + always(DIRECTORIES_DELETED, 0), + always(FILES_DELETED, 1), + + // keeping: create no parent dirs or delete parents + keeping(DIRECTORIES_CREATED, 0), + keeping(OBJECT_DELETE_REQUESTS, HeadListCosts.DELETE_OBJECT_REQUEST), + + // deleting: create a parent and delete any of its parents + deleting(DIRECTORIES_CREATED, 1), + deleting(OBJECT_DELETE_REQUESTS, + HeadListCosts.DELETE_OBJECT_REQUEST + + HeadListCosts.DELETE_MARKER_REQUEST) + ); + // there is an empty dir for a parent + S3AFileStatus status = verifyRawGetFileStatus(dir, true, + StatusProbeEnum.ALL, HeadListCosts.GFS_DIR_H, HeadListCosts.GFS_DIR_L); + assertEmptyDirStatus(status, Tristate.TRUE); } @Test @@ -361,19 +442,18 @@ public void testCostOfCopyFromLocalFile() throws Throwable { byte[] data = dataset(len, 'A', 'Z'); writeDataset(localFS, localPath, data, len, 1024, true); S3AFileSystem s3a = getFileSystem(); - MetricDiff copyLocalOps = new MetricDiff(s3a, - INVOCATION_COPY_FROM_LOCAL_FILE); - MetricDiff putRequests = new MetricDiff(s3a, - OBJECT_PUT_REQUESTS); - MetricDiff putBytes = new MetricDiff(s3a, - OBJECT_PUT_BYTES); - - Path remotePath = path("copied"); - s3a.copyFromLocalFile(false, true, localPath, remotePath); + + + Path remotePath = methodPath(); + + verifyMetrics(() -> { + s3a.copyFromLocalFile(false, true, localPath, remotePath); + return "copy"; + }, + always(INVOCATION_COPY_FROM_LOCAL_FILE, 1), + always(OBJECT_PUT_REQUESTS, 1), + always(OBJECT_PUT_BYTES, len)); verifyFileContents(s3a, remotePath, data); - copyLocalOps.assertDiffEquals(1); - putRequests.assertDiffEquals(1); - putBytes.assertDiffEquals(len); // print final stats LOG.info("Filesystem {}", s3a); } finally { @@ -381,133 +461,129 @@ public void testCostOfCopyFromLocalFile() throws Throwable { } } - private boolean reset(MetricDiff... diffs) { - for (MetricDiff diff : diffs) { - diff.reset(); - } - return true; + @Test + public void testDirMarkersSubdir() throws Throwable { + describe("verify cost of deep subdir creation"); + + Path subDir = new Path(methodPath(), "1/2/3/4/5/6"); + // one dir created, possibly a parent removed + verifyMetrics(() -> { + mkdirs(subDir); + return "after mkdir(subDir) " + metricSummary; + }, + always(DIRECTORIES_CREATED, 1), + always(DIRECTORIES_DELETED, 0), + keeping(OBJECT_DELETE_REQUESTS, 0), + keeping(FAKE_DIRECTORIES_DELETED, 0), + deleting(OBJECT_DELETE_REQUESTS, HeadListCosts.DELETE_MARKER_REQUEST), + // delete all possible fake dirs above the subdirectory + deleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(subDir) - 1)); } @Test - public void testFakeDirectoryDeletion() throws Throwable { - describe("Verify whether create file works after renaming a file. " - + "In S3, rename deletes any fake directories as a part of " - + "clean up activity"); - S3AFileSystem fs = getFileSystem(); + public void testDirMarkersFileCreation() throws Throwable { + describe("verify cost of file creation"); + + Path srcBaseDir = dir(methodPath()); + + Path srcDir = dir(new Path(srcBaseDir, "1/2/3/4/5/6")); + + // creating a file should trigger demise of the src dir marker + // unless markers are being kept + + verifyMetrics(() -> { + file(new Path(srcDir, "source.txt")); + return "after touch(fs, srcFilePath) " + metricSummary; + }, + always(DIRECTORIES_CREATED, 0), + always(DIRECTORIES_DELETED, 0), + // keeping: no delete operations. + keeping(OBJECT_DELETE_REQUESTS, 0), + keeping(FAKE_DIRECTORIES_DELETED, 0), + // delete all possible fake dirs above the file + deleting(OBJECT_DELETE_REQUESTS, 1), + deleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(srcDir))); + } - Path srcBaseDir = path("src"); - mkdirs(srcBaseDir); - 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); - - // 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) " + 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.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) " + summary; - deleteRequests.assertDiffEquals(state, 1); - directoriesCreated.assertDiffEquals(state, 0); - directoriesDeleted.assertDiffEquals(state, 0); - fakeDirectoriesDeleted.assertDiffEquals(state, srcDirDepth); - - reset.call(); + @Test + public void testRenameFileToDifferentDirectory() throws Throwable { + describe("rename a file to a different directory, " + + "keeping the source dir present"); + S3AFileSystem fs = getFileSystem(); - // 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) " + summary; + Path baseDir = dir(methodPath()); - int destDirDepth = directoriesInPath(destDir); - directoriesCreated.assertDiffEquals(state, 1); - deleteRequests.assertDiffEquals(state, 1); - directoriesDeleted.assertDiffEquals(state, 0); - fakeDirectoriesDeleted.assertDiffEquals(state, destDirDepth - 1); + Path srcDir = new Path(baseDir, "1/2/3/4/5/6"); + final Path srcFilePath = file(new Path(srcDir, "source.txt")); // 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); + // are different object instances and so equals() rather than == + // is + Path parent2 = srcFilePath.getParent(); + Path srcFile2 = file(new Path(parent2, "source2.txt")); + Assertions.assertThat(srcDir) + .isNotSameAs(parent2); + Assertions.assertThat(srcFilePath.getParent()) + .isEqualTo(srcFile2.getParent()); - reset.call(); + // create a directory tree, expect the dir to be created and + // possibly a request to delete all parent directories made. + Path destBaseDir = new Path(baseDir, "dest"); + Path destDir = dir(new Path(destBaseDir, "a/b/c/d")); + Path destFilePath = new Path(destDir, "dest.txt"); // rename the source file to the destination file. - // this tests the file rename path, not the dir rename path + // this tests file rename, not dir rename // as srcFile2 exists, the parent dir of srcFilePath must not be created. - fs.rename(srcFilePath, destFilePath); - 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); - - // these asserts come after the checks on iop counts, so they don't - // interfere + verifyMetrics(() -> + execRename(srcFilePath, destFilePath), + raw(OBJECT_METADATA_REQUESTS, + HeadListCosts.RENAME_SINGLE_FILE_RENAME_H), + raw(OBJECT_LIST_REQUESTS, + HeadListCosts.RENAME_SINGLE_FILE_RENAME_DIFFERENT_DIR_L), + always(DIRECTORIES_CREATED, 0), + always(DIRECTORIES_DELETED, 0), + // keeping: only the core delete operation is issued. + keeping(OBJECT_DELETE_REQUESTS, HeadListCosts.DELETE_OBJECT_REQUEST), + keeping(FAKE_DIRECTORIES_DELETED, 0), + // deleting: delete any fake marker above the destination. + deleting(OBJECT_DELETE_REQUESTS, + HeadListCosts.DELETE_OBJECT_REQUEST + HeadListCosts.DELETE_MARKER_REQUEST), + deleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(destDir))); + 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); + /** + * Same directory rename is lower cost as there's no need to + * look for the parent dir of the dest path or worry about + * deleting markers. + */ + @Test + public void testRenameSameDirectory() throws Throwable { + describe("rename a file to a different directory, " + + "keeping the source dir present"); - // 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); - } + Path baseDir = dir(methodPath()); + final Path sourceFile = file(new Path(baseDir, "source.txt")); - private int directoriesInPath(Path path) { - return path.isRoot() ? 0 : 1 + directoriesInPath(path.getParent()); + // create a new source file. + // Explicitly use a new path object to guarantee that the parent paths + // are different object instances and so equals() rather than == + // is + Path parent2 = sourceFile.getParent(); + Path destFile = new Path(parent2, "dest"); + verifyMetrics(() -> + execRename(sourceFile, destFile), + raw(OBJECT_METADATA_REQUESTS, HeadListCosts.RENAME_SINGLE_FILE_RENAME_H), + raw(OBJECT_LIST_REQUESTS, HeadListCosts.GFS_FNFE_L), + always(OBJECT_COPY_REQUESTS, 1), + always(DIRECTORIES_CREATED, 0), + always(OBJECT_DELETE_REQUESTS, HeadListCosts.DELETE_OBJECT_REQUEST), + always(FAKE_DIRECTORIES_DELETED, 0)); } @Test @@ -518,56 +594,44 @@ public void testCostOfRootRename() throws Throwable { // 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 src = file(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); + + verifyMetrics(() -> { + fs.rename(src, dest); + return "after fs.rename(/src,/dest) " + metricSummary; + }, + // TWO HEAD for exists, one for source MD in copy + raw(OBJECT_METADATA_REQUESTS, + HeadListCosts.RENAME_SINGLE_FILE_RENAME_H), + raw(OBJECT_LIST_REQUESTS, + HeadListCosts.GFS_FNFE_L), + // here we expect there to be no fake directories + always(DIRECTORIES_CREATED, 0), + // one for the renamed file only + always(OBJECT_DELETE_REQUESTS, + HeadListCosts.DELETE_OBJECT_REQUEST), + // no directories are deleted: This is root + always(DIRECTORIES_DELETED, 0), + // no fake directories are deleted: This is root + always(FAKE_DIRECTORIES_DELETED, 0), + always(FILES_DELETED, 1)); // delete that destination file, assert only the file delete was issued - reset(deleteRequests, directoriesCreated, directoriesDeleted, - fakeDirectoriesDeleted); + verifyMetrics(() -> { + fs.delete(dest, false); + return "after fs.delete(/dest) " + metricSummary; + }, + always(DIRECTORIES_CREATED, 0), + always(DIRECTORIES_DELETED, 0), + always(FAKE_DIRECTORIES_DELETED, 0), + always(FILES_DELETED, 1), + always(OBJECT_DELETE_REQUESTS, HeadListCosts.DELETE_OBJECT_REQUEST), + raw(OBJECT_METADATA_REQUESTS, HeadListCosts.GFS_FILE_PROBE_H), + raw(OBJECT_LIST_REQUESTS, 0) /* no need to look at parent. */ + ); - 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); @@ -576,73 +640,126 @@ public String toString() { @Test public void testDirProbes() throws Throwable { - describe("Test directory probe cost -raw only"); + describe("Test directory probe cost"); + assumeUnguarded(); S3AFileSystem fs = getFileSystem(); - assume("Unguarded FS only", !fs.hasMetadataStore()); - String dir = "testEmptyDirHeadProbe"; - Path emptydir = path(dir); // Create the empty directory. - fs.mkdirs(emptydir); + Path emptydir = dir(methodPath()); - // metrics and assertions. - resetMetricDiffs(); + // head probe fails + verifyRawGetFileStatusFNFE(emptydir, false, + StatusProbeEnum.HEAD_ONLY, + HeadListCosts.GFS_FILE_PROBE_H, + HeadListCosts.GFS_FILE_PROBE_L); + + // a LIST will find it and declare as empty + S3AFileStatus status = verifyRawGetFileStatus(emptydir, true, + StatusProbeEnum.LIST_ONLY, 0, + HeadListCosts.GFS_EMPTY_DIR_L); + assertEmptyDirStatus(status, Tristate.TRUE); - intercept(FileNotFoundException.class, () -> - fs.innerGetFileStatus(emptydir, false, - StatusProbeEnum.HEAD_ONLY)); - verifyOperationCount(1, 0); - - // a LIST will find it -but it doesn't consider it an empty dir. - S3AFileStatus status = fs.innerGetFileStatus(emptydir, true, - StatusProbeEnum.LIST_ONLY); - verifyOperationCount(0, 1); - Assertions.assertThat(status) - .describedAs("LIST output is not considered empty") - .matches(s -> !s.isEmptyDirectory().equals(Tristate.TRUE), "is empty"); - - // finally, skip all probes and expect no operations toThere are - // take place - intercept(FileNotFoundException.class, () -> - fs.innerGetFileStatus(emptydir, false, - EnumSet.noneOf(StatusProbeEnum.class))); - verifyOperationCount(0, 0); + // skip all probes and expect no operations to take place + verifyRawGetFileStatusFNFE(emptydir, false, + EnumSet.noneOf(StatusProbeEnum.class), + 0, 0); // now add a trailing slash to the key and use the // deep internal s3GetFileStatus method call. String emptyDirTrailingSlash = fs.pathToKey(emptydir.getParent()) - + "/" + dir + "/"; + + "/" + emptydir.getName() + "/"; // A HEAD request does not probe for keys with a trailing / - intercept(FileNotFoundException.class, () -> + verifyRawHeadListIntercepting(FileNotFoundException.class, "", + 0, 0, () -> fs.s3GetFileStatus(emptydir, emptyDirTrailingSlash, - StatusProbeEnum.HEAD_ONLY, null)); - verifyOperationCount(0, 0); + StatusProbeEnum.HEAD_ONLY, null, false)); // but ask for a directory marker and you get the entry - status = fs.s3GetFileStatus(emptydir, - emptyDirTrailingSlash, - StatusProbeEnum.DIR_MARKER_ONLY, null); - verifyOperationCount(1, 0); + status = verifyRawHeadList(0, HeadListCosts.GFS_EMPTY_DIR_L, () -> + fs.s3GetFileStatus(emptydir, + emptyDirTrailingSlash, + StatusProbeEnum.LIST_ONLY, + null, + true)); assertEquals(emptydir, status.getPath()); + assertEmptyDirStatus(status, Tristate.TRUE); + } + + /** + * Assert the empty directory status of a file is as expected. + * @param status status to probe. + * @param expected expected value + */ + protected void assertEmptyDirStatus(final S3AFileStatus status, + final Tristate expected) { + Assertions.assertThat(status.isEmptyDirectory()) + .describedAs(dynamicDescription(() -> + "FileStatus says directory is not empty: " + status + + "\n" + ContractTestUtils.ls(getFileSystem(), status.getPath()))) + .isEqualTo(expected); } @Test public void testCreateCost() throws Throwable { describe("Test file creation cost -raw only"); - S3AFileSystem fs = getFileSystem(); - assume("Unguarded FS only", !fs.hasMetadataStore()); - resetMetricDiffs(); - Path testFile = path("testCreateCost"); - + assumeUnguarded(); + Path testFile = methodPath(); // when overwrite is false, the path is checked for existence. - try (FSDataOutputStream out = fs.create(testFile, false)) { - verifyOperationCount(2, 1); - } - + create(testFile, false, HeadListCosts.GFS_FNFE_H, HeadListCosts.GFS_FNFE_L); // but when true: only the directory checks take place. - try (FSDataOutputStream out = fs.create(testFile, true)) { - verifyOperationCount(1, 1); - } + create(testFile, true, 0, HeadListCosts.GFS_FNFE_L); + } + + @Test + public void testCreateCostFileExists() throws Throwable { + describe("Test cost of create file failing with existing file"); + assumeUnguarded(); + Path testFile = file(methodPath()); + + // now there is a file there, an attempt with overwrite == false will + // fail on the first HEAD. + verifyRawHeadListIntercepting(FileAlreadyExistsException.class, "", + HeadListCosts.GFS_FILE_PROBE_H, 0, + () -> file(testFile, false)); + } + @Test + public void testCreateCostDirExists() throws Throwable { + describe("Test cost of create file failing with existing dir"); + assumeUnguarded(); + Path testFile = dir(methodPath()); + + // now there is a file there, an attempt with overwrite == false will + // fail on the first HEAD. + verifyRawHeadListIntercepting(FileAlreadyExistsException.class, "", + HeadListCosts.GFS_MARKER_H, HeadListCosts.GFS_EMPTY_DIR_L, + () -> file(testFile, false)); + } + + /** + * Use the builder API. + * This always looks for a parent unless the caller says otherwise. + */ + @Test + public void testCreateBuilder() throws Throwable { + describe("Test builder file creation cost -raw only"); + assumeUnguarded(); + Path testFile = methodPath(); + dir(testFile.getParent()); + + // builder defaults to looking for parent existence (non-recursive) + buildFile(testFile, false, false, + HeadListCosts.GFS_FILE_PROBE_H, // destination file + HeadListCosts.GFS_DIR_PROBE_L * 2); // destination file and parent dir + // recursive = false and overwrite=true: + // only make sure the dest path isn't a directory. + buildFile(testFile, true, true, + HeadListCosts.GFS_DIR_PROBE_H, HeadListCosts.GFS_DIR_PROBE_L); + + // now there is a file there, an attempt with overwrite == false will + // fail on the first HEAD. + verifyRawHeadListIntercepting(FileAlreadyExistsException.class, "", + HeadListCosts.GFS_FILE_PROBE_H, 0, () -> + buildFile(testFile, false, true, HeadListCosts.GFS_FILE_PROBE_H, 0)); } @Test @@ -688,4 +805,342 @@ public void testCostOfGlobStatusNoSymlinkResolution() throws Throwable { // no additional operations from symlink resolution verifyOperationCount(2, 2); } + + /** + * Create then close the file. + * @param path path + * @param overwrite overwrite flag + * @param head expected head count + * @param list expected list count + * @return path to new object. + */ + private Path create(Path path, boolean overwrite, + int head, int list) throws Exception { + return verifyRawHeadList(head, list, () -> + file(path, overwrite)); + } + + /** + * Create then close the file through the builder API. + * @param path path + * @param overwrite overwrite flag + * @param recursive true == skip parent existence check + * @param head expected head count + * @param list expected list count + * @return path to new object. + */ + private Path buildFile(Path path, + boolean overwrite, + boolean recursive, + int head, + int list) throws IOException { + resetStatistics(); + verifyRawHeadList(head, list, () -> { + FSDataOutputStreamBuilder builder = getFileSystem().createFile(path) + .overwrite(overwrite); + if (recursive) { + builder.recursive(); + } + builder.build().close(); + }); + return path; + } + + /** + * Create a directory, returning its path. + * @param p path to dir. + * @return path of new dir + */ + private Path dir(Path p) throws IOException { + mkdirs(p); + return p; + } + + /** + * Create a file, returning its path. + * @param p path to file. + * @return path of new file + */ + private Path file(Path p) throws IOException { + return file(p, true); + } + + /** + * Create a file, returning its path. + * @param path path to file. + * @param overwrite overwrite flag + * @return path of new file + */ + private Path file(Path path, final boolean overwrite) + throws IOException { + getFileSystem().create(path, overwrite).close(); + return path; + } + + + /** + * Execute rename, returning the current metrics. + * For use in l-expressions. + * @param source source path. + * @param dest dest path + * @return a string for exceptions. + */ + public String execRename(final Path source, + final Path dest) throws IOException { + getFileSystem().rename(source, dest); + return String.format("rename(%s, %s): %s", dest, source, metricSummary); + } + + /** + * How many directories are in a path? + * @param path path to probe. + * @return the number of entries below root this path is + */ + private int directoriesInPath(Path path) { + return path.isRoot() ? 0 : 1 + directoriesInPath(path.getParent()); + } + + /** + * Reset all the metrics being tracked. + */ + private void resetStatistics() { + costValidator.resetMetricDiffs(); + } + + /** + * Execute a closure and verify the metrics. + * @param eval closure to evaluate + * @param expected varargs list of expected diffs + * @param return type. + * @return the result of the evaluation + */ + private T verifyMetrics( + Callable eval, + OperationCostValidator.ExpectedProbe... expected) throws Exception { + return costValidator.verify(eval, expected); + + } + + /** + * Execute a closure, expecting an exception. + * Verify the metrics after the exception has been caught and + * validated. + * @param clazz type of exception + * @param text text to look for in exception (optional) + * @param eval closure to evaluate + * @param expected varargs list of expected diffs + * @param return type of closure + * @param exception type + * @return the exception caught. + * @throws Exception any other exception + */ + private E verifyMetricsIntercepting( + Class clazz, + String text, + Callable eval, + OperationCostValidator.ExpectedProbe... expected) throws Exception { + return costValidator.verifyIntercepting(clazz, text, eval, expected); + } + + /** + * Execute a closure expecting an exception. + * @param clazz type of exception + * @param text text to look for in exception (optional) + * @param head expected head request count. + * @param list expected list request count. + * @param eval closure to evaluate + * @param return type of closure + * @param exception type + * @return the exception caught. + * @throws Exception any other exception + */ + private E verifyRawHeadListIntercepting( + Class clazz, + String text, + int head, + int list, + Callable eval) throws Exception { + return verifyMetricsIntercepting(clazz, text, eval, + raw(OBJECT_METADATA_REQUESTS, head), + raw(OBJECT_LIST_REQUESTS, list)); + } + + + /** + * Verify that the head and list calls match expectations, + * then reset the counters ready for the next operation. + * @param head expected HEAD count + * @param list expected LIST count + * @return a probe list + */ + private OperationCostValidator.ExpectedProbe + expectHeadList(boolean enabled, + int head, int list) { + return probes(enabled, + probe(OBJECT_METADATA_REQUESTS, head), + probe(OBJECT_LIST_REQUESTS, list)); + } + + /** + * Execute a closure expecting a specific number of HEAD/LIST calls + * on raw S3 stores only. + * @param head expected head request count. + * @param list expected list request count. + * @param eval closure to evaluate + * @param return type of closure + * @return the result of the evaluation + */ + private T verifyRawHeadList( + int head, + int list, + Callable eval) throws Exception { + return verifyMetrics(eval, + expectHeadList(isRaw, head, list)); + } + + /** + * Execute innerGetFileStatus for the given probes + * and expect in raw FS to have the specific HEAD/LIST count. + */ + public S3AFileStatus verifyRawGetFileStatus(final Path path, + boolean needEmptyDirectoryFlag, + Set probes, int head, int list) throws Exception { + return verifyRawHeadList(head, list, () -> + getFileSystem().innerGetFileStatus(path, needEmptyDirectoryFlag, + probes)); + } + + /** + * Execute innerGetFileStatus for the given probes and expect failure + * and expect in raw FS to have the specific HEAD/LIST count. + */ + public void verifyRawGetFileStatusFNFE(final Path path, + boolean needEmptyDirectoryFlag, + Set probes, int head, int list) throws Exception { + verifyRawHeadListIntercepting(FileNotFoundException.class, "", + head, list, () -> + getFileSystem().innerGetFileStatus(path, needEmptyDirectoryFlag, + probes)); + } + + /** + * Probe for a path being a directory. + * Metrics are only checked on unguarded stores. + * @param path path + * @param expected expected outcome + * @param head head count (unguarded) + * @param list listCount (unguarded) + */ + private void isDir(Path path, boolean expected, + int head, int list) throws Exception { + boolean b = verifyRawHeadList(head, list, () -> + getFileSystem().isDirectory(path)); + Assertions.assertThat(b) + .describedAs("isDirectory(%s)", path) + .isEqualTo(expected); + } + + /** + * Probe for a path being a file. + * Metrics are only checked on unguarded stores. + * @param path path + * @param expected expected outcome + * @param head head count (unguarded) + * @param list listCount (unguarded) + */ + private void isFile(Path path, boolean expected, + int head, int list) throws Exception { + boolean b = verifyRawHeadList(head, list, () -> + getFileSystem().isFile(path)); + Assertions.assertThat(b) + .describedAs("isFile(%s)", path) + .isEqualTo(expected); + } + + /** + * A metric diff which must always hold. + * @param Statistic metric source + * @param expected expected value. + * @return the diff. + */ + private OperationCostValidator.ExpectedProbe always( + final Statistic Statistic, final int expected) { + return probe(Statistic, expected); + } + + /** + * A metric diff which must hold when the fs is unguarded. + * @param Statistic metric source + * @param expected expected value. + * @return the diff. + */ + private OperationCostValidator.ExpectedProbe raw( + final Statistic Statistic, final int expected) { + return probe(isRaw, Statistic, expected); + } + + /** + * A metric diff which must hold when the fs is guarded. + * @param Statistic metric source + * @param expected expected value. + * @return the diff. + */ + private OperationCostValidator.ExpectedProbe guarded( + final Statistic Statistic, + final int expected) { + return probe(isGuarded, Statistic, expected); + } + + /** + * A metric diff which must hold when the fs is guarded + authoritative. + * @param Statistic metric source + * @param expected expected value. + * @return the diff. + */ + private OperationCostValidator.ExpectedProbe authoritative(final Statistic Statistic, + final int expected) { + return probe(isAuthoritative, Statistic, expected); + } + + /** + * A metric diff which must hold when the fs is guarded + authoritative. + * @param Statistic metric source + * @param expected expected value. + * @return the diff. + */ + private OperationCostValidator.ExpectedProbe nonauth(final Statistic Statistic, + final int expected) { + return probe(isNonAuth, Statistic, expected); + } + + /** + * A metric diff which must hold when the fs is keeping markers + * @param Statistic metric source + * @param expected expected value. + * @return the diff. + */ + private OperationCostValidator.ExpectedProbe keeping(final Statistic Statistic, + final int expected) { + return probe(isKeeping, Statistic, expected); + } + + /** + * A metric diff which must hold when the fs is keeping markers + * @param Statistic metric source + * @param expected expected value. + * @return the diff. + */ + private OperationCostValidator.ExpectedProbe deleting( + final Statistic Statistic, + final int expected) { + return probe(isDeleting, Statistic, expected); + } + + + /** + * A special object whose toString() value is the current + * state of the metrics. + */ + private final Object metricSummary = costValidator; + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java index 3fd70be931997..01a14ef8e9300 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARemoteFileChanged.java @@ -326,7 +326,7 @@ protected Path path() throws IOException { * @return a number >= 0. */ private int getFileStatusHeadCount() { - return authMode ? 0 : 1; + return authMode ? 0 : 0; } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java index 32ead7f3fed71..1b3f62d185b61 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java @@ -56,6 +56,8 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.contract.ContractTestUtils.writeTextFile; import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH; +import static org.apache.hadoop.fs.s3a.Constants.CHANGE_DETECT_MODE; +import static org.apache.hadoop.fs.s3a.Constants.CHANGE_DETECT_MODE_NONE; import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_METADATASTORE_METADATA_TTL; import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE; import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_METADATA_TTL; @@ -169,12 +171,16 @@ protected Configuration createConfiguration() { RETRY_LIMIT, RETRY_INTERVAL, S3GUARD_CONSISTENCY_RETRY_INTERVAL, - S3GUARD_CONSISTENCY_RETRY_LIMIT); + S3GUARD_CONSISTENCY_RETRY_LIMIT, + CHANGE_DETECT_MODE, + METADATASTORE_METADATA_TTL); conf.setInt(RETRY_LIMIT, 3); conf.setInt(S3GUARD_CONSISTENCY_RETRY_LIMIT, 3); + conf.set(CHANGE_DETECT_MODE, CHANGE_DETECT_MODE_NONE); final String delay = "10ms"; conf.set(RETRY_INTERVAL, delay); conf.set(S3GUARD_CONSISTENCY_RETRY_INTERVAL, delay); + conf.set(METADATASTORE_METADATA_TTL, delay); return conf; } @@ -232,12 +238,13 @@ private S3AFileSystem createGuardedFS(boolean authoritativeMode) URI uri = testFS.getUri(); removeBaseAndBucketOverrides(uri.getHost(), config, + CHANGE_DETECT_MODE, METADATASTORE_AUTHORITATIVE, METADATASTORE_METADATA_TTL, AUTHORITATIVE_PATH); config.setBoolean(METADATASTORE_AUTHORITATIVE, authoritativeMode); config.setLong(METADATASTORE_METADATA_TTL, - DEFAULT_METADATASTORE_METADATA_TTL); + 5_000); final S3AFileSystem gFs = createFS(uri, config); // set back the same metadata store instance gFs.setMetadataStore(realMs); @@ -857,7 +864,7 @@ private void verifyFileStatusAsExpected(final String firstText, expectedLength, guardedLength); } else { assertEquals( - "File length in authoritative table with " + stats, + "File length in non-authoritative table with " + stats, expectedLength, guardedLength); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index aa5979dbf751e..e47c70ef42353 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -618,6 +618,12 @@ public static Configuration prepareTestConfiguration(final Configuration conf) { // add this so that even on tests where the FS is shared, // the FS is always "magic" conf.setBoolean(MAGIC_COMMITTER_ENABLED, true); + String directoryRetention = getTestProperty( + conf, + DIRECTORY_MARKER_POLICY, + DEFAULT_DIRECTORY_MARKER_POLICY); + conf.set(DIRECTORY_MARKER_POLICY, directoryRetention); + return conf; } @@ -882,7 +888,8 @@ public static T terminateService(final T service) { public static S3AFileStatus getStatusWithEmptyDirFlag( final S3AFileSystem fs, final Path dir) throws IOException { - return fs.innerGetFileStatus(dir, true, StatusProbeEnum.ALL); + return fs.innerGetFileStatus(dir, true, + StatusProbeEnum.ALL); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java index e90518a9cbd0f..34a275b580f25 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AGetFileStatus.java @@ -76,11 +76,15 @@ public void testFakeDirectory() throws Exception { String key = path.toUri().getPath().substring(1); when(s3.getObjectMetadata(argThat(correctGetMetadataRequest(BUCKET, key)))) .thenThrow(NOT_FOUND); - ObjectMetadata meta = new ObjectMetadata(); - meta.setContentLength(0L); - when(s3.getObjectMetadata(argThat( - correctGetMetadataRequest(BUCKET, key + "/")) - )).thenReturn(meta); + String keyDir = key + "/"; + ListObjectsV2Result listResult = new ListObjectsV2Result(); + S3ObjectSummary objectSummary = new S3ObjectSummary(); + objectSummary.setKey(keyDir); + objectSummary.setSize(0L); + listResult.getObjectSummaries().add(objectSummary); + when(s3.listObjectsV2(argThat( + matchListV2Request(BUCKET, keyDir)) + )).thenReturn(listResult); FileStatus stat = fs.getFileStatus(path); assertNotNull(stat); assertEquals(fs.makeQualified(path), stat.getPath()); @@ -161,4 +165,14 @@ private ArgumentMatcher correctGetMetadataRequest( && request.getBucketName().equals(bucket) && request.getKey().equals(key); } + + private ArgumentMatcher matchListV2Request( + String bucket, String key) { + return (ListObjectsV2Request request) -> { + return request != null + && request.getBucketName().equals(bucket) + && request.getPrefix().equals(key); + }; + } + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java index a8e7a57057605..2848fb70b6d61 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java @@ -410,8 +410,7 @@ public void checkBasicFileOperations() throws Throwable { // this is HEAD + "/" on S3; get on S3Guard auth when the path exists, - accessDeniedIf(!s3guard, () -> - readonlyFS.listStatus(emptyDir)); + readonlyFS.listStatus(emptyDir); // a recursive list of the no-read-directory works because // there is no directory marker, it becomes a LIST call. @@ -421,14 +420,9 @@ public void checkBasicFileOperations() throws Throwable { // and so working. readonlyFS.getFileStatus(noReadDir); - // empty dir checks work when guarded because even in non-auth mode - // there are no checks for directories being out of date - // without S3, the HEAD path + "/" is blocked - accessDeniedIf(!s3guard, () -> - readonlyFS.getFileStatus(emptyDir)); - + readonlyFS.getFileStatus(emptyDir); // now look at a file; the outcome depends on the mode. - accessDeniedIf(!guardedInAuthMode, () -> + accessDeniedIf(!s3guard, () -> readonlyFS.getFileStatus(subdirFile)); // irrespective of mode, the attempt to read the data will fail. @@ -443,7 +437,7 @@ public void checkBasicFileOperations() throws Throwable { // This means that permissions on the file do not get checked. // See: HADOOP-16464. Optional optIn = accessDeniedIf( - !guardedInAuthMode, () -> readonlyFS.open(emptyFile)); + !s3guard, () -> readonlyFS.open(emptyFile)); if (optIn.isPresent()) { try (FSDataInputStream is = optIn.get()) { Assertions.assertThat(is.read()) @@ -461,17 +455,17 @@ public void checkGlobOperations() throws Throwable { describe("Glob Status operations"); // baseline: the real filesystem on a subdir globFS(getFileSystem(), subdirFile, null, false, 1); - // a file fails if not in auth mode - globFS(readonlyFS, subdirFile, null, !guardedInAuthMode, 1); + // a file fails if not guarded + globFS(readonlyFS, subdirFile, null, !s3guard, 1); // empty directories don't fail. - FileStatus[] st = globFS(readonlyFS, emptyDir, null, !s3guard, 1); + FileStatus[] st = globFS(readonlyFS, emptyDir, null, false, 1); if (s3guard) { assertStatusPathEquals(emptyDir, st); } st = globFS(readonlyFS, noReadWildcard, - null, !s3guard, 2); + null, false, 2); if (s3guard) { Assertions.assertThat(st) .extracting(FileStatus::getPath) @@ -481,12 +475,12 @@ public void checkGlobOperations() throws Throwable { // there is precisely one .docx file (subdir2File2.docx) globFS(readonlyFS, new Path(noReadDir, "*/*.docx"), - null, !s3guard, 1); + null, false, 1); // there are no .doc files. globFS(readonlyFS, new Path(noReadDir, "*/*.doc"), - null, !s3guard, 0); + null, false, 0); globFS(readonlyFS, noReadDir, EVERYTHING, false, 1); // and a filter without any wildcarded pattern only finds @@ -513,17 +507,14 @@ public void checkSingleThreadedLocatedFileStatus() throws Throwable { true, HIDDEN_FILE_FILTER, true); - accessDeniedIf(!s3guard, - () -> fetcher.getFileStatuses()) - .ifPresent(stats -> { - Assertions.assertThat(stats) - .describedAs("result of located scan").flatExtracting(FileStatus::getPath) - .containsExactlyInAnyOrder( - emptyFile, - subdirFile, - subdir2File1, - subdir2File2); - }); + Assertions.assertThat(fetcher.getFileStatuses()) + .describedAs("result of located scan") + .flatExtracting(FileStatus::getPath) + .containsExactlyInAnyOrder( + emptyFile, + subdirFile, + subdir2File1, + subdir2File2); } /** @@ -542,15 +533,11 @@ public void checkLocatedFileStatusFourThreads() throws Throwable { true, EVERYTHING, true); - accessDeniedIf(!s3guard, - () -> fetcher.getFileStatuses()) - .ifPresent(stats -> { - Assertions.assertThat(stats) - .describedAs("result of located scan") - .isNotNull() - .flatExtracting(FileStatus::getPath) - .containsExactlyInAnyOrder(subdirFile, subdir2File1); - }); + Assertions.assertThat(fetcher.getFileStatuses()) + .describedAs("result of located scan") + .isNotNull() + .flatExtracting(FileStatus::getPath) + .containsExactlyInAnyOrder(subdirFile, subdir2File1); } /** @@ -567,7 +554,7 @@ public void checkLocatedFileStatusScanFile() throws Throwable { true, TEXT_FILE, true); - accessDeniedIf(!guardedInAuthMode, + accessDeniedIf(!s3guard, () -> fetcher.getFileStatuses()) .ifPresent(stats -> { Assertions.assertThat(stats) @@ -631,19 +618,16 @@ public void checkLocatedFileStatusNonexistentPath() throws Throwable { */ public void checkDeleteOperations() throws Throwable { describe("Testing delete operations"); - - if (!authMode) { - // unguarded or non-auth S3Guard to fail on HEAD + / - accessDenied(() -> readonlyFS.delete(emptyDir, true)); + readonlyFS.delete(emptyDir, true); + if (!s3guard) { // to fail on HEAD accessDenied(() -> readonlyFS.delete(emptyFile, true)); } else { - // auth mode checks DDB for status and then issues the DELETE - readonlyFS.delete(emptyDir, true); + // checks DDB for status and then issues the DELETE readonlyFS.delete(emptyFile, true); } - // this will succeed for both as there is no subdir marker. + // this will succeed for both readonlyFS.delete(subDir, true); // after which it is not there fileNotFound(() -> readonlyFS.getFileStatus(subDir)); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java new file mode 100644 index 0000000000000..362fcf2f6e6a6 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java @@ -0,0 +1,120 @@ +/* + * 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.util.Arrays; +import java.util.Collection; +import java.util.function.Predicate; + +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.test.AbstractHadoopTestBase; + +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY; + +@RunWith(Parameterized.class) +public class TestDirectoryMarkerPolicy extends AbstractHadoopTestBase { + + @Parameterized.Parameters(name = "{0}") + public static Collection data() { + return Arrays.asList(new Object[][]{ + { + DirectoryPolicy.MarkerPolicy.Delete, + failIfInvoked, + false, false + }, + { + DirectoryPolicy.MarkerPolicy.Keep, + failIfInvoked, + true, true + }, + { + DirectoryPolicy.MarkerPolicy.Authoritative, + authPathOnly, + false, true + }}); + } + + private final DirectoryPolicyImpl retention; + private final boolean expectNonAuthDelete; + private final boolean expectAuthDelete; + + public TestDirectoryMarkerPolicy( + final DirectoryPolicy.MarkerPolicy markerPolicy, + final Predicate authoritativeness, + final boolean expectNonAuthDelete, + final boolean expectAuthDelete) { + this.retention = retention(markerPolicy, authoritativeness); + this.expectNonAuthDelete = expectNonAuthDelete; + this.expectAuthDelete = expectAuthDelete; + } + + /** + * Create a new retention policy. + * @param markerPolicy policy option + * @param authoritativeness predicate for determining if + * a path is authoritative. + * @return the retention policy. + */ + private DirectoryPolicyImpl retention( + DirectoryPolicy.MarkerPolicy markerPolicy, + Predicate authoritativeness) { + Configuration c = new Configuration(false); + c.set(DIRECTORY_MARKER_POLICY, markerPolicy.name()); + return new DirectoryPolicyImpl(c, authoritativeness); + } + + private static final Predicate authPathOnly = + (p) -> p.toUri().getPath().startsWith("/auth/"); + + private static final Predicate failIfInvoked = (p) -> { + throw new RuntimeException("failed"); + }; + + private final Path nonAuthPath = new Path("s3a://bucket/nonauth/data"); + private final Path authPath = new Path("s3a://bucket/auth/data1"); + private final Path deepAuth = new Path("s3a://bucket/auth/d1/d2/data2"); + + private void assertRetention(Path path, boolean retain) { + Assertions.assertThat(retention.keepDirectoryMarkers(path)) + .describedAs("Retention of path %s by %s", path, retention) + .isEqualTo(retain); + } + + @Test + public void testNonAuthPath() throws Throwable { + assertRetention(nonAuthPath, expectNonAuthDelete); + } + + @Test + public void testAuthPath() throws Throwable { + assertRetention(authPath, expectAuthDelete); + } + + @Test + public void testDeepAuthPath() throws Throwable { + assertRetention(deepAuth, expectAuthDelete); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java index b2e6b3e93a8b3..70557588136a8 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java @@ -38,9 +38,12 @@ import org.apache.hadoop.fs.s3a.impl.StoreContext; import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE; import static org.apache.hadoop.fs.s3a.Constants.ENABLE_MULTI_DELETE; import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY; import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles; @@ -82,13 +85,17 @@ protected int getTestTimeoutMillis() { protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); String bucketName = getTestBucketName(conf); + disableFilesystemCaching(conf); // set a sleep time of 0 on pruning, for speedier test runs. - removeBucketOverrides(bucketName, conf, ENABLE_MULTI_DELETE); + removeBucketOverrides(bucketName, conf, ENABLE_MULTI_DELETE, + DIRECTORY_MARKER_POLICY); conf.setTimeDuration( S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY, 0, TimeUnit.MILLISECONDS); + conf.set(DIRECTORY_MARKER_POLICY, + DIRECTORY_MARKER_POLICY_DELETE); return conf; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java index 1ce3ee56ce0a5..5f7a6fbd072df 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestDirListingMetadata.java @@ -316,18 +316,23 @@ public void testRemoveExpiredEntriesFromListing() { List listing = Arrays.asList(pathMeta1, pathMeta2, pathMeta3); DirListingMetadata meta = new DirListingMetadata(path, listing, false); - meta.removeExpiredEntriesFromListing(ttl, now); + List expired = meta.removeExpiredEntriesFromListing(ttl, + now); Assertions.assertThat(meta.getListing()) .describedAs("Metadata listing for %s", path) .doesNotContain(pathMeta1) .contains(pathMeta2) .contains(pathMeta3); + Assertions.assertThat(expired) + .describedAs("Expire entries underr %s", path) + .doesNotContain(pathMeta2) + .contains(pathMeta1); } - /* + /** * Create DirListingMetadata with two dirs and one file living in directory - * 'parent' + * 'parent'. */ private static DirListingMetadata makeTwoDirsOneFile(Path parent) { PathMetadata pathMeta1 = new PathMetadata( diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/HeadListCosts.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/HeadListCosts.java new file mode 100644 index 0000000000000..378d143b7c1fa --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/HeadListCosts.java @@ -0,0 +1,82 @@ +/* + * 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; + +/** + * Declaration of the costs of head and list calls for various FS IO operations. + */ +public class HeadListCosts { + + /** Head costs for getFileStatus() directory probe: {@value}. */ + public static final int GFS_DIR_PROBE_H = 0; + + /** List costs for getFileStatus() directory probe: {@value}. */ + public static final int GFS_DIR_PROBE_L = 1; + + + /** List costs for getFileStatus() on a non-empty directory: {@value}. */ + public static final int GFS_DIR_L = GFS_DIR_PROBE_L; + + /** List costs for getFileStatus() on an non-empty directory: {@value}. */ + public static final int GFS_EMPTY_DIR_L = GFS_DIR_PROBE_L; + + /** Head cost getFileStatus() file probe only. */ + public static final int GFS_FILE_PROBE_H = 1; + + /** Head costs getFileStatus() no file or dir. */ + public static final int GFS_FNFE_H = GFS_FILE_PROBE_H; + + /** List costs for getFileStatus() on an empty path: {@value}. */ + + public static final int GFS_FNFE_L = GFS_DIR_PROBE_L; + + /** + * Cost of renaming a file to a diffrent directory. + * LIST on dest not found, look for dest dir, and then, at + * end of rename, whether a parent dir needs to be created. + */ + public static final int RENAME_SINGLE_FILE_RENAME_DIFFERENT_DIR_L = + GFS_FNFE_L + GFS_DIR_L * 2; + + /** source is found, dest not found, copy metadataRequests */ + public static final int RENAME_SINGLE_FILE_RENAME_H = + GFS_FILE_PROBE_H + GFS_FNFE_H + 1; + + /** getFileStatus() directory which is non-empty. */ + public static final int GFS_DIR_H = GFS_FILE_PROBE_H; + + /** getFileStatus() directory marker which exists. */ + public static final int GFS_MARKER_H = GFS_FILE_PROBE_H; + + /** getFileStatus() on a file which exists. */ + public static final int GFS_SINGLE_FILE_H = GFS_FILE_PROBE_H; + + public static final int GFS_FILE_PROBE_L = 0; + + public static final int GFS_SINGLE_FILE_L = 0; + + public static final int DELETE_OBJECT_REQUEST = 1; + + public static final int DELETE_MARKER_REQUEST = 1; + + /** listLocatedStatus always does a list. */ + public static final int LIST_LOCATED_STATUS_L = 1; + + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/OperationCostValidator.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/OperationCostValidator.java new file mode 100644 index 0000000000000..79b3eb629a2a3 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/OperationCostValidator.java @@ -0,0 +1,327 @@ +/* + * 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.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.Callable; +import java.util.stream.Collectors; + +import org.assertj.core.api.Assumptions; + +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.S3ATestUtils; +import org.apache.hadoop.fs.s3a.Statistic; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Support for declarative assertions about operation cost. + */ +public class OperationCostValidator { + + /** + * The empty probe. This is a no-op. + */ + public static final ExpectedProbe EMPTY_PROBE = + new ProbeList(new ArrayList<>()); + + private final Map metricDiffs + = new TreeMap<>(); + + public OperationCostValidator(Builder builder) { + builder.metrics.forEach(stat -> + metricDiffs.put(stat.getSymbol(), + new S3ATestUtils.MetricDiff(builder.fs, stat)) + ); + builder.metrics.clear(); + } + + + /** + * Reset all the metrics being tracked. + */ + public void resetMetricDiffs() { + metricDiffs.values().forEach(S3ATestUtils.MetricDiff::reset); + } + + /** + * Get the diff of a statistic. + * @param stat statistic to look up + * @return the value + * @throws NullPointerException if there is no match + */ + public S3ATestUtils.MetricDiff get(Statistic stat) { + S3ATestUtils.MetricDiff diff = + requireNonNull(metricDiffs.get(stat.getSymbol()), + () -> "No metric tracking for " + stat); + return diff; + } + + + /** + * Execute a closure and verify the metrics. + *

+ * If no probes are active, the operation will + * raise an Assumption exception for the test to be skipped. + * @param eval closure to evaluate + * @param expected varargs list of expected diffs + * @param return type. + * @return the result of the evaluation + */ + public T verify( + Callable eval, + ExpectedProbe... expected) throws Exception { + resetMetricDiffs(); + // verify that 1+ probe is enabled + assumeProbesEnabled(expected); + // if we get here, then yes. + // evaluate it + T r = eval.call(); + // build the text for errors + String text = r != null ? r.toString() : "operation returning null"; + for (ExpectedProbe ed : expected) { + ed.verify(this, text); + } + return r; + } + + /** + * Scan all probes for being enabled. + *

+ * If none of them are enabled, the evaluation will be skipped. + * @param expected list of expected probes + */ + private void assumeProbesEnabled(ExpectedProbe[] expected) { + boolean enabled = false; + for (ExpectedProbe ed : expected) { + enabled |= ed.isEnabled(); + } + Assumptions.assumeThat(enabled) + .describedAs("metrics to probe for") + .isTrue(); + } + + /** + * Execute a closure, expecting an exception. + * Verify the metrics after the exception has been caught and + * validated. + * @param clazz type of exception + * @param text text to look for in exception (optional) + * @param eval closure to evaluate + * @param expected varargs list of expected diffs + * @param return type of closure + * @param exception type + * @return the exception caught. + * @throws Exception any other exception + */ + public E verifyIntercepting( + Class clazz, + String text, + Callable eval, + ExpectedProbe... expected) throws Exception { + + return verify(() -> + intercept(clazz, text, eval), + expected); + } + + @Override + public String toString() { + return metricDiffs.values().stream() + .map(S3ATestUtils.MetricDiff::toString) + .collect(Collectors.joining(", ")); + } + + /** + * Create a builder for the cost checker. + * + * @param fs filesystem. + * @return builder. + */ + public static Builder builder(S3AFileSystem fs) { + return new Builder(fs); + } + + /** + * builder. + */ + public static final class Builder { + + private final List metrics = new ArrayList<>(); + + private final S3AFileSystem fs; + + + public Builder(final S3AFileSystem fs) { + this.fs = requireNonNull(fs); + } + + + public Builder withMetric(Statistic statistic) { + return withMetric(statistic); + } + + + public Builder withMetrics(Statistic...stats) { + metrics.addAll(Arrays.asList(stats)); + return this; + } + + public OperationCostValidator build() { + return new OperationCostValidator(this); + } + } + + /** + * Create a probe of a statistic which is always enabled. + * @param statistic statistic to check. + * @param expected expected value. + * @return a probe. + */ + public static ExpectedProbe probe( + final Statistic statistic, + final int expected) { + return probe(true, statistic, expected); + } + + /** + * Create a probe of a statistic which is conditionally enabled. + * @param enabled is the probe enabled? + * @param statistic statistic to check. + * @param expected expected value. + * @return a probe. + */ + public static ExpectedProbe probe( + final boolean enabled, + final Statistic statistic, + final int expected) { + return enabled + ? new ExpectSingleStatistic(statistic, expected) + : EMPTY_PROBE; + } + + /** + * Create an aggregate probe from a vararges list of probes. + * @param enabled should the probes be enabled? + * @param plist probe list + * @return a probe + */ + public static ExpectedProbe probes( + final boolean enabled, + final ExpectedProbe...plist) { + return new ProbeList(Arrays.asList(plist)); + } + + /** + * An expected probe to verify given criteria to trigger an eval. + *

+ * Probes can be conditional, in which case they are only evaluated + * when true. + */ + public interface ExpectedProbe { + + /** + * Verify a diff if the FS instance is compatible. + * @param message message to print; metric name is appended + */ + void verify(OperationCostValidator diffs, String message); + + boolean isEnabled(); + } + + /** + * Simple probe is a single statistic. + */ + public static final class ExpectSingleStatistic implements ExpectedProbe { + + private final Statistic statistic; + + private final int expected; + + /** + * Create. + * @param statistic statistic + * @param expected expected value. + */ + private ExpectSingleStatistic(final Statistic statistic, + final int expected) { + this.statistic = statistic; + this.expected = expected; + } + + /** + * Verify a diff if the FS instance is compatible. + * @param message message to print; metric name is appended + */ + @Override + public void verify(OperationCostValidator diffs, String message) { + diffs.get(statistic).assertDiffEquals(message, expected); + } + + public Statistic getStatistic() { + return statistic; + } + + public int getExpected() { + return expected; + } + + @Override + public boolean isEnabled() { + return true; + } + } + + /** + * A list of probes; the verify operation + * verifies them all + */ + public static class ProbeList implements ExpectedProbe { + + private final List probes; + + public ProbeList(final List probes) { + this.probes = probes; + } + + @Override + public void verify(final OperationCostValidator diffs, + final String message) { + probes.forEach(p -> p.verify(diffs, message)); + } + + /** + * Enabled if 1+ probe is enabled. + * @return true if enabled. + */ + @Override + public boolean isEnabled() { + boolean enabled = false; + for (ExpectedProbe probe : probes) { + enabled |= probe.isEnabled(); + } + return enabled; + } + } +} From 93be7b6e9e5a8de19911b152aa5dff87c9d20955 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 15 Jul 2020 22:22:36 +0100 Subject: [PATCH 02/20] HADOOP-13230 dir marker -cost assertion classes and tests evolving Change-Id: I1093fcce9737ac18a5e9a6cfb4f31da6731faf70 --- .../fs/s3a/ITestS3AFileOperationCost.java | 334 +++++++++++------- .../s3a/test/{ => costs}/HeadListCosts.java | 86 +++-- .../{ => costs}/OperationCostValidator.java | 58 ++- 3 files changed, 312 insertions(+), 166 deletions(-) rename hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/{ => costs}/HeadListCosts.java (51%) rename hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/{ => costs}/OperationCostValidator.java (84%) 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 15bf9ecc2fe58..f1ec75affedc0 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 @@ -26,8 +26,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; -import org.apache.hadoop.fs.s3a.test.HeadListCosts; -import org.apache.hadoop.fs.s3a.test.OperationCostValidator; +import org.apache.hadoop.fs.s3a.test.costs.OperationCostValidator; import org.assertj.core.api.Assertions; import org.junit.Test; @@ -55,8 +54,9 @@ import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; import static org.apache.hadoop.fs.s3a.Statistic.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; -import static org.apache.hadoop.fs.s3a.test.OperationCostValidator.probe; -import static org.apache.hadoop.fs.s3a.test.OperationCostValidator.probes; +import static org.apache.hadoop.fs.s3a.test.costs.HeadListCosts.*; +import static org.apache.hadoop.fs.s3a.test.costs.OperationCostValidator.probe; +import static org.apache.hadoop.fs.s3a.test.costs.OperationCostValidator.probes; import static org.apache.hadoop.test.AssertExtensions.dynamicDescription; import static org.apache.hadoop.test.GenericTestUtils.getTestDir; @@ -156,14 +156,14 @@ public void setup() throws Exception { // build up the states isGuarded = isGuarded(); - boolean isUnguarded = !isGuarded; - boolean isAuthoritative = isGuarded && authoritative; - boolean isNonAuth = isGuarded && !authoritative; + isRaw = !isGuarded; + isAuthoritative = isGuarded && authoritative; + isNonAuth = isGuarded && !authoritative; - boolean isKeeping = isKeepingMarkers (); + isKeeping = isKeepingMarkers (); - boolean isDeleting = !isKeeping; + isDeleting = !isKeeping; costValidator = OperationCostValidator.builder(getFileSystem()) .withMetrics(INVOCATION_COPY_FROM_LOCAL_FILE, @@ -184,14 +184,46 @@ public void assumeUnguarded() { assume("Unguarded FS only", !isGuarded()); } + /** + * Is the store guarded authoritatively on the test path? + * @return true if the condition is met on this test run. + */ public boolean isAuthoritative() { return authoritative; } + /** + * Is the store guarded? + * @return true if the condition is met on this test run. + */ public boolean isGuarded() { return s3guard; } + /** + * Is the store raw? + * @return true if the condition is met on this test run. + */ + public boolean isRaw() { + return isRaw; + } + + /** + * Is the store guarded non-authoritatively on the test path? + * @return true if the condition is met on this test run. + */ + public boolean isNonAuth() { + return isNonAuth; + } + + public boolean isKeeping() { + return isKeeping; + } + + public boolean isDeleting() { + return isDeleting; + } + public boolean isKeepingMarkers() { return keepMarkers; } @@ -202,9 +234,9 @@ public void testCostOfLocatedFileStatusOnFile() throws Throwable { Path file = file(methodPath()); S3AFileSystem fs = getFileSystem(); verifyMetrics(() -> fs.listLocatedStatus(file), - raw(OBJECT_LIST_REQUESTS, HeadListCosts.LIST_LOCATED_STATUS_L), - nonauth(OBJECT_LIST_REQUESTS, HeadListCosts.LIST_LOCATED_STATUS_L), - raw(OBJECT_METADATA_REQUESTS, HeadListCosts.GFS_FILE_PROBE_H)); + raw(OBJECT_LIST_REQUESTS, LIST_LOCATED_STATUS_L), + nonauth(OBJECT_LIST_REQUESTS, LIST_LOCATED_STATUS_L), + raw(OBJECT_METADATA_REQUESTS, FILESTATUS_FILE_PROBE_H)); } @Test @@ -214,14 +246,12 @@ public void testCostOfListLocatedStatusOnEmptyDir() throws Throwable { S3AFileSystem fs = getFileSystem(); verifyMetrics(() -> fs.listLocatedStatus(dir), - raw(OBJECT_METADATA_REQUESTS, - HeadListCosts.GFS_FILE_PROBE_H), - raw(OBJECT_LIST_REQUESTS, - HeadListCosts.LIST_LOCATED_STATUS_L + HeadListCosts.GFS_DIR_L), + expectRawHeadList(FILESTATUS_FILE_PROBE_H, + LIST_LOCATED_STATUS_L + GETFILESTATUS_DIR_L), guarded(OBJECT_METADATA_REQUESTS, 0), authoritative(OBJECT_LIST_REQUESTS, 0), nonauth(OBJECT_LIST_REQUESTS, - HeadListCosts.LIST_LOCATED_STATUS_L)); + LIST_LOCATED_STATUS_L)); } @Test @@ -234,10 +264,10 @@ public void testCostOfListLocatedStatusOnNonEmptyDir() throws Throwable { fs.listLocatedStatus(dir), always(OBJECT_METADATA_REQUESTS, 0), raw(OBJECT_LIST_REQUESTS, - HeadListCosts.LIST_LOCATED_STATUS_L), + LIST_LOCATED_STATUS_L), authoritative(OBJECT_LIST_REQUESTS, 0), nonauth(OBJECT_LIST_REQUESTS, - HeadListCosts.LIST_LOCATED_STATUS_L)); + LIST_LOCATED_STATUS_L)); } @Test @@ -248,36 +278,26 @@ public void testCostOfListFilesOnFile() throws Throwable { touch(fs, file); verifyMetrics(() -> fs.listFiles(file, true), - raw(OBJECT_METADATA_REQUESTS, 1), - raw(OBJECT_LIST_REQUESTS, - HeadListCosts.LIST_LOCATED_STATUS_L), + expectRawHeadList(GETFILESTATUS_SINGLE_FILE_H, + LIST_LOCATED_STATUS_L), authoritative(OBJECT_LIST_REQUESTS, 0), nonauth(OBJECT_LIST_REQUESTS, - 1)); -/* resetMetricDiffs(); - fs.listFiles(file, true); - if (!fs.hasMetadataStore()) { - metadataRequests.assertDiffEquals(1); - } else { - if (fs.allowAuthoritative(file)) { - listRequests.assertDiffEquals(0); - } else { - listRequests.assertDiffEquals(1); - } - }*/ + LIST_FILES_L)); } @Test public void testCostOfListFilesOnEmptyDir() throws Throwable { - describe("Performing listFiles() on an empty dir"); + describe("Perpforming listFiles() on an empty dir with marker"); + // this attem Path dir = path(getMethodName()); S3AFileSystem fs = getFileSystem(); fs.mkdirs(dir); verifyMetrics(() -> fs.listFiles(dir, true), - expectHeadList(isRaw, 2, 1), - expectHeadList(isAuthoritative, 0, 0), - expectHeadList(isNonAuth, 0, 1)); + expectRawHeadList(GETFILESTATUS_EMPTY_DIR_H, + LIST_FILES_L + GETFILESTATUS_EMPTY_DIR_L), + expectAuthHeadList(0, 0), + expectNonauthHeadList(0, LIST_FILES_L)); } @Test @@ -288,17 +308,11 @@ public void testCostOfListFilesOnNonEmptyDir() throws Throwable { fs.mkdirs(dir); Path file = new Path(dir, "file.txt"); touch(fs, file); - resetMetricDiffs(); - fs.listFiles(dir, true); - if (!fs.hasMetadataStore()) { - verifyOperationCount(0, 1); - } else { - if (fs.allowAuthoritative(dir)) { - verifyOperationCount(0, 0); - } else { - verifyOperationCount(0, 1); - } - } + verifyMetrics(() -> + fs.listFiles(dir, true), + expectRawHeadList(0, LIST_FILES_L), + expectAuthHeadList(0, 0), + expectNonauthHeadList(0, LIST_FILES_L)); } @Test @@ -306,10 +320,12 @@ public void testCostOfListFilesOnNonExistingDir() throws Throwable { describe("Performing listFiles() on a non existing dir"); Path dir = path(getMethodName()); S3AFileSystem fs = getFileSystem(); - resetMetricDiffs(); - intercept(FileNotFoundException.class, - () -> fs.listFiles(dir, true)); - verifyOperationCount(2, 2); + verifyMetricsIntercepting(FileNotFoundException.class, "", + () -> fs.listFiles(dir, true), + expectRawHeadList(GETFILESTATUS_FNFE_H, + GETFILESTATUS_FNFE_L + LIST_FILES_L) + + ); } @Test @@ -318,8 +334,8 @@ public void testCostOfGetFileStatusOnFile() throws Throwable { Path simpleFile = file(methodPath()); S3AFileStatus status = verifyRawGetFileStatus(simpleFile, true, StatusProbeEnum.ALL, - HeadListCosts.GFS_SINGLE_FILE_H, - HeadListCosts.GFS_SINGLE_FILE_L); + GETFILESTATUS_SINGLE_FILE_H, + GETFILESTATUS_SINGLE_FILE_L); assertTrue("not a file: " + status, status.isFile()); } @@ -330,21 +346,21 @@ public void testCostOfGetFileStatusOnEmptyDir() throws Throwable { Path dir = dir(methodPath()); S3AFileStatus status = verifyRawGetFileStatus(dir, true, StatusProbeEnum.ALL, - HeadListCosts.GFS_MARKER_H, - HeadListCosts.GFS_EMPTY_DIR_L); + GETFILESTATUS_MARKER_H, + GETFILESTATUS_EMPTY_DIR_L); assertSame("not empty: " + status, Tristate.TRUE, status.isEmptyDirectory()); // but now only ask for the directories and the file check is skipped. verifyRawGetFileStatus(dir, false, StatusProbeEnum.DIRECTORIES, - HeadListCosts.GFS_DIR_PROBE_H, - HeadListCosts.GFS_EMPTY_DIR_L); + FILESTATUS_DIR_PROBE_H, + GETFILESTATUS_EMPTY_DIR_L); // now look at isFile/isDir against the same entry isDir(dir, true, 0, - HeadListCosts.GFS_EMPTY_DIR_L); + GETFILESTATUS_EMPTY_DIR_L); isFile(dir, false, - HeadListCosts.GFS_SINGLE_FILE_H, HeadListCosts.GFS_SINGLE_FILE_L); + GETFILESTATUS_SINGLE_FILE_H, GETFILESTATUS_SINGLE_FILE_L); } @Test @@ -352,8 +368,8 @@ public void testCostOfGetFileStatusOnMissingFile() throws Throwable { describe("performing getFileStatus on a missing file"); verifyRawGetFileStatusFNFE(methodPath(), false, StatusProbeEnum.ALL, - HeadListCosts.GFS_FNFE_H, - HeadListCosts.GFS_FNFE_L); + GETFILESTATUS_FNFE_H, + GETFILESTATUS_FNFE_L); } @Test @@ -362,21 +378,21 @@ public void testIsDirIsFileMissingPath() throws Throwable { Path path = methodPath(); // now look at isFile/isDir against the same entry isDir(path, false, - HeadListCosts.GFS_DIR_PROBE_H, - HeadListCosts.GFS_DIR_PROBE_L); + FILESTATUS_DIR_PROBE_H, + FILESTATUS_DIR_PROBE_L); isFile(path, false, - HeadListCosts.GFS_FILE_PROBE_H, - HeadListCosts.GFS_FILE_PROBE_L); + FILESTATUS_FILE_PROBE_H, + FILESTATUS_FILE_PROBE_L); } @Test public void testCostOfGetFileStatusOnNonEmptyDir() throws Throwable { describe("performing getFileStatus on a non-empty directory"); Path dir = dir(methodPath()); - Path simpleFile = file(new Path(dir, "simple.txt")); + file(new Path(dir, "simple.txt")); S3AFileStatus status = verifyRawGetFileStatus(dir, true, StatusProbeEnum.ALL, - HeadListCosts.GFS_DIR_H, HeadListCosts.GFS_DIR_L); + GETFILESTATUS_DIR_H, GETFILESTATUS_DIR_L); assertEmptyDirStatus(status, Tristate.FALSE); } @@ -400,28 +416,28 @@ public void testDeleteFile() throws Throwable { }, // delete file. For keeping: that's it probe(isRaw && isKeeping, OBJECT_METADATA_REQUESTS, - HeadListCosts.GFS_FILE_PROBE_H), + FILESTATUS_FILE_PROBE_H), // if deleting markers, look for the parent too probe(isRaw && isDeleting, OBJECT_METADATA_REQUESTS, - HeadListCosts.GFS_FILE_PROBE_H + HeadListCosts.GFS_DIR_PROBE_H), + FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H), raw(OBJECT_LIST_REQUESTS, - HeadListCosts.GFS_FILE_PROBE_L + HeadListCosts.GFS_DIR_PROBE_L), + FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L), always(DIRECTORIES_DELETED, 0), always(FILES_DELETED, 1), // keeping: create no parent dirs or delete parents keeping(DIRECTORIES_CREATED, 0), - keeping(OBJECT_DELETE_REQUESTS, HeadListCosts.DELETE_OBJECT_REQUEST), + keeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), // deleting: create a parent and delete any of its parents deleting(DIRECTORIES_CREATED, 1), deleting(OBJECT_DELETE_REQUESTS, - HeadListCosts.DELETE_OBJECT_REQUEST - + HeadListCosts.DELETE_MARKER_REQUEST) + DELETE_OBJECT_REQUEST + + DELETE_MARKER_REQUEST) ); // there is an empty dir for a parent S3AFileStatus status = verifyRawGetFileStatus(dir, true, - StatusProbeEnum.ALL, HeadListCosts.GFS_DIR_H, HeadListCosts.GFS_DIR_L); + StatusProbeEnum.ALL, GETFILESTATUS_DIR_H, GETFILESTATUS_DIR_L); assertEmptyDirStatus(status, Tristate.TRUE); } @@ -475,7 +491,7 @@ public void testDirMarkersSubdir() throws Throwable { always(DIRECTORIES_DELETED, 0), keeping(OBJECT_DELETE_REQUESTS, 0), keeping(FAKE_DIRECTORIES_DELETED, 0), - deleting(OBJECT_DELETE_REQUESTS, HeadListCosts.DELETE_MARKER_REQUEST), + deleting(OBJECT_DELETE_REQUESTS, DELETE_MARKER_REQUEST), // delete all possible fake dirs above the subdirectory deleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(subDir) - 1)); } @@ -539,17 +555,17 @@ public void testRenameFileToDifferentDirectory() throws Throwable { verifyMetrics(() -> execRename(srcFilePath, destFilePath), raw(OBJECT_METADATA_REQUESTS, - HeadListCosts.RENAME_SINGLE_FILE_RENAME_H), + RENAME_SINGLE_FILE_RENAME_H), raw(OBJECT_LIST_REQUESTS, - HeadListCosts.RENAME_SINGLE_FILE_RENAME_DIFFERENT_DIR_L), + RENAME_SINGLE_FILE_RENAME_DIFFERENT_DIR_L), always(DIRECTORIES_CREATED, 0), always(DIRECTORIES_DELETED, 0), // keeping: only the core delete operation is issued. - keeping(OBJECT_DELETE_REQUESTS, HeadListCosts.DELETE_OBJECT_REQUEST), + keeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), keeping(FAKE_DIRECTORIES_DELETED, 0), // deleting: delete any fake marker above the destination. deleting(OBJECT_DELETE_REQUESTS, - HeadListCosts.DELETE_OBJECT_REQUEST + HeadListCosts.DELETE_MARKER_REQUEST), + DELETE_OBJECT_REQUEST + DELETE_MARKER_REQUEST), deleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(destDir))); assertIsFile(destFilePath); @@ -578,11 +594,11 @@ public void testRenameSameDirectory() throws Throwable { Path destFile = new Path(parent2, "dest"); verifyMetrics(() -> execRename(sourceFile, destFile), - raw(OBJECT_METADATA_REQUESTS, HeadListCosts.RENAME_SINGLE_FILE_RENAME_H), - raw(OBJECT_LIST_REQUESTS, HeadListCosts.GFS_FNFE_L), + raw(OBJECT_METADATA_REQUESTS, RENAME_SINGLE_FILE_RENAME_H), + raw(OBJECT_LIST_REQUESTS, GETFILESTATUS_FNFE_L), always(OBJECT_COPY_REQUESTS, 1), always(DIRECTORIES_CREATED, 0), - always(OBJECT_DELETE_REQUESTS, HeadListCosts.DELETE_OBJECT_REQUEST), + always(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), always(FAKE_DIRECTORIES_DELETED, 0)); } @@ -603,15 +619,13 @@ public void testCostOfRootRename() throws Throwable { return "after fs.rename(/src,/dest) " + metricSummary; }, // TWO HEAD for exists, one for source MD in copy - raw(OBJECT_METADATA_REQUESTS, - HeadListCosts.RENAME_SINGLE_FILE_RENAME_H), - raw(OBJECT_LIST_REQUESTS, - HeadListCosts.GFS_FNFE_L), + expectRawHeadList(RENAME_SINGLE_FILE_RENAME_H, + GETFILESTATUS_FNFE_L)); // here we expect there to be no fake directories always(DIRECTORIES_CREATED, 0), // one for the renamed file only always(OBJECT_DELETE_REQUESTS, - HeadListCosts.DELETE_OBJECT_REQUEST), + DELETE_OBJECT_REQUEST), // no directories are deleted: This is root always(DIRECTORIES_DELETED, 0), // no fake directories are deleted: This is root @@ -627,10 +641,8 @@ public void testCostOfRootRename() throws Throwable { always(DIRECTORIES_DELETED, 0), always(FAKE_DIRECTORIES_DELETED, 0), always(FILES_DELETED, 1), - always(OBJECT_DELETE_REQUESTS, HeadListCosts.DELETE_OBJECT_REQUEST), - raw(OBJECT_METADATA_REQUESTS, HeadListCosts.GFS_FILE_PROBE_H), - raw(OBJECT_LIST_REQUESTS, 0) /* no need to look at parent. */ - ); + always(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + expectRawHeadList(FILESTATUS_FILE_PROBE_H, 0)); /* no need to look at parent. */ } finally { fs.delete(src, false); @@ -649,13 +661,13 @@ public void testDirProbes() throws Throwable { // head probe fails verifyRawGetFileStatusFNFE(emptydir, false, StatusProbeEnum.HEAD_ONLY, - HeadListCosts.GFS_FILE_PROBE_H, - HeadListCosts.GFS_FILE_PROBE_L); + FILESTATUS_FILE_PROBE_H, + FILESTATUS_FILE_PROBE_L); // a LIST will find it and declare as empty S3AFileStatus status = verifyRawGetFileStatus(emptydir, true, StatusProbeEnum.LIST_ONLY, 0, - HeadListCosts.GFS_EMPTY_DIR_L); + GETFILESTATUS_EMPTY_DIR_L); assertEmptyDirStatus(status, Tristate.TRUE); // skip all probes and expect no operations to take place @@ -674,7 +686,7 @@ public void testDirProbes() throws Throwable { StatusProbeEnum.HEAD_ONLY, null, false)); // but ask for a directory marker and you get the entry - status = verifyRawHeadList(0, HeadListCosts.GFS_EMPTY_DIR_L, () -> + status = verifyRawHeadList(0, GETFILESTATUS_EMPTY_DIR_L, () -> fs.s3GetFileStatus(emptydir, emptyDirTrailingSlash, StatusProbeEnum.LIST_ONLY, @@ -704,9 +716,10 @@ public void testCreateCost() throws Throwable { assumeUnguarded(); Path testFile = methodPath(); // when overwrite is false, the path is checked for existence. - create(testFile, false, HeadListCosts.GFS_FNFE_H, HeadListCosts.GFS_FNFE_L); + create(testFile, false, + CREATE_FILE_NO_OVERWRITE_H, CREATE_FILE_NO_OVERWRITE_L); // but when true: only the directory checks take place. - create(testFile, true, 0, HeadListCosts.GFS_FNFE_L); + create(testFile, true, CREATE_FILE_OVERWRITE_H, CREATE_FILE_OVERWRITE_L); } @Test @@ -718,7 +731,7 @@ public void testCreateCostFileExists() throws Throwable { // now there is a file there, an attempt with overwrite == false will // fail on the first HEAD. verifyRawHeadListIntercepting(FileAlreadyExistsException.class, "", - HeadListCosts.GFS_FILE_PROBE_H, 0, + FILESTATUS_FILE_PROBE_H, 0, () -> file(testFile, false)); } @@ -731,7 +744,7 @@ public void testCreateCostDirExists() throws Throwable { // now there is a file there, an attempt with overwrite == false will // fail on the first HEAD. verifyRawHeadListIntercepting(FileAlreadyExistsException.class, "", - HeadListCosts.GFS_MARKER_H, HeadListCosts.GFS_EMPTY_DIR_L, + GETFILESTATUS_MARKER_H, GETFILESTATUS_EMPTY_DIR_L, () -> file(testFile, false)); } @@ -748,18 +761,18 @@ public void testCreateBuilder() throws Throwable { // builder defaults to looking for parent existence (non-recursive) buildFile(testFile, false, false, - HeadListCosts.GFS_FILE_PROBE_H, // destination file - HeadListCosts.GFS_DIR_PROBE_L * 2); // destination file and parent dir + FILESTATUS_FILE_PROBE_H, // destination file + FILESTATUS_DIR_PROBE_L * 2); // destination file and parent dir // recursive = false and overwrite=true: // only make sure the dest path isn't a directory. buildFile(testFile, true, true, - HeadListCosts.GFS_DIR_PROBE_H, HeadListCosts.GFS_DIR_PROBE_L); + FILESTATUS_DIR_PROBE_H, FILESTATUS_DIR_PROBE_L); // now there is a file there, an attempt with overwrite == false will // fail on the first HEAD. verifyRawHeadListIntercepting(FileAlreadyExistsException.class, "", - HeadListCosts.GFS_FILE_PROBE_H, 0, () -> - buildFile(testFile, false, true, HeadListCosts.GFS_FILE_PROBE_H, 0)); + FILESTATUS_FILE_PROBE_H, 0, () -> + buildFile(testFile, false, true, FILESTATUS_FILE_PROBE_H, 0)); } @Test @@ -773,15 +786,15 @@ public void testCostOfGlobStatus() throws Throwable { // create a bunch of files int filesToCreate = 10; for (int i = 0; i < filesToCreate; i++) { - try (FSDataOutputStream out = fs.create(basePath.suffix("/" + i))) { - verifyOperationCount(1, 1); - } + create(basePath.suffix("/" + i)); } fs.globStatus(basePath.suffix("/*")); // 2 head + 1 list from getFileStatus on path, // plus 1 list to match the glob pattern - verifyOperationCount(2, 2); + verifyRawHeadList(GETFILESTATUS_DIR_H, + GETFILESTATUS_DIR_L + 1, () -> + fs.globStatus(basePath.suffix("/*"))); } @Test @@ -795,15 +808,26 @@ public void testCostOfGlobStatusNoSymlinkResolution() throws Throwable { // create a single file, globStatus returning a single file on a pattern // triggers attempts at symlinks resolution if configured String fileName = "/notASymlinkDOntResolveMeLikeOne"; - try (FSDataOutputStream out = fs.create(basePath.suffix(fileName))) { - verifyOperationCount(1, 1); - } - - fs.globStatus(basePath.suffix("/*")); + create(basePath.suffix(fileName)); // unguarded: 2 head + 1 list from getFileStatus on path, // plus 1 list to match the glob pattern // no additional operations from symlink resolution - verifyOperationCount(2, 2); + verifyRawHeadList( + GETFILESTATUS_DIR_H, + GETFILESTATUS_DIR_L + 1, () -> + fs.globStatus(basePath.suffix("/*"))); + } + + + /** + * Touch a file, overwriting. + * @param path path + * @return path to new object. + */ + private Path create(Path path) throws Exception { + return create(path, true, + CREATE_FILE_OVERWRITE_H, + CREATE_FILE_OVERWRITE_L); } /** @@ -833,7 +857,7 @@ private Path buildFile(Path path, boolean overwrite, boolean recursive, int head, - int list) throws IOException { + int list) throws Exception { resetStatistics(); verifyRawHeadList(head, list, () -> { FSDataOutputStreamBuilder builder = getFileSystem().createFile(path) @@ -841,7 +865,9 @@ private Path buildFile(Path path, if (recursive) { builder.recursive(); } - builder.build().close(); + FSDataOutputStream stream = builder.build(); + stream.close(); + return stream.toString(); }); return path; } @@ -961,25 +987,68 @@ private E verifyRawHeadListIntercepting( int list, Callable eval) throws Exception { return verifyMetricsIntercepting(clazz, text, eval, - raw(OBJECT_METADATA_REQUESTS, head), - raw(OBJECT_LIST_REQUESTS, list)); + expectRawHeadList(head, list)); } - /** - * Verify that the head and list calls match expectations, - * then reset the counters ready for the next operation. + * Create the probes to expect a given set of head and list requests. + * @param enabled is the probe enabled? * @param head expected HEAD count * @param list expected LIST count * @return a probe list */ private OperationCostValidator.ExpectedProbe - expectHeadList(boolean enabled, - int head, int list) { + expectHeadList(boolean enabled, int head, int list) { return probes(enabled, probe(OBJECT_METADATA_REQUESTS, head), probe(OBJECT_LIST_REQUESTS, list)); } + /** + * Create the probes to expect a given set of head and list requests. + * @param head expected HEAD count + * @param list expected LIST count + * @return a probe list + */ + private OperationCostValidator.ExpectedProbe + expectHeadList(int head, int list) { + return probes(true, + probe(OBJECT_METADATA_REQUESTS, head), + probe(OBJECT_LIST_REQUESTS, list)); + } + + /** + * Declare the expected head and list requests on a raw FS. + * @param head expected HEAD count + * @param list expected LIST count + * @return a probe list + */ + private OperationCostValidator.ExpectedProbe + expectRawHeadList(int head, int list) { + return expectHeadList(isRaw(), head, list); + } + + /** + * Declare the expected head and list requests on an authoritative FS. + * @param head expected HEAD count + * @param list expected LIST count + * @return a probe list + */ + private OperationCostValidator.ExpectedProbe + expectAuthHeadList(int head, int list) { + return expectHeadList(isAuthoritative(), head, list); + } + + /** + * Declare the expected head and list requests on a + * non authoritative FS. + * @param head expected HEAD count + * @param list expected LIST count + * @return a probe list + */ + private OperationCostValidator.ExpectedProbe + expectNonauthHeadList(int head, int list) { + return expectHeadList(isNonAuth(), head, list); + } /** * Execute a closure expecting a specific number of HEAD/LIST calls @@ -995,7 +1064,7 @@ private T verifyRawHeadList( int list, Callable eval) throws Exception { return verifyMetrics(eval, - expectHeadList(isRaw, head, list)); + expectRawHeadList(head, list)); } /** @@ -1097,7 +1166,8 @@ private OperationCostValidator.ExpectedProbe guarded( * @param expected expected value. * @return the diff. */ - private OperationCostValidator.ExpectedProbe authoritative(final Statistic Statistic, + private OperationCostValidator.ExpectedProbe authoritative( + final Statistic Statistic, final int expected) { return probe(isAuthoritative, Statistic, expected); } @@ -1108,9 +1178,10 @@ private OperationCostValidator.ExpectedProbe authoritative(final Statistic Stati * @param expected expected value. * @return the diff. */ - private OperationCostValidator.ExpectedProbe nonauth(final Statistic Statistic, + private OperationCostValidator.ExpectedProbe nonauth( + final Statistic Statistic, final int expected) { - return probe(isNonAuth, Statistic, expected); + return probe(isNonAuth(), Statistic, expected); } /** @@ -1119,9 +1190,10 @@ private OperationCostValidator.ExpectedProbe nonauth(final Statistic Statistic, * @param expected expected value. * @return the diff. */ - private OperationCostValidator.ExpectedProbe keeping(final Statistic Statistic, + private OperationCostValidator.ExpectedProbe keeping( + final Statistic Statistic, final int expected) { - return probe(isKeeping, Statistic, expected); + return probe(isKeepingMarkers(), Statistic, expected); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/HeadListCosts.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/HeadListCosts.java similarity index 51% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/HeadListCosts.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/HeadListCosts.java index 378d143b7c1fa..0033e004072c7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/HeadListCosts.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/HeadListCosts.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.test; +package org.apache.hadoop.fs.s3a.test.costs; /** * Declaration of the costs of head and list calls for various FS IO operations. @@ -24,52 +24,45 @@ public class HeadListCosts { /** Head costs for getFileStatus() directory probe: {@value}. */ - public static final int GFS_DIR_PROBE_H = 0; + public static final int FILESTATUS_DIR_PROBE_H = 0; /** List costs for getFileStatus() directory probe: {@value}. */ - public static final int GFS_DIR_PROBE_L = 1; + public static final int FILESTATUS_DIR_PROBE_L = 1; - /** List costs for getFileStatus() on a non-empty directory: {@value}. */ - public static final int GFS_DIR_L = GFS_DIR_PROBE_L; + /** Head cost getFileStatus() file probe only. */ + public static final int FILESTATUS_FILE_PROBE_H = 1; - /** List costs for getFileStatus() on an non-empty directory: {@value}. */ - public static final int GFS_EMPTY_DIR_L = GFS_DIR_PROBE_L; + /** Liast cost getFileStatus() file probe only. */ - /** Head cost getFileStatus() file probe only. */ - public static final int GFS_FILE_PROBE_H = 1; + public static final int FILESTATUS_FILE_PROBE_L = 0; /** Head costs getFileStatus() no file or dir. */ - public static final int GFS_FNFE_H = GFS_FILE_PROBE_H; + public static final int GETFILESTATUS_FNFE_H = FILESTATUS_FILE_PROBE_H; /** List costs for getFileStatus() on an empty path: {@value}. */ - public static final int GFS_FNFE_L = GFS_DIR_PROBE_L; + public static final int GETFILESTATUS_FNFE_L = FILESTATUS_DIR_PROBE_L; - /** - * Cost of renaming a file to a diffrent directory. - * LIST on dest not found, look for dest dir, and then, at - * end of rename, whether a parent dir needs to be created. - */ - public static final int RENAME_SINGLE_FILE_RENAME_DIFFERENT_DIR_L = - GFS_FNFE_L + GFS_DIR_L * 2; + /** getFileStatus() directory which is non-empty. */ + public static final int GETFILESTATUS_DIR_H = FILESTATUS_FILE_PROBE_H; - /** source is found, dest not found, copy metadataRequests */ - public static final int RENAME_SINGLE_FILE_RENAME_H = - GFS_FILE_PROBE_H + GFS_FNFE_H + 1; + /** List costs for getFileStatus() on a non-empty directory: {@value}. */ + public static final int GETFILESTATUS_DIR_L = FILESTATUS_DIR_PROBE_L; - /** getFileStatus() directory which is non-empty. */ - public static final int GFS_DIR_H = GFS_FILE_PROBE_H; + /** List costs for getFileStatus() on an non-empty directory: {@value}. */ + public static final int GETFILESTATUS_EMPTY_DIR_L = FILESTATUS_DIR_PROBE_L; + /** List costs for getFileStatus() on an non-empty directory: {@value}. */ + public static final int GETFILESTATUS_EMPTY_DIR_H = GETFILESTATUS_DIR_H; /** getFileStatus() directory marker which exists. */ - public static final int GFS_MARKER_H = GFS_FILE_PROBE_H; + public static final int GETFILESTATUS_MARKER_H = FILESTATUS_FILE_PROBE_H; /** getFileStatus() on a file which exists. */ - public static final int GFS_SINGLE_FILE_H = GFS_FILE_PROBE_H; + public static final int GETFILESTATUS_SINGLE_FILE_H = FILESTATUS_FILE_PROBE_H; - public static final int GFS_FILE_PROBE_L = 0; - public static final int GFS_SINGLE_FILE_L = 0; + public static final int GETFILESTATUS_SINGLE_FILE_L = 0; public static final int DELETE_OBJECT_REQUEST = 1; @@ -78,5 +71,44 @@ public class HeadListCosts { /** listLocatedStatus always does a list. */ public static final int LIST_LOCATED_STATUS_L = 1; + public static final int LIST_FILES_L = 1; + + /** + * Cost of renaming a file to a different directory. + *

+ * LIST on dest not found, look for dest dir, and then, at + * end of rename, whether a parent dir needs to be created. + */ + public static final int RENAME_SINGLE_FILE_RENAME_DIFFERENT_DIR_L = + GETFILESTATUS_FNFE_L + GETFILESTATUS_DIR_L * 2; + + /** + * Rename a single file. + *

+ * source is found, dest not found, copy adds a + * metadata request. + */ + public static final int RENAME_SINGLE_FILE_RENAME_H = + FILESTATUS_FILE_PROBE_H + GETFILESTATUS_FNFE_H + 1; + + /** + * Create file no overwrite head : {@value}. + */ + public static final int CREATE_FILE_OVERWRITE_H = 0; + + /** + * Create file no overwrite list : {@value}. + */ + public static final int CREATE_FILE_OVERWRITE_L = FILESTATUS_DIR_PROBE_L; + + /** + * Create file no overwrite head : {@value}. + */ + public static final int CREATE_FILE_NO_OVERWRITE_H = FILESTATUS_FILE_PROBE_H; + + /** + * Create file no overwrite list : {@value}. + */ + public static final int CREATE_FILE_NO_OVERWRITE_L = FILESTATUS_DIR_PROBE_L; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/OperationCostValidator.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/OperationCostValidator.java similarity index 84% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/OperationCostValidator.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/OperationCostValidator.java index 79b3eb629a2a3..fb0c57dcd8914 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/OperationCostValidator.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/OperationCostValidator.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.test; +package org.apache.hadoop.fs.s3a.test.costs; import java.util.ArrayList; import java.util.Arrays; @@ -27,7 +27,10 @@ import java.util.stream.Collectors; import org.assertj.core.api.Assumptions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.s3a.ITestS3AFileOperationCost; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; @@ -40,8 +43,11 @@ */ public class OperationCostValidator { + private static final Logger LOG = + LoggerFactory.getLogger(OperationCostValidator.class); + /** - * The empty probe. This is a no-op. + * The empty probe. */ public static final ExpectedProbe EMPTY_PROBE = new ProbeList(new ArrayList<>()); @@ -91,7 +97,8 @@ public S3ATestUtils.MetricDiff get(Statistic stat) { */ public T verify( Callable eval, - ExpectedProbe... expected) throws Exception { + ExpectedProbe... expectedA) throws Exception { + List expected = Arrays.asList(expectedA); resetMetricDiffs(); // verify that 1+ probe is enabled assumeProbesEnabled(expected); @@ -99,7 +106,12 @@ public T verify( // evaluate it T r = eval.call(); // build the text for errors - String text = r != null ? r.toString() : "operation returning null"; + String text = + "operation returning " + + (r != null ? r.toString() : "null"); + LOG.info("{}", text); + LOG.info("state {}", this); + LOG.info("probes {}", expected); for (ExpectedProbe ed : expected) { ed.verify(this, text); } @@ -112,13 +124,16 @@ public T verify( * If none of them are enabled, the evaluation will be skipped. * @param expected list of expected probes */ - private void assumeProbesEnabled(ExpectedProbe[] expected) { + private void assumeProbesEnabled(List expected) { boolean enabled = false; for (ExpectedProbe ed : expected) { enabled |= ed.isEnabled(); } + String pstr = expected.stream() + .map(Object::toString) + .collect(Collectors.joining(", ")); Assumptions.assumeThat(enabled) - .describedAs("metrics to probe for") + .describedAs("metrics to probe for are not enabled in %s", pstr) .isTrue(); } @@ -230,7 +245,9 @@ public static ExpectedProbe probe( public static ExpectedProbe probes( final boolean enabled, final ExpectedProbe...plist) { - return new ProbeList(Arrays.asList(plist)); + return enabled + ? new ProbeList(Arrays.asList(plist)) + : EMPTY_PROBE; } /** @@ -291,16 +308,33 @@ public int getExpected() { public boolean isEnabled() { return true; } + + @Override + public String toString() { + String sb = "ExpectSingleStatistic{" + + statistic + + ", expected=" + expected + + ", enabled=" + isEnabled() + + '}'; + return sb; + } } /** * A list of probes; the verify operation - * verifies them all + * verifies them all. */ public static class ProbeList implements ExpectedProbe { + /** + * Probe list. + */ private final List probes; + /** + * Constructor. + * @param probes probe list. + */ public ProbeList(final List probes) { this.probes = probes; } @@ -323,5 +357,13 @@ public boolean isEnabled() { } return enabled; } + + @Override + public String toString() { + String pstr = probes.stream() + .map(Object::toString) + .collect(Collectors.joining(", ")); + return "ProbeList{" + pstr + '}'; + } } } From f21b78701723697e436f92ddef561e90692fd947 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 16 Jul 2020 15:17:16 +0100 Subject: [PATCH 03/20] HADOOP-13230 operation cost tests -always have one run in auth mode -cleaning up and javadocing -plan to pull up a base class and then move the delete and rename ops away Change-Id: I7fbf885de73eccc0bc842a4987ac12682ff1c63c --- .../fs/s3a/ITestS3AFileOperationCost.java | 292 ++++++++++-------- .../fs/s3a/test/costs/HeadListCosts.java | 11 +- .../test/costs/OperationCostValidator.java | 133 ++++++-- 3 files changed, 288 insertions(+), 148 deletions(-) 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 f1ec75affedc0..c38050a703702 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 @@ -47,6 +47,7 @@ import java.util.concurrent.Callable; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; +import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH; import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY; import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE; import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP; @@ -79,10 +80,10 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase { @Parameterized.Parameters(name = "{0}") public static Collection params() { return Arrays.asList(new Object[][]{ - {"raw-keep-markers", false, true}, - {"raw-delete-markers", false, false}, - {"guarded-keep-markers", true, true}, - {"guarded-delete-markers", true, false} + {"raw-keep-markers", false, true, false}, + {"raw-delete-markers", false, false, false}, + {"nonauth-keep-markers", true, true, false}, + {"auth-delete-markers", true, false, true} }); } @@ -99,7 +100,7 @@ public static Collection params() { /** * Is this an auth mode test run? */ - private boolean authoritative; + private final boolean authoritative; /* probe states calculated from the configuration options. */ boolean isGuarded; @@ -114,9 +115,11 @@ public static Collection params() { public ITestS3AFileOperationCost(final String name, final boolean s3guard, - final boolean keepMarkers) { + final boolean keepMarkers, + final boolean authoritative) { this.s3guard = s3guard; this.keepMarkers = keepMarkers; + this.authoritative = authoritative; } @Override @@ -132,7 +135,9 @@ public Configuration createConfiguration() { } // directory marker options removeBaseAndBucketOverrides(bucketName, conf, - DIRECTORY_MARKER_POLICY); + DIRECTORY_MARKER_POLICY, + METADATASTORE_AUTHORITATIVE, + AUTHORITATIVE_PATH); conf.set(DIRECTORY_MARKER_POLICY, keepMarkers ? DIRECTORY_MARKER_POLICY_KEEP @@ -152,7 +157,6 @@ public void setup() throws Exception { } S3AFileSystem fs = getFileSystem(); skipDuringFaultInjection(fs); - authoritative = fs.allowAuthoritative(new Path("/")); // build up the states isGuarded = isGuarded(); @@ -165,14 +169,16 @@ public void setup() throws Exception { isDeleting = !isKeeping; + // insert new metrics so as to keep the list sorted costValidator = OperationCostValidator.builder(getFileSystem()) - .withMetrics(INVOCATION_COPY_FROM_LOCAL_FILE, - OBJECT_COPY_REQUESTS, - OBJECT_DELETE_REQUESTS, + .withMetrics( DIRECTORIES_CREATED, DIRECTORIES_DELETED, FAKE_DIRECTORIES_DELETED, FILES_DELETED, + INVOCATION_COPY_FROM_LOCAL_FILE, + OBJECT_COPY_REQUESTS, + OBJECT_DELETE_REQUESTS, OBJECT_LIST_REQUESTS, OBJECT_METADATA_REQUESTS, OBJECT_PUT_BYTES, @@ -216,10 +222,6 @@ public boolean isNonAuth() { return isNonAuth; } - public boolean isKeeping() { - return isKeeping; - } - public boolean isDeleting() { return isDeleting; } @@ -228,15 +230,32 @@ public boolean isKeepingMarkers() { return keepMarkers; } + + private final Object metricSummary = costValidator; + + /** + * A special object whose toString() value is the current + * state of the metrics. + */ + protected Object getMetricSummary() { + return metricSummary; + } + + /** + * Test the cost of {@code listLocatedStatus(file)}. + * There's a minor inefficiency in that calling this on + * a file in S3Guard still executes a LIST call, even + * though the file record is in the store. + */ @Test public void testCostOfLocatedFileStatusOnFile() throws Throwable { describe("performing listLocatedStatus on a file"); Path file = file(methodPath()); S3AFileSystem fs = getFileSystem(); verifyMetrics(() -> fs.listLocatedStatus(file), - raw(OBJECT_LIST_REQUESTS, LIST_LOCATED_STATUS_L), - nonauth(OBJECT_LIST_REQUESTS, LIST_LOCATED_STATUS_L), - raw(OBJECT_METADATA_REQUESTS, FILESTATUS_FILE_PROBE_H)); + rawHeadList(FILESTATUS_FILE_PROBE_H, LIST_LOCATED_STATUS_L), + authHeadList(0, LIST_LOCATED_STATUS_L), + nonauthHeadList(0, LIST_LOCATED_STATUS_L)); } @Test @@ -246,12 +265,10 @@ public void testCostOfListLocatedStatusOnEmptyDir() throws Throwable { S3AFileSystem fs = getFileSystem(); verifyMetrics(() -> fs.listLocatedStatus(dir), - expectRawHeadList(FILESTATUS_FILE_PROBE_H, + rawHeadList(FILESTATUS_FILE_PROBE_H, LIST_LOCATED_STATUS_L + GETFILESTATUS_DIR_L), - guarded(OBJECT_METADATA_REQUESTS, 0), - authoritative(OBJECT_LIST_REQUESTS, 0), - nonauth(OBJECT_LIST_REQUESTS, - LIST_LOCATED_STATUS_L)); + authHeadList(0, 0), + nonauthHeadList(0, LIST_LOCATED_STATUS_L)); } @Test @@ -262,12 +279,9 @@ public void testCostOfListLocatedStatusOnNonEmptyDir() throws Throwable { Path file = file(new Path(dir, "file.txt")); verifyMetrics(() -> fs.listLocatedStatus(dir), - always(OBJECT_METADATA_REQUESTS, 0), - raw(OBJECT_LIST_REQUESTS, - LIST_LOCATED_STATUS_L), - authoritative(OBJECT_LIST_REQUESTS, 0), - nonauth(OBJECT_LIST_REQUESTS, - LIST_LOCATED_STATUS_L)); + rawHeadList(0, LIST_LOCATED_STATUS_L), + authHeadList(0, 0), + nonauthHeadList(0, LIST_LOCATED_STATUS_L)); } @Test @@ -278,7 +292,7 @@ public void testCostOfListFilesOnFile() throws Throwable { touch(fs, file); verifyMetrics(() -> fs.listFiles(file, true), - expectRawHeadList(GETFILESTATUS_SINGLE_FILE_H, + rawHeadList(GETFILESTATUS_SINGLE_FILE_H, LIST_LOCATED_STATUS_L), authoritative(OBJECT_LIST_REQUESTS, 0), nonauth(OBJECT_LIST_REQUESTS, @@ -294,10 +308,10 @@ public void testCostOfListFilesOnEmptyDir() throws Throwable { fs.mkdirs(dir); verifyMetrics(() -> fs.listFiles(dir, true), - expectRawHeadList(GETFILESTATUS_EMPTY_DIR_H, + rawHeadList(GETFILESTATUS_EMPTY_DIR_H, LIST_FILES_L + GETFILESTATUS_EMPTY_DIR_L), - expectAuthHeadList(0, 0), - expectNonauthHeadList(0, LIST_FILES_L)); + authHeadList(0, 0), + nonauthHeadList(0, LIST_FILES_L)); } @Test @@ -310,9 +324,9 @@ public void testCostOfListFilesOnNonEmptyDir() throws Throwable { touch(fs, file); verifyMetrics(() -> fs.listFiles(dir, true), - expectRawHeadList(0, LIST_FILES_L), - expectAuthHeadList(0, 0), - expectNonauthHeadList(0, LIST_FILES_L)); + rawHeadList(0, LIST_FILES_L), + authHeadList(0, 0), + nonauthHeadList(0, LIST_FILES_L)); } @Test @@ -322,7 +336,7 @@ public void testCostOfListFilesOnNonExistingDir() throws Throwable { S3AFileSystem fs = getFileSystem(); verifyMetricsIntercepting(FileNotFoundException.class, "", () -> fs.listFiles(dir, true), - expectRawHeadList(GETFILESTATUS_FNFE_H, + rawHeadList(GETFILESTATUS_FNFE_H, GETFILESTATUS_FNFE_L + LIST_FILES_L) ); @@ -332,7 +346,7 @@ public void testCostOfListFilesOnNonExistingDir() throws Throwable { public void testCostOfGetFileStatusOnFile() throws Throwable { describe("performing getFileStatus on a file"); Path simpleFile = file(methodPath()); - S3AFileStatus status = verifyRawGetFileStatus(simpleFile, true, + S3AFileStatus status = verifyRawInnerGetFileStatus(simpleFile, true, StatusProbeEnum.ALL, GETFILESTATUS_SINGLE_FILE_H, GETFILESTATUS_SINGLE_FILE_L); @@ -344,29 +358,29 @@ public void testCostOfGetFileStatusOnFile() throws Throwable { public void testCostOfGetFileStatusOnEmptyDir() throws Throwable { describe("performing getFileStatus on an empty directory"); Path dir = dir(methodPath()); - S3AFileStatus status = verifyRawGetFileStatus(dir, true, + S3AFileStatus status = verifyRawInnerGetFileStatus(dir, true, StatusProbeEnum.ALL, GETFILESTATUS_MARKER_H, GETFILESTATUS_EMPTY_DIR_L); assertSame("not empty: " + status, Tristate.TRUE, status.isEmptyDirectory()); // but now only ask for the directories and the file check is skipped. - verifyRawGetFileStatus(dir, false, + verifyRawInnerGetFileStatus(dir, false, StatusProbeEnum.DIRECTORIES, FILESTATUS_DIR_PROBE_H, GETFILESTATUS_EMPTY_DIR_L); // now look at isFile/isDir against the same entry - isDir(dir, true, 0, + isDir(dir, true, FILESTATUS_DIR_PROBE_H, GETFILESTATUS_EMPTY_DIR_L); isFile(dir, false, - GETFILESTATUS_SINGLE_FILE_H, GETFILESTATUS_SINGLE_FILE_L); + FILESTATUS_FILE_PROBE_H, FILESTATUS_FILE_PROBE_L); } @Test public void testCostOfGetFileStatusOnMissingFile() throws Throwable { describe("performing getFileStatus on a missing file"); - verifyRawGetFileStatusFNFE(methodPath(), false, + interceptRawGetFileStatusFNFE(methodPath(), false, StatusProbeEnum.ALL, GETFILESTATUS_FNFE_H, GETFILESTATUS_FNFE_L); @@ -390,7 +404,7 @@ public void testCostOfGetFileStatusOnNonEmptyDir() throws Throwable { describe("performing getFileStatus on a non-empty directory"); Path dir = dir(methodPath()); file(new Path(dir, "simple.txt")); - S3AFileStatus status = verifyRawGetFileStatus(dir, true, + S3AFileStatus status = verifyRawInnerGetFileStatus(dir, true, StatusProbeEnum.ALL, GETFILESTATUS_DIR_H, GETFILESTATUS_DIR_L); assertEmptyDirStatus(status, Tristate.FALSE); @@ -410,15 +424,17 @@ public void testDeleteFile() throws Throwable { // still be there Path simpleFile = file(new Path(dir, "simple.txt")); + boolean rawAndKeeping = isRaw && isKeeping; + boolean rawAndDeleting = isRaw && isDeleting; verifyMetrics(() -> { fs.delete(simpleFile, false); - return "after fs.delete(simpleFile) " + metricSummary; + return "after fs.delete(simpleFile) " + getMetricSummary(); }, // delete file. For keeping: that's it - probe(isRaw && isKeeping, OBJECT_METADATA_REQUESTS, + probe(rawAndKeeping, OBJECT_METADATA_REQUESTS, FILESTATUS_FILE_PROBE_H), // if deleting markers, look for the parent too - probe(isRaw && isDeleting, OBJECT_METADATA_REQUESTS, + probe(rawAndDeleting, OBJECT_METADATA_REQUESTS, FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H), raw(OBJECT_LIST_REQUESTS, FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L), @@ -436,7 +452,7 @@ public void testDeleteFile() throws Throwable { + DELETE_MARKER_REQUEST) ); // there is an empty dir for a parent - S3AFileStatus status = verifyRawGetFileStatus(dir, true, + S3AFileStatus status = verifyRawInnerGetFileStatus(dir, true, StatusProbeEnum.ALL, GETFILESTATUS_DIR_H, GETFILESTATUS_DIR_L); assertEmptyDirStatus(status, Tristate.TRUE); } @@ -485,7 +501,7 @@ public void testDirMarkersSubdir() throws Throwable { // one dir created, possibly a parent removed verifyMetrics(() -> { mkdirs(subDir); - return "after mkdir(subDir) " + metricSummary; + return "after mkdir(subDir) " + getMetricSummary(); }, always(DIRECTORIES_CREATED, 1), always(DIRECTORIES_DELETED, 0), @@ -509,7 +525,7 @@ public void testDirMarkersFileCreation() throws Throwable { verifyMetrics(() -> { file(new Path(srcDir, "source.txt")); - return "after touch(fs, srcFilePath) " + metricSummary; + return "after touch(fs, srcFilePath) " + getMetricSummary(); }, always(DIRECTORIES_CREATED, 0), always(DIRECTORIES_DELETED, 0), @@ -525,7 +541,6 @@ public void testDirMarkersFileCreation() throws Throwable { public void testRenameFileToDifferentDirectory() throws Throwable { describe("rename a file to a different directory, " + "keeping the source dir present"); - S3AFileSystem fs = getFileSystem(); Path baseDir = dir(methodPath()); @@ -554,9 +569,7 @@ public void testRenameFileToDifferentDirectory() throws Throwable { // as srcFile2 exists, the parent dir of srcFilePath must not be created. verifyMetrics(() -> execRename(srcFilePath, destFilePath), - raw(OBJECT_METADATA_REQUESTS, - RENAME_SINGLE_FILE_RENAME_H), - raw(OBJECT_LIST_REQUESTS, + rawHeadList(RENAME_SINGLE_FILE_RENAME_H, RENAME_SINGLE_FILE_RENAME_DIFFERENT_DIR_L), always(DIRECTORIES_CREATED, 0), always(DIRECTORIES_DELETED, 0), @@ -594,8 +607,8 @@ public void testRenameSameDirectory() throws Throwable { Path destFile = new Path(parent2, "dest"); verifyMetrics(() -> execRename(sourceFile, destFile), - raw(OBJECT_METADATA_REQUESTS, RENAME_SINGLE_FILE_RENAME_H), - raw(OBJECT_LIST_REQUESTS, GETFILESTATUS_FNFE_L), + rawHeadList(RENAME_SINGLE_FILE_RENAME_H, + RENAME_SINGLE_FILE_RENAME_SAME_DIR_L), always(OBJECT_COPY_REQUESTS, 1), always(DIRECTORIES_CREATED, 0), always(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), @@ -616,11 +629,11 @@ public void testCostOfRootRename() throws Throwable { verifyMetrics(() -> { fs.rename(src, dest); - return "after fs.rename(/src,/dest) " + metricSummary; + return "after fs.rename(/src,/dest) " + getMetricSummary(); }, // TWO HEAD for exists, one for source MD in copy - expectRawHeadList(RENAME_SINGLE_FILE_RENAME_H, - GETFILESTATUS_FNFE_L)); + rawHeadList(RENAME_SINGLE_FILE_RENAME_H, + GETFILESTATUS_FNFE_L), // here we expect there to be no fake directories always(DIRECTORIES_CREATED, 0), // one for the renamed file only @@ -635,14 +648,14 @@ public void testCostOfRootRename() throws Throwable { // delete that destination file, assert only the file delete was issued verifyMetrics(() -> { fs.delete(dest, false); - return "after fs.delete(/dest) " + metricSummary; + return "after fs.delete(/dest) " + getMetricSummary(); }, always(DIRECTORIES_CREATED, 0), always(DIRECTORIES_DELETED, 0), always(FAKE_DIRECTORIES_DELETED, 0), always(FILES_DELETED, 1), always(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), - expectRawHeadList(FILESTATUS_FILE_PROBE_H, 0)); /* no need to look at parent. */ + rawHeadList(FILESTATUS_FILE_PROBE_H, 0)); /* no need to look at parent. */ } finally { fs.delete(src, false); @@ -659,19 +672,19 @@ public void testDirProbes() throws Throwable { Path emptydir = dir(methodPath()); // head probe fails - verifyRawGetFileStatusFNFE(emptydir, false, + interceptRawGetFileStatusFNFE(emptydir, false, StatusProbeEnum.HEAD_ONLY, FILESTATUS_FILE_PROBE_H, FILESTATUS_FILE_PROBE_L); // a LIST will find it and declare as empty - S3AFileStatus status = verifyRawGetFileStatus(emptydir, true, + S3AFileStatus status = verifyRawInnerGetFileStatus(emptydir, true, StatusProbeEnum.LIST_ONLY, 0, GETFILESTATUS_EMPTY_DIR_L); assertEmptyDirStatus(status, Tristate.TRUE); // skip all probes and expect no operations to take place - verifyRawGetFileStatusFNFE(emptydir, false, + interceptRawGetFileStatusFNFE(emptydir, false, EnumSet.noneOf(StatusProbeEnum.class), 0, 0); @@ -680,7 +693,7 @@ public void testDirProbes() throws Throwable { String emptyDirTrailingSlash = fs.pathToKey(emptydir.getParent()) + "/" + emptydir.getName() + "/"; // A HEAD request does not probe for keys with a trailing / - verifyRawHeadListIntercepting(FileNotFoundException.class, "", + interceptRawHeadList(FileNotFoundException.class, "", 0, 0, () -> fs.s3GetFileStatus(emptydir, emptyDirTrailingSlash, StatusProbeEnum.HEAD_ONLY, null, false)); @@ -730,7 +743,7 @@ public void testCreateCostFileExists() throws Throwable { // now there is a file there, an attempt with overwrite == false will // fail on the first HEAD. - verifyRawHeadListIntercepting(FileAlreadyExistsException.class, "", + interceptRawHeadList(FileAlreadyExistsException.class, "", FILESTATUS_FILE_PROBE_H, 0, () -> file(testFile, false)); } @@ -743,7 +756,7 @@ public void testCreateCostDirExists() throws Throwable { // now there is a file there, an attempt with overwrite == false will // fail on the first HEAD. - verifyRawHeadListIntercepting(FileAlreadyExistsException.class, "", + interceptRawHeadList(FileAlreadyExistsException.class, "", GETFILESTATUS_MARKER_H, GETFILESTATUS_EMPTY_DIR_L, () -> file(testFile, false)); } @@ -770,7 +783,7 @@ public void testCreateBuilder() throws Throwable { // now there is a file there, an attempt with overwrite == false will // fail on the first HEAD. - verifyRawHeadListIntercepting(FileAlreadyExistsException.class, "", + interceptRawHeadList(FileAlreadyExistsException.class, "", FILESTATUS_FILE_PROBE_H, 0, () -> buildFile(testFile, false, true, FILESTATUS_FILE_PROBE_H, 0)); } @@ -818,32 +831,6 @@ public void testCostOfGlobStatusNoSymlinkResolution() throws Throwable { fs.globStatus(basePath.suffix("/*"))); } - - /** - * Touch a file, overwriting. - * @param path path - * @return path to new object. - */ - private Path create(Path path) throws Exception { - return create(path, true, - CREATE_FILE_OVERWRITE_H, - CREATE_FILE_OVERWRITE_L); - } - - /** - * Create then close the file. - * @param path path - * @param overwrite overwrite flag - * @param head expected head count - * @param list expected list count - * @return path to new object. - */ - private Path create(Path path, boolean overwrite, - int head, int list) throws Exception { - return verifyRawHeadList(head, list, () -> - file(path, overwrite)); - } - /** * Create then close the file through the builder API. * @param path path @@ -904,6 +891,32 @@ private Path file(Path path, final boolean overwrite) } + /** + * Touch a file, overwriting. + * @param path path + * @return path to new object. + */ + private Path create(Path path) throws Exception { + return create(path, true, + CREATE_FILE_OVERWRITE_H, + CREATE_FILE_OVERWRITE_L); + } + + /** + * Create then close the file. + * @param path path + * @param overwrite overwrite flag + * @param head expected head count + * @param list expected list count + * @return path to new object. + */ + private Path create(Path path, boolean overwrite, + int head, int list) throws Exception { + return verifyRawHeadList(head, list, () -> + file(path, overwrite)); + } + + /** * Execute rename, returning the current metrics. * For use in l-expressions. @@ -914,7 +927,7 @@ private Path file(Path path, final boolean overwrite) public String execRename(final Path source, final Path dest) throws IOException { getFileSystem().rename(source, dest); - return String.format("rename(%s, %s): %s", dest, source, metricSummary); + return String.format("rename(%s, %s): %s", dest, source, getMetricSummary()); } /** @@ -943,7 +956,7 @@ private void resetStatistics() { private T verifyMetrics( Callable eval, OperationCostValidator.ExpectedProbe... expected) throws Exception { - return costValidator.verify(eval, expected); + return costValidator.exec(eval, expected); } @@ -965,7 +978,7 @@ private E verifyMetricsIntercepting( String text, Callable eval, OperationCostValidator.ExpectedProbe... expected) throws Exception { - return costValidator.verifyIntercepting(clazz, text, eval, expected); + return costValidator.intercepting(clazz, text, eval, expected); } /** @@ -980,14 +993,14 @@ private E verifyMetricsIntercepting( * @return the exception caught. * @throws Exception any other exception */ - private E verifyRawHeadListIntercepting( + private E interceptRawHeadList( Class clazz, String text, int head, int list, Callable eval) throws Exception { return verifyMetricsIntercepting(clazz, text, eval, - expectRawHeadList(head, list)); + rawHeadList(head, list)); } /** @@ -1010,10 +1023,8 @@ private E verifyRawHeadListIntercepting( * @return a probe list */ private OperationCostValidator.ExpectedProbe - expectHeadList(int head, int list) { - return probes(true, - probe(OBJECT_METADATA_REQUESTS, head), - probe(OBJECT_LIST_REQUESTS, list)); + alwaysHeadList(int head, int list) { + return expectHeadList(true, head, list); } /** @@ -1023,7 +1034,7 @@ private E verifyRawHeadListIntercepting( * @return a probe list */ private OperationCostValidator.ExpectedProbe - expectRawHeadList(int head, int list) { + rawHeadList(int head, int list) { return expectHeadList(isRaw(), head, list); } @@ -1034,7 +1045,7 @@ private E verifyRawHeadListIntercepting( * @return a probe list */ private OperationCostValidator.ExpectedProbe - expectAuthHeadList(int head, int list) { + authHeadList(int head, int list) { return expectHeadList(isAuthoritative(), head, list); } @@ -1046,7 +1057,7 @@ private E verifyRawHeadListIntercepting( * @return a probe list */ private OperationCostValidator.ExpectedProbe - expectNonauthHeadList(int head, int list) { + nonauthHeadList(int head, int list) { return expectHeadList(isNonAuth(), head, list); } @@ -1064,31 +1075,61 @@ private T verifyRawHeadList( int list, Callable eval) throws Exception { return verifyMetrics(eval, - expectRawHeadList(head, list)); + rawHeadList(head, list)); } /** - * Execute innerGetFileStatus for the given probes - * and expect in raw FS to have the specific HEAD/LIST count. + * Execute {@link S3AFileSystem#innerGetFileStatus(Path, boolean, Set)} + * for the given probes. + * expect the specific HEAD/LIST count. + *

+ * Raw FS only. + *

+ * @param path path + * @param needEmptyDirectoryFlag look for empty directory + * @param probes file status probes to perform + * @param head expected head calls + * @param list expected list calls + * @return the status */ - public S3AFileStatus verifyRawGetFileStatus(final Path path, + public S3AFileStatus verifyRawInnerGetFileStatus( + Path path, boolean needEmptyDirectoryFlag, - Set probes, int head, int list) throws Exception { + Set probes, + int head, + int list) throws Exception { return verifyRawHeadList(head, list, () -> - getFileSystem().innerGetFileStatus(path, needEmptyDirectoryFlag, + getFileSystem().innerGetFileStatus( + path, + needEmptyDirectoryFlag, probes)); } /** - * Execute innerGetFileStatus for the given probes and expect failure - * and expect in raw FS to have the specific HEAD/LIST count. + * Execute {@link S3AFileSystem#innerGetFileStatus(Path, boolean, Set)} + * for the given probes -expect a FileNotFoundException, + * and the specific HEAD/LIST count. + *

+ * Raw FS only. + *

+ * @param path path + * @param needEmptyDirectoryFlag look for empty directory + * @param probes file status probes to perform + * @param head expected head calls + * @param list expected list calls + * @return the status */ - public void verifyRawGetFileStatusFNFE(final Path path, + public void interceptRawGetFileStatusFNFE( + Path path, boolean needEmptyDirectoryFlag, - Set probes, int head, int list) throws Exception { - verifyRawHeadListIntercepting(FileNotFoundException.class, "", + Set probes, + int head, + int list) throws Exception { + interceptRawHeadList(FileNotFoundException.class, "", head, list, () -> - getFileSystem().innerGetFileStatus(path, needEmptyDirectoryFlag, + getFileSystem().innerGetFileStatus( + path, + needEmptyDirectoryFlag, probes)); } @@ -1145,7 +1186,7 @@ private OperationCostValidator.ExpectedProbe always( */ private OperationCostValidator.ExpectedProbe raw( final Statistic Statistic, final int expected) { - return probe(isRaw, Statistic, expected); + return probe(isRaw(), Statistic, expected); } /** @@ -1157,7 +1198,7 @@ private OperationCostValidator.ExpectedProbe raw( private OperationCostValidator.ExpectedProbe guarded( final Statistic Statistic, final int expected) { - return probe(isGuarded, Statistic, expected); + return probe(isGuarded(), Statistic, expected); } /** @@ -1169,7 +1210,7 @@ private OperationCostValidator.ExpectedProbe guarded( private OperationCostValidator.ExpectedProbe authoritative( final Statistic Statistic, final int expected) { - return probe(isAuthoritative, Statistic, expected); + return probe(isAuthoritative(), Statistic, expected); } /** @@ -1205,14 +1246,7 @@ private OperationCostValidator.ExpectedProbe keeping( private OperationCostValidator.ExpectedProbe deleting( final Statistic Statistic, final int expected) { - return probe(isDeleting, Statistic, expected); + return probe(isDeleting(), Statistic, expected); } - - /** - * A special object whose toString() value is the current - * state of the metrics. - */ - private final Object metricSummary = costValidator; - } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/HeadListCosts.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/HeadListCosts.java index 0033e004072c7..ef15a725d7acf 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/HeadListCosts.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/HeadListCosts.java @@ -62,7 +62,7 @@ public class HeadListCosts { public static final int GETFILESTATUS_SINGLE_FILE_H = FILESTATUS_FILE_PROBE_H; - public static final int GETFILESTATUS_SINGLE_FILE_L = 0; + public static final int GETFILESTATUS_SINGLE_FILE_L = FILESTATUS_FILE_PROBE_L; public static final int DELETE_OBJECT_REQUEST = 1; @@ -82,6 +82,15 @@ public class HeadListCosts { public static final int RENAME_SINGLE_FILE_RENAME_DIFFERENT_DIR_L = GETFILESTATUS_FNFE_L + GETFILESTATUS_DIR_L * 2; + /** + * Cost of renaming a file to a different directory. + *

+ * LIST on dest not found, look for dest dir, and then, at + * end of rename, whether a parent dir needs to be created. + */ + public static final int RENAME_SINGLE_FILE_RENAME_SAME_DIR_L = + GETFILESTATUS_FNFE_L; + /** * Rename a single file. *

diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/OperationCostValidator.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/OperationCostValidator.java index fb0c57dcd8914..63f9f77cb6eb2 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/OperationCostValidator.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/OperationCostValidator.java @@ -30,7 +30,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.fs.s3a.ITestS3AFileOperationCost; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; @@ -40,30 +39,67 @@ /** * Support for declarative assertions about operation cost. + *

+ * Usage: A builder is used to declare the set of statistics + * to be monitored in the filesystem. + *

+ * A call to {@link #exec(Callable, ExpectedProbe...)} + * executes the callable if 1+ probe is enabled; after + * invocation the probes are validated. + * The result of the callable is returned. + *

+ * A call of {@link #intercepting(Class, String, Callable, ExpectedProbe...)} + * Invokes the callable if 1+ probe is enabled, expects an exception + * to be raised and then verifies metrics declared in the probes. + *

+ * Probes are built up from the static method to create probes + * for metrics: + *
    + *
  • {@link #probe(boolean, Statistic, int)}
  • + *
  • {@link #probe(Statistic, int)}
  • + *
  • {@link #probes(boolean, ExpectedProbe...)} (Statistic, int)}
  • + *
  • {@link #always()}
  • + *
+ * If any probe evaluates to false, an assertion is raised. + *

+ * When this happens: look in the logs! + * The logs will contain the whole set of metrics, the probe details + * and the result of the call. */ -public class OperationCostValidator { +public final class OperationCostValidator { private static final Logger LOG = LoggerFactory.getLogger(OperationCostValidator.class); /** - * The empty probe. + * The empty probe: declared as disabled. */ - public static final ExpectedProbe EMPTY_PROBE = - new ProbeList(new ArrayList<>()); + private static final ExpectedProbe EMPTY_PROBE = + new EmptyProbe("empty", false); + /** + * A probe which is always enabled. + */ + private static final ExpectedProbe ALWAYS_PROBE = + new EmptyProbe("always", true); + + /** + * The map of metric diffs to track. + */ private final Map metricDiffs = new TreeMap<>(); - public OperationCostValidator(Builder builder) { + /** + * Build the instance. + * @param builder builder containing all options. + */ + private OperationCostValidator(Builder builder) { builder.metrics.forEach(stat -> metricDiffs.put(stat.getSymbol(), - new S3ATestUtils.MetricDiff(builder.fs, stat)) - ); + new S3ATestUtils.MetricDiff(builder.filesystem, stat))); builder.metrics.clear(); } - /** * Reset all the metrics being tracked. */ @@ -84,7 +120,6 @@ public S3ATestUtils.MetricDiff get(Statistic stat) { return diff; } - /** * Execute a closure and verify the metrics. *

@@ -95,7 +130,7 @@ public S3ATestUtils.MetricDiff get(Statistic stat) { * @param return type. * @return the result of the evaluation */ - public T verify( + public T exec( Callable eval, ExpectedProbe... expectedA) throws Exception { List expected = Arrays.asList(expectedA); @@ -150,13 +185,13 @@ private void assumeProbesEnabled(List expected) { * @return the exception caught. * @throws Exception any other exception */ - public E verifyIntercepting( + public E intercepting( Class clazz, String text, Callable eval, ExpectedProbe... expected) throws Exception { - return verify(() -> + return exec(() -> intercept(clazz, text, eval), expected); } @@ -183,31 +218,62 @@ public static Builder builder(S3AFileSystem fs) { */ public static final class Builder { - private final List metrics = new ArrayList<>(); + /** + * Filesystem. + */ + private final S3AFileSystem filesystem; - private final S3AFileSystem fs; + /** + * Metrics to create. + */ + private final List metrics = new ArrayList<>(); - public Builder(final S3AFileSystem fs) { - this.fs = requireNonNull(fs); + /** + * Create with a required filesystem. + * @param filesystem monitored filesystem + */ + public Builder(final S3AFileSystem filesystem) { + this.filesystem = requireNonNull(filesystem); } + /** + * Add a single metric. + * @param statistic statistic to monitor. + * @return this + */ public Builder withMetric(Statistic statistic) { return withMetric(statistic); } - + /** + * Add a varargs list of metrics. + * @param stat statistics to monitor. + * @return this. + */ public Builder withMetrics(Statistic...stats) { metrics.addAll(Arrays.asList(stats)); return this; } + /** + * Instantiate. + * @return the validator. + */ public OperationCostValidator build() { return new OperationCostValidator(this); } } + /** + * Get the "always" probe. + * @return a probe which always triggers execution. + */ + public static ExpectedProbe always() { + return ALWAYS_PROBE; + } + /** * Create a probe of a statistic which is always enabled. * @param statistic statistic to check. @@ -366,4 +432,35 @@ public String toString() { return "ProbeList{" + pstr + '}'; } } + + /** + * The empty probe always runs; it can be used to force + * a verification to execute. + */ + private static final class EmptyProbe implements ExpectedProbe { + + private final String name; + + private final boolean enabled; + + private EmptyProbe(final String name, boolean enabled) { + this.name = name; + this.enabled = enabled; + } + + @Override + public void verify(final OperationCostValidator diffs, + final String message) { + } + + @Override + public boolean isEnabled() { + return enabled; + } + + @Override + public String toString() { + return name; + } + } } From 4ec17cd3bef670fa7c3192112bce581ed01dd762 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 16 Jul 2020 16:05:44 +0100 Subject: [PATCH 04/20] HADOOP-13230 factor out base class for cost tests and split suite by category Change-Id: Ifadd671a6e39a4a8f4c4a6ab600132bc812ef79e --- .../fs/s3a/ITestS3AFileOperationCost.java | 828 +----------------- .../apache/hadoop/fs/s3a/S3ATestUtils.java | 22 + .../s3a/test/costs/AbstractS3ACostTest.java | 640 ++++++++++++++ .../fs/s3a/test/costs/ITestS3ADeleteCost.java | 164 ++++ .../fs/s3a/test/costs/ITestS3ARenameCost.java | 197 +++++ 5 files changed, 1030 insertions(+), 821 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/AbstractS3ACostTest.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/ITestS3ADeleteCost.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/ITestS3ARenameCost.java 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 c38050a703702..95e1feeb50a4c 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 @@ -18,17 +18,14 @@ package org.apache.hadoop.fs.s3a; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FSDataOutputStreamBuilder; + import org.apache.hadoop.fs.FileAlreadyExistsException; 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.impl.StatusProbeEnum; -import org.apache.hadoop.fs.s3a.test.costs.OperationCostValidator; +import org.apache.hadoop.fs.s3a.test.costs.AbstractS3ACostTest; + -import org.assertj.core.api.Assertions; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -37,43 +34,28 @@ import java.io.File; import java.io.FileNotFoundException; -import java.io.IOException; import java.net.URI; import java.util.Arrays; import java.util.Collection; import java.util.EnumSet; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.Callable; + import static org.apache.hadoop.fs.contract.ContractTestUtils.*; -import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH; -import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY; -import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE; -import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP; -import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE; -import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; import static org.apache.hadoop.fs.s3a.Statistic.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.test.costs.HeadListCosts.*; -import static org.apache.hadoop.fs.s3a.test.costs.OperationCostValidator.probe; -import static org.apache.hadoop.fs.s3a.test.costs.OperationCostValidator.probes; -import static org.apache.hadoop.test.AssertExtensions.dynamicDescription; import static org.apache.hadoop.test.GenericTestUtils.getTestDir; /** - * Use metrics to assert about the cost of file status queries. - * {@link S3AFileSystem#getFileStatus(Path)}. + * Use metrics to assert about the cost of file API calls. * Parameterized on guarded vs raw. and directory marker keep vs delete */ @RunWith(Parameterized.class) -public class ITestS3AFileOperationCost extends AbstractS3ATestBase { +public class ITestS3AFileOperationCost extends AbstractS3ACostTest { private static final Logger LOG = LoggerFactory.getLogger(ITestS3AFileOperationCost.class); - private OperationCostValidator costValidator; - /** * Parameterization. */ @@ -87,158 +69,11 @@ public static Collection params() { }); } - /** - * Parameter: should the stores be guarded? - */ - private final boolean s3guard; - - /** - * Parameter: should directory markers be retained? - */ - private final boolean keepMarkers; - - /** - * Is this an auth mode test run? - */ - private final boolean authoritative; - - /* probe states calculated from the configuration options. */ - boolean isGuarded; - boolean isRaw; - - boolean isAuthoritative; - boolean isNonAuth; - - boolean isKeeping; - - boolean isDeleting; - public ITestS3AFileOperationCost(final String name, final boolean s3guard, final boolean keepMarkers, final boolean authoritative) { - this.s3guard = s3guard; - this.keepMarkers = keepMarkers; - this.authoritative = authoritative; - } - - @Override - public Configuration createConfiguration() { - Configuration conf = super.createConfiguration(); - String bucketName = getTestBucketName(conf); - removeBucketOverrides(bucketName, conf, - S3_METADATA_STORE_IMPL); - if (!isGuarded()) { - // in a raw run remove all s3guard settings - removeBaseAndBucketOverrides(bucketName, conf, - S3_METADATA_STORE_IMPL); - } - // directory marker options - removeBaseAndBucketOverrides(bucketName, conf, - DIRECTORY_MARKER_POLICY, - METADATASTORE_AUTHORITATIVE, - AUTHORITATIVE_PATH); - conf.set(DIRECTORY_MARKER_POLICY, - keepMarkers - ? DIRECTORY_MARKER_POLICY_KEEP - : DIRECTORY_MARKER_POLICY_DELETE); - conf.setBoolean(METADATASTORE_AUTHORITATIVE, authoritative); - disableFilesystemCaching(conf); - return conf; - } - - @Override - public void setup() throws Exception { - super.setup(); - if (isGuarded()) { - // s3guard is required for those test runs where any of the - // guard options are set - assumeS3GuardState(true, getConfiguration()); - } - S3AFileSystem fs = getFileSystem(); - skipDuringFaultInjection(fs); - - // build up the states - isGuarded = isGuarded(); - - isRaw = !isGuarded; - isAuthoritative = isGuarded && authoritative; - isNonAuth = isGuarded && !authoritative; - - isKeeping = isKeepingMarkers (); - - isDeleting = !isKeeping; - - // insert new metrics so as to keep the list sorted - costValidator = OperationCostValidator.builder(getFileSystem()) - .withMetrics( - DIRECTORIES_CREATED, - DIRECTORIES_DELETED, - FAKE_DIRECTORIES_DELETED, - FILES_DELETED, - INVOCATION_COPY_FROM_LOCAL_FILE, - OBJECT_COPY_REQUESTS, - OBJECT_DELETE_REQUESTS, - OBJECT_LIST_REQUESTS, - OBJECT_METADATA_REQUESTS, - OBJECT_PUT_BYTES, - OBJECT_PUT_REQUESTS) - .build(); - } - - public void assumeUnguarded() { - assume("Unguarded FS only", !isGuarded()); - } - - /** - * Is the store guarded authoritatively on the test path? - * @return true if the condition is met on this test run. - */ - public boolean isAuthoritative() { - return authoritative; - } - - /** - * Is the store guarded? - * @return true if the condition is met on this test run. - */ - public boolean isGuarded() { - return s3guard; - } - - /** - * Is the store raw? - * @return true if the condition is met on this test run. - */ - public boolean isRaw() { - return isRaw; - } - - /** - * Is the store guarded non-authoritatively on the test path? - * @return true if the condition is met on this test run. - */ - public boolean isNonAuth() { - return isNonAuth; - } - - public boolean isDeleting() { - return isDeleting; - } - - public boolean isKeepingMarkers() { - return keepMarkers; - } - - - private final Object metricSummary = costValidator; - - /** - * A special object whose toString() value is the current - * state of the metrics. - */ - protected Object getMetricSummary() { - return metricSummary; + super(s3guard, keepMarkers, authoritative); } /** @@ -409,54 +244,6 @@ public void testCostOfGetFileStatusOnNonEmptyDir() throws Throwable { GETFILESTATUS_DIR_H, GETFILESTATUS_DIR_L); assertEmptyDirStatus(status, Tristate.FALSE); } - - /** - * This creates a directory with a child and then deletes it. - * The parent dir must be found and declared as empty. - */ - @Test - public void testDeleteFile() throws Throwable { - describe("performing getFileStatus on newly emptied directory"); - S3AFileSystem fs = getFileSystem(); - // creates the marker - Path dir = dir(methodPath()); - // file creation may have deleted that marker, but it may - // still be there - Path simpleFile = file(new Path(dir, "simple.txt")); - - boolean rawAndKeeping = isRaw && isKeeping; - boolean rawAndDeleting = isRaw && isDeleting; - verifyMetrics(() -> { - fs.delete(simpleFile, false); - return "after fs.delete(simpleFile) " + getMetricSummary(); - }, - // delete file. For keeping: that's it - probe(rawAndKeeping, OBJECT_METADATA_REQUESTS, - FILESTATUS_FILE_PROBE_H), - // if deleting markers, look for the parent too - probe(rawAndDeleting, OBJECT_METADATA_REQUESTS, - FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H), - raw(OBJECT_LIST_REQUESTS, - FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L), - always(DIRECTORIES_DELETED, 0), - always(FILES_DELETED, 1), - - // keeping: create no parent dirs or delete parents - keeping(DIRECTORIES_CREATED, 0), - keeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), - - // deleting: create a parent and delete any of its parents - deleting(DIRECTORIES_CREATED, 1), - deleting(OBJECT_DELETE_REQUESTS, - DELETE_OBJECT_REQUEST - + DELETE_MARKER_REQUEST) - ); - // there is an empty dir for a parent - S3AFileStatus status = verifyRawInnerGetFileStatus(dir, true, - StatusProbeEnum.ALL, GETFILESTATUS_DIR_H, GETFILESTATUS_DIR_L); - assertEmptyDirStatus(status, Tristate.TRUE); - } - @Test public void testCostOfCopyFromLocalFile() throws Throwable { describe("testCostOfCopyFromLocalFile"); @@ -493,176 +280,6 @@ public void testCostOfCopyFromLocalFile() throws Throwable { } } - @Test - public void testDirMarkersSubdir() throws Throwable { - describe("verify cost of deep subdir creation"); - - Path subDir = new Path(methodPath(), "1/2/3/4/5/6"); - // one dir created, possibly a parent removed - verifyMetrics(() -> { - mkdirs(subDir); - return "after mkdir(subDir) " + getMetricSummary(); - }, - always(DIRECTORIES_CREATED, 1), - always(DIRECTORIES_DELETED, 0), - keeping(OBJECT_DELETE_REQUESTS, 0), - keeping(FAKE_DIRECTORIES_DELETED, 0), - deleting(OBJECT_DELETE_REQUESTS, DELETE_MARKER_REQUEST), - // delete all possible fake dirs above the subdirectory - deleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(subDir) - 1)); - } - - @Test - public void testDirMarkersFileCreation() throws Throwable { - describe("verify cost of file creation"); - - Path srcBaseDir = dir(methodPath()); - - Path srcDir = dir(new Path(srcBaseDir, "1/2/3/4/5/6")); - - // creating a file should trigger demise of the src dir marker - // unless markers are being kept - - verifyMetrics(() -> { - file(new Path(srcDir, "source.txt")); - return "after touch(fs, srcFilePath) " + getMetricSummary(); - }, - always(DIRECTORIES_CREATED, 0), - always(DIRECTORIES_DELETED, 0), - // keeping: no delete operations. - keeping(OBJECT_DELETE_REQUESTS, 0), - keeping(FAKE_DIRECTORIES_DELETED, 0), - // delete all possible fake dirs above the file - deleting(OBJECT_DELETE_REQUESTS, 1), - deleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(srcDir))); - } - - @Test - public void testRenameFileToDifferentDirectory() throws Throwable { - describe("rename a file to a different directory, " - + "keeping the source dir present"); - - Path baseDir = dir(methodPath()); - - Path srcDir = new Path(baseDir, "1/2/3/4/5/6"); - final Path srcFilePath = file(new Path(srcDir, "source.txt")); - - // create a new source file. - // Explicitly use a new path object to guarantee that the parent paths - // are different object instances and so equals() rather than == - // is - Path parent2 = srcFilePath.getParent(); - Path srcFile2 = file(new Path(parent2, "source2.txt")); - Assertions.assertThat(srcDir) - .isNotSameAs(parent2); - Assertions.assertThat(srcFilePath.getParent()) - .isEqualTo(srcFile2.getParent()); - - // create a directory tree, expect the dir to be created and - // possibly a request to delete all parent directories made. - Path destBaseDir = new Path(baseDir, "dest"); - Path destDir = dir(new Path(destBaseDir, "a/b/c/d")); - Path destFilePath = new Path(destDir, "dest.txt"); - - // rename the source file to the destination file. - // this tests file rename, not dir rename - // as srcFile2 exists, the parent dir of srcFilePath must not be created. - verifyMetrics(() -> - execRename(srcFilePath, destFilePath), - rawHeadList(RENAME_SINGLE_FILE_RENAME_H, - RENAME_SINGLE_FILE_RENAME_DIFFERENT_DIR_L), - always(DIRECTORIES_CREATED, 0), - always(DIRECTORIES_DELETED, 0), - // keeping: only the core delete operation is issued. - keeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), - keeping(FAKE_DIRECTORIES_DELETED, 0), - // deleting: delete any fake marker above the destination. - deleting(OBJECT_DELETE_REQUESTS, - DELETE_OBJECT_REQUEST + DELETE_MARKER_REQUEST), - deleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(destDir))); - - assertIsFile(destFilePath); - assertIsDirectory(srcDir); - assertPathDoesNotExist("should have gone in the rename", srcFilePath); - } - - /** - * Same directory rename is lower cost as there's no need to - * look for the parent dir of the dest path or worry about - * deleting markers. - */ - @Test - public void testRenameSameDirectory() throws Throwable { - describe("rename a file to a different directory, " - + "keeping the source dir present"); - - Path baseDir = dir(methodPath()); - final Path sourceFile = file(new Path(baseDir, "source.txt")); - - // create a new source file. - // Explicitly use a new path object to guarantee that the parent paths - // are different object instances and so equals() rather than == - // is - Path parent2 = sourceFile.getParent(); - Path destFile = new Path(parent2, "dest"); - verifyMetrics(() -> - execRename(sourceFile, destFile), - rawHeadList(RENAME_SINGLE_FILE_RENAME_H, - RENAME_SINGLE_FILE_RENAME_SAME_DIR_L), - always(OBJECT_COPY_REQUESTS, 1), - always(DIRECTORIES_CREATED, 0), - always(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), - always(FAKE_DIRECTORIES_DELETED, 0)); - } - - @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 = file(new Path("/src-" + uuid)); - Path dest = new Path("/dest-" + uuid); - try { - - verifyMetrics(() -> { - fs.rename(src, dest); - return "after fs.rename(/src,/dest) " + getMetricSummary(); - }, - // TWO HEAD for exists, one for source MD in copy - rawHeadList(RENAME_SINGLE_FILE_RENAME_H, - GETFILESTATUS_FNFE_L), - // here we expect there to be no fake directories - always(DIRECTORIES_CREATED, 0), - // one for the renamed file only - always(OBJECT_DELETE_REQUESTS, - DELETE_OBJECT_REQUEST), - // no directories are deleted: This is root - always(DIRECTORIES_DELETED, 0), - // no fake directories are deleted: This is root - always(FAKE_DIRECTORIES_DELETED, 0), - always(FILES_DELETED, 1)); - - // delete that destination file, assert only the file delete was issued - verifyMetrics(() -> { - fs.delete(dest, false); - return "after fs.delete(/dest) " + getMetricSummary(); - }, - always(DIRECTORIES_CREATED, 0), - always(DIRECTORIES_DELETED, 0), - always(FAKE_DIRECTORIES_DELETED, 0), - always(FILES_DELETED, 1), - always(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), - rawHeadList(FILESTATUS_FILE_PROBE_H, 0)); /* no need to look at parent. */ - - } finally { - fs.delete(src, false); - fs.delete(dest, false); - } - } - @Test public void testDirProbes() throws Throwable { describe("Test directory probe cost"); @@ -709,20 +326,6 @@ public void testDirProbes() throws Throwable { assertEmptyDirStatus(status, Tristate.TRUE); } - /** - * Assert the empty directory status of a file is as expected. - * @param status status to probe. - * @param expected expected value - */ - protected void assertEmptyDirStatus(final S3AFileStatus status, - final Tristate expected) { - Assertions.assertThat(status.isEmptyDirectory()) - .describedAs(dynamicDescription(() -> - "FileStatus says directory is not empty: " + status - + "\n" + ContractTestUtils.ls(getFileSystem(), status.getPath()))) - .isEqualTo(expected); - } - @Test public void testCreateCost() throws Throwable { describe("Test file creation cost -raw only"); @@ -831,422 +434,5 @@ public void testCostOfGlobStatusNoSymlinkResolution() throws Throwable { fs.globStatus(basePath.suffix("/*"))); } - /** - * Create then close the file through the builder API. - * @param path path - * @param overwrite overwrite flag - * @param recursive true == skip parent existence check - * @param head expected head count - * @param list expected list count - * @return path to new object. - */ - private Path buildFile(Path path, - boolean overwrite, - boolean recursive, - int head, - int list) throws Exception { - resetStatistics(); - verifyRawHeadList(head, list, () -> { - FSDataOutputStreamBuilder builder = getFileSystem().createFile(path) - .overwrite(overwrite); - if (recursive) { - builder.recursive(); - } - FSDataOutputStream stream = builder.build(); - stream.close(); - return stream.toString(); - }); - return path; - } - - /** - * Create a directory, returning its path. - * @param p path to dir. - * @return path of new dir - */ - private Path dir(Path p) throws IOException { - mkdirs(p); - return p; - } - - /** - * Create a file, returning its path. - * @param p path to file. - * @return path of new file - */ - private Path file(Path p) throws IOException { - return file(p, true); - } - - /** - * Create a file, returning its path. - * @param path path to file. - * @param overwrite overwrite flag - * @return path of new file - */ - private Path file(Path path, final boolean overwrite) - throws IOException { - getFileSystem().create(path, overwrite).close(); - return path; - } - - - /** - * Touch a file, overwriting. - * @param path path - * @return path to new object. - */ - private Path create(Path path) throws Exception { - return create(path, true, - CREATE_FILE_OVERWRITE_H, - CREATE_FILE_OVERWRITE_L); - } - - /** - * Create then close the file. - * @param path path - * @param overwrite overwrite flag - * @param head expected head count - * @param list expected list count - * @return path to new object. - */ - private Path create(Path path, boolean overwrite, - int head, int list) throws Exception { - return verifyRawHeadList(head, list, () -> - file(path, overwrite)); - } - - - /** - * Execute rename, returning the current metrics. - * For use in l-expressions. - * @param source source path. - * @param dest dest path - * @return a string for exceptions. - */ - public String execRename(final Path source, - final Path dest) throws IOException { - getFileSystem().rename(source, dest); - return String.format("rename(%s, %s): %s", dest, source, getMetricSummary()); - } - - /** - * How many directories are in a path? - * @param path path to probe. - * @return the number of entries below root this path is - */ - private int directoriesInPath(Path path) { - return path.isRoot() ? 0 : 1 + directoriesInPath(path.getParent()); - } - - /** - * Reset all the metrics being tracked. - */ - private void resetStatistics() { - costValidator.resetMetricDiffs(); - } - - /** - * Execute a closure and verify the metrics. - * @param eval closure to evaluate - * @param expected varargs list of expected diffs - * @param return type. - * @return the result of the evaluation - */ - private T verifyMetrics( - Callable eval, - OperationCostValidator.ExpectedProbe... expected) throws Exception { - return costValidator.exec(eval, expected); - - } - - /** - * Execute a closure, expecting an exception. - * Verify the metrics after the exception has been caught and - * validated. - * @param clazz type of exception - * @param text text to look for in exception (optional) - * @param eval closure to evaluate - * @param expected varargs list of expected diffs - * @param return type of closure - * @param exception type - * @return the exception caught. - * @throws Exception any other exception - */ - private E verifyMetricsIntercepting( - Class clazz, - String text, - Callable eval, - OperationCostValidator.ExpectedProbe... expected) throws Exception { - return costValidator.intercepting(clazz, text, eval, expected); - } - - /** - * Execute a closure expecting an exception. - * @param clazz type of exception - * @param text text to look for in exception (optional) - * @param head expected head request count. - * @param list expected list request count. - * @param eval closure to evaluate - * @param return type of closure - * @param exception type - * @return the exception caught. - * @throws Exception any other exception - */ - private E interceptRawHeadList( - Class clazz, - String text, - int head, - int list, - Callable eval) throws Exception { - return verifyMetricsIntercepting(clazz, text, eval, - rawHeadList(head, list)); - } - - /** - * Create the probes to expect a given set of head and list requests. - * @param enabled is the probe enabled? - * @param head expected HEAD count - * @param list expected LIST count - * @return a probe list - */ - private OperationCostValidator.ExpectedProbe - expectHeadList(boolean enabled, int head, int list) { - return probes(enabled, - probe(OBJECT_METADATA_REQUESTS, head), - probe(OBJECT_LIST_REQUESTS, list)); - } - /** - * Create the probes to expect a given set of head and list requests. - * @param head expected HEAD count - * @param list expected LIST count - * @return a probe list - */ - private OperationCostValidator.ExpectedProbe - alwaysHeadList(int head, int list) { - return expectHeadList(true, head, list); - } - - /** - * Declare the expected head and list requests on a raw FS. - * @param head expected HEAD count - * @param list expected LIST count - * @return a probe list - */ - private OperationCostValidator.ExpectedProbe - rawHeadList(int head, int list) { - return expectHeadList(isRaw(), head, list); - } - - /** - * Declare the expected head and list requests on an authoritative FS. - * @param head expected HEAD count - * @param list expected LIST count - * @return a probe list - */ - private OperationCostValidator.ExpectedProbe - authHeadList(int head, int list) { - return expectHeadList(isAuthoritative(), head, list); - } - - /** - * Declare the expected head and list requests on a - * non authoritative FS. - * @param head expected HEAD count - * @param list expected LIST count - * @return a probe list - */ - private OperationCostValidator.ExpectedProbe - nonauthHeadList(int head, int list) { - return expectHeadList(isNonAuth(), head, list); - } - - /** - * Execute a closure expecting a specific number of HEAD/LIST calls - * on raw S3 stores only. - * @param head expected head request count. - * @param list expected list request count. - * @param eval closure to evaluate - * @param return type of closure - * @return the result of the evaluation - */ - private T verifyRawHeadList( - int head, - int list, - Callable eval) throws Exception { - return verifyMetrics(eval, - rawHeadList(head, list)); - } - - /** - * Execute {@link S3AFileSystem#innerGetFileStatus(Path, boolean, Set)} - * for the given probes. - * expect the specific HEAD/LIST count. - *

- * Raw FS only. - *

- * @param path path - * @param needEmptyDirectoryFlag look for empty directory - * @param probes file status probes to perform - * @param head expected head calls - * @param list expected list calls - * @return the status - */ - public S3AFileStatus verifyRawInnerGetFileStatus( - Path path, - boolean needEmptyDirectoryFlag, - Set probes, - int head, - int list) throws Exception { - return verifyRawHeadList(head, list, () -> - getFileSystem().innerGetFileStatus( - path, - needEmptyDirectoryFlag, - probes)); - } - - /** - * Execute {@link S3AFileSystem#innerGetFileStatus(Path, boolean, Set)} - * for the given probes -expect a FileNotFoundException, - * and the specific HEAD/LIST count. - *

- * Raw FS only. - *

- * @param path path - * @param needEmptyDirectoryFlag look for empty directory - * @param probes file status probes to perform - * @param head expected head calls - * @param list expected list calls - * @return the status - */ - public void interceptRawGetFileStatusFNFE( - Path path, - boolean needEmptyDirectoryFlag, - Set probes, - int head, - int list) throws Exception { - interceptRawHeadList(FileNotFoundException.class, "", - head, list, () -> - getFileSystem().innerGetFileStatus( - path, - needEmptyDirectoryFlag, - probes)); - } - - /** - * Probe for a path being a directory. - * Metrics are only checked on unguarded stores. - * @param path path - * @param expected expected outcome - * @param head head count (unguarded) - * @param list listCount (unguarded) - */ - private void isDir(Path path, boolean expected, - int head, int list) throws Exception { - boolean b = verifyRawHeadList(head, list, () -> - getFileSystem().isDirectory(path)); - Assertions.assertThat(b) - .describedAs("isDirectory(%s)", path) - .isEqualTo(expected); - } - - /** - * Probe for a path being a file. - * Metrics are only checked on unguarded stores. - * @param path path - * @param expected expected outcome - * @param head head count (unguarded) - * @param list listCount (unguarded) - */ - private void isFile(Path path, boolean expected, - int head, int list) throws Exception { - boolean b = verifyRawHeadList(head, list, () -> - getFileSystem().isFile(path)); - Assertions.assertThat(b) - .describedAs("isFile(%s)", path) - .isEqualTo(expected); - } - - /** - * A metric diff which must always hold. - * @param Statistic metric source - * @param expected expected value. - * @return the diff. - */ - private OperationCostValidator.ExpectedProbe always( - final Statistic Statistic, final int expected) { - return probe(Statistic, expected); - } - - /** - * A metric diff which must hold when the fs is unguarded. - * @param Statistic metric source - * @param expected expected value. - * @return the diff. - */ - private OperationCostValidator.ExpectedProbe raw( - final Statistic Statistic, final int expected) { - return probe(isRaw(), Statistic, expected); - } - - /** - * A metric diff which must hold when the fs is guarded. - * @param Statistic metric source - * @param expected expected value. - * @return the diff. - */ - private OperationCostValidator.ExpectedProbe guarded( - final Statistic Statistic, - final int expected) { - return probe(isGuarded(), Statistic, expected); - } - - /** - * A metric diff which must hold when the fs is guarded + authoritative. - * @param Statistic metric source - * @param expected expected value. - * @return the diff. - */ - private OperationCostValidator.ExpectedProbe authoritative( - final Statistic Statistic, - final int expected) { - return probe(isAuthoritative(), Statistic, expected); - } - - /** - * A metric diff which must hold when the fs is guarded + authoritative. - * @param Statistic metric source - * @param expected expected value. - * @return the diff. - */ - private OperationCostValidator.ExpectedProbe nonauth( - final Statistic Statistic, - final int expected) { - return probe(isNonAuth(), Statistic, expected); - } - - /** - * A metric diff which must hold when the fs is keeping markers - * @param Statistic metric source - * @param expected expected value. - * @return the diff. - */ - private OperationCostValidator.ExpectedProbe keeping( - final Statistic Statistic, - final int expected) { - return probe(isKeepingMarkers(), Statistic, expected); - } - - /** - * A metric diff which must hold when the fs is keeping markers - * @param Statistic metric source - * @param expected expected value. - * @return the diff. - */ - private OperationCostValidator.ExpectedProbe deleting( - final Statistic Statistic, - final int expected) { - return probe(isDeleting(), Statistic, expected); - } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index e47c70ef42353..65821c3d5c600 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -1448,4 +1448,26 @@ public static Set getCurrentThreadNames() { .collect(Collectors.toCollection(TreeSet::new)); return threads; } + + /** + * Call the package-private {@code innerGetFileStatus()} method + * on the passed in FS. + * @param fs filesystem + * @param path path + * @param needEmptyDirectoryFlag look for empty directory + * @param probes file status probes to perform + * @return the status + * @throws IOException + */ + public static S3AFileStatus innerGetFileStatus( + S3AFileSystem fs, + Path path, + boolean needEmptyDirectoryFlag, + Set probes) throws IOException { + + return fs.innerGetFileStatus( + path, + needEmptyDirectoryFlag, + probes); + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/AbstractS3ACostTest.java new file mode 100644 index 0000000000000..e5f098e58480c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/AbstractS3ACostTest.java @@ -0,0 +1,640 @@ +/* + * 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.costs; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Set; +import java.util.concurrent.Callable; + +import org.assertj.core.api.Assertions; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSDataOutputStreamBuilder; +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.S3AFileStatus; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.Tristate; +import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; + +import static org.apache.hadoop.fs.s3a.Constants.*; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; +import static org.apache.hadoop.fs.s3a.Statistic.*; +import static org.apache.hadoop.fs.s3a.test.costs.HeadListCosts.*; +import static org.apache.hadoop.fs.s3a.test.costs.OperationCostValidator.probe; +import static org.apache.hadoop.fs.s3a.test.costs.OperationCostValidator.probes; +import static org.apache.hadoop.test.AssertExtensions.dynamicDescription; + +/** + * Abstract class for tests which make assertions about cost. + *

+ * Factored out from {@code ITestS3AFileOperationCost} + */ +public class AbstractS3ACostTest extends AbstractS3ATestBase { + + /** + * Parameter: should the stores be guarded? + */ + protected final boolean s3guard; + + /** + * Parameter: should directory markers be retained? + */ + protected final boolean keepMarkers; + + /** + * Is this an auth mode test run? + */ + protected final boolean authoritative; + + /** probe states calculated from the configuration options. */ + boolean isGuarded; + + boolean isRaw; + + boolean isAuthoritative; + + boolean isNonAuth; + + boolean isKeeping; + + boolean isDeleting; + + private OperationCostValidator costValidator; + + public AbstractS3ACostTest( + final boolean s3guard, + final boolean keepMarkers, + final boolean authoritative) { + this.s3guard = s3guard; + this.keepMarkers = keepMarkers; + this.authoritative = authoritative; + } + + @Override + public Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + String bucketName = getTestBucketName(conf); + removeBucketOverrides(bucketName, conf, + S3_METADATA_STORE_IMPL); + if (!isGuarded()) { + // in a raw run remove all s3guard settings + removeBaseAndBucketOverrides(bucketName, conf, + S3_METADATA_STORE_IMPL); + } + // directory marker options + removeBaseAndBucketOverrides(bucketName, conf, + DIRECTORY_MARKER_POLICY, + METADATASTORE_AUTHORITATIVE, + AUTHORITATIVE_PATH); + conf.set(DIRECTORY_MARKER_POLICY, + keepMarkers + ? DIRECTORY_MARKER_POLICY_KEEP + : DIRECTORY_MARKER_POLICY_DELETE); + conf.setBoolean(METADATASTORE_AUTHORITATIVE, authoritative); + disableFilesystemCaching(conf); + return conf; + } + + @Override + public void setup() throws Exception { + super.setup(); + if (isGuarded()) { + // s3guard is required for those test runs where any of the + // guard options are set + assumeS3GuardState(true, getConfiguration()); + } + S3AFileSystem fs = getFileSystem(); + skipDuringFaultInjection(fs); + + // build up the states + isGuarded = isGuarded(); + + isRaw = !isGuarded; + isAuthoritative = isGuarded && authoritative; + isNonAuth = isGuarded && !authoritative; + + isKeeping = isKeepingMarkers (); + + isDeleting = !isKeeping; + + // insert new metrics so as to keep the list sorted + costValidator = OperationCostValidator.builder(getFileSystem()) + .withMetrics( + DIRECTORIES_CREATED, + DIRECTORIES_DELETED, + FAKE_DIRECTORIES_DELETED, + FILES_DELETED, + INVOCATION_COPY_FROM_LOCAL_FILE, + OBJECT_COPY_REQUESTS, + OBJECT_DELETE_REQUESTS, + OBJECT_LIST_REQUESTS, + OBJECT_METADATA_REQUESTS, + OBJECT_PUT_BYTES, + OBJECT_PUT_REQUESTS) + .build(); + } + + public void assumeUnguarded() { + assume("Unguarded FS only", !isGuarded()); + } + + /** + * Is the store guarded authoritatively on the test path? + * @return true if the condition is met on this test run. + */ + public boolean isAuthoritative() { + return authoritative; + } + + /** + * Is the store guarded? + * @return true if the condition is met on this test run. + */ + public boolean isGuarded() { + return s3guard; + } + + /** + * Is the store raw? + * @return true if the condition is met on this test run. + */ + public boolean isRaw() { + return isRaw; + } + + /** + * Is the store guarded non-authoritatively on the test path? + * @return true if the condition is met on this test run. + */ + public boolean isNonAuth() { + return isNonAuth; + } + + public boolean isDeleting() { + return isDeleting; + } + + public boolean isKeepingMarkers() { + return keepMarkers; + } + + /** + * A special object whose toString() value is the current + * state of the metrics. + */ + protected Object getMetricSummary() { + return costValidator; + } + + /** + * Create then close the file through the builder API. + * @param path path + * @param overwrite overwrite flag + * @param recursive true == skip parent existence check + * @param head expected head count + * @param list expected list count + * @return path to new object. + */ + protected Path buildFile(Path path, + boolean overwrite, + boolean recursive, + int head, + int list) throws Exception { + resetStatistics(); + verifyRawHeadList(head, list, () -> { + FSDataOutputStreamBuilder builder = getFileSystem().createFile(path) + .overwrite(overwrite); + if (recursive) { + builder.recursive(); + } + FSDataOutputStream stream = builder.build(); + stream.close(); + return stream.toString(); + }); + return path; + } + + /** + * Create a directory, returning its path. + * @param p path to dir. + * @return path of new dir + */ + protected Path dir(Path p) throws IOException { + mkdirs(p); + return p; + } + + /** + * Create a file, returning its path. + * @param p path to file. + * @return path of new file + */ + protected Path file(Path p) throws IOException { + return file(p, true); + } + + /** + * Create a file, returning its path. + * @param path path to file. + * @param overwrite overwrite flag + * @return path of new file + */ + protected Path file(Path path, final boolean overwrite) + throws IOException { + getFileSystem().create(path, overwrite).close(); + return path; + } + + /** + * Touch a file, overwriting. + * @param path path + * @return path to new object. + */ + protected Path create(Path path) throws Exception { + return create(path, true, + CREATE_FILE_OVERWRITE_H, + CREATE_FILE_OVERWRITE_L); + } + + /** + * Create then close the file. + * @param path path + * @param overwrite overwrite flag + * @param head expected head count + * @param list expected list count + * @return path to new object. + */ + protected Path create(Path path, boolean overwrite, + int head, int list) throws Exception { + return verifyRawHeadList(head, list, () -> + file(path, overwrite)); + } + + /** + * Execute rename, returning the current metrics. + * For use in l-expressions. + * @param source source path. + * @param dest dest path + * @return a string for exceptions. + */ + public String execRename(final Path source, + final Path dest) throws IOException { + getFileSystem().rename(source, dest); + return String.format("rename(%s, %s): %s", dest, source, getMetricSummary()); + } + + /** + * How many directories are in a path? + * @param path path to probe. + * @return the number of entries below root this path is + */ + protected int directoriesInPath(Path path) { + return path.isRoot() ? 0 : 1 + directoriesInPath(path.getParent()); + } + + /** + * Reset all the metrics being tracked. + */ + private void resetStatistics() { + costValidator.resetMetricDiffs(); + } + + /** + * Execute a closure and verify the metrics. + * @param eval closure to evaluate + * @param expected varargs list of expected diffs + * @param return type. + * @return the result of the evaluation + */ + protected T verifyMetrics( + Callable eval, + OperationCostValidator.ExpectedProbe... expected) throws Exception { + return costValidator.exec(eval, expected); + + } + + /** + * Execute a closure, expecting an exception. + * Verify the metrics after the exception has been caught and + * validated. + * @param clazz type of exception + * @param text text to look for in exception (optional) + * @param eval closure to evaluate + * @param expected varargs list of expected diffs + * @param return type of closure + * @param exception type + * @return the exception caught. + * @throws Exception any other exception + */ + protected E verifyMetricsIntercepting( + Class clazz, + String text, + Callable eval, + OperationCostValidator.ExpectedProbe... expected) throws Exception { + return costValidator.intercepting(clazz, text, eval, expected); + } + + /** + * Execute a closure expecting an exception. + * @param clazz type of exception + * @param text text to look for in exception (optional) + * @param head expected head request count. + * @param list expected list request count. + * @param eval closure to evaluate + * @param return type of closure + * @param exception type + * @return the exception caught. + * @throws Exception any other exception + */ + protected E interceptRawHeadList( + Class clazz, + String text, + int head, + int list, + Callable eval) throws Exception { + return verifyMetricsIntercepting(clazz, text, eval, + rawHeadList(head, list)); + } + + /** + * Create the probes to expect a given set of head and list requests. + * @param enabled is the probe enabled? + * @param head expected HEAD count + * @param list expected LIST count + * @return a probe list + */ + private OperationCostValidator.ExpectedProbe + expectHeadList(boolean enabled, int head, int list) { + return probes(enabled, + probe(OBJECT_METADATA_REQUESTS, head), + probe(OBJECT_LIST_REQUESTS, list)); + } + + /** + * Create the probes to expect a given set of head and list requests. + * @param head expected HEAD count + * @param list expected LIST count + * @return a probe list + */ + private OperationCostValidator.ExpectedProbe + alwaysHeadList(int head, int list) { + return expectHeadList(true, head, list); + } + + /** + * Declare the expected head and list requests on a raw FS. + * @param head expected HEAD count + * @param list expected LIST count + * @return a probe list + */ + protected OperationCostValidator.ExpectedProbe + rawHeadList(int head, int list) { + return expectHeadList(isRaw(), head, list); + } + + /** + * Declare the expected head and list requests on an authoritative FS. + * @param head expected HEAD count + * @param list expected LIST count + * @return a probe list + */ + protected OperationCostValidator.ExpectedProbe + authHeadList(int head, int list) { + return expectHeadList(isAuthoritative(), head, list); + } + + /** + * Declare the expected head and list requests on a + * non authoritative FS. + * @param head expected HEAD count + * @param list expected LIST count + * @return a probe list + */ + protected OperationCostValidator.ExpectedProbe + nonauthHeadList(int head, int list) { + return expectHeadList(isNonAuth(), head, list); + } + + /** + * Execute a closure expecting a specific number of HEAD/LIST calls + * on raw S3 stores only. + * @param head expected head request count. + * @param list expected list request count. + * @param eval closure to evaluate + * @param return type of closure + * @return the result of the evaluation + */ + protected T verifyRawHeadList( + int head, + int list, + Callable eval) throws Exception { + return verifyMetrics(eval, + rawHeadList(head, list)); + } + + /** + * Execute {@link S3AFileSystem#innerGetFileStatus(Path, boolean, Set)} + * for the given probes. + * expect the specific HEAD/LIST count. + *

+ * Raw FS only. + *

+ * @param path path + * @param needEmptyDirectoryFlag look for empty directory + * @param probes file status probes to perform + * @param head expected head calls + * @param list expected list calls + * @return the status + */ + public S3AFileStatus verifyRawInnerGetFileStatus( + Path path, + boolean needEmptyDirectoryFlag, + Set probes, + int head, + int list) throws Exception { + return verifyRawHeadList(head, list, () -> + innerGetFileStatus(getFileSystem(), + path, + needEmptyDirectoryFlag, + probes)); + } + + /** + * Execute {@link S3AFileSystem#innerGetFileStatus(Path, boolean, Set)} + * for the given probes -expect a FileNotFoundException, + * and the specific HEAD/LIST count. + *

+ * Raw FS only. + *

+ * @param path path + * @param needEmptyDirectoryFlag look for empty directory + * @param probes file status probes to perform + * @param head expected head calls + * @param list expected list calls + * @return the status + */ + public void interceptRawGetFileStatusFNFE( + Path path, + boolean needEmptyDirectoryFlag, + Set probes, + int head, + int list) throws Exception { + interceptRawHeadList(FileNotFoundException.class, "", + head, list, () -> + innerGetFileStatus(getFileSystem(), + path, + needEmptyDirectoryFlag, + probes)); + } + + /** + * Probe for a path being a directory. + * Metrics are only checked on unguarded stores. + * @param path path + * @param expected expected outcome + * @param head head count (unguarded) + * @param list listCount (unguarded) + */ + protected void isDir(Path path, boolean expected, + int head, int list) throws Exception { + boolean b = verifyRawHeadList(head, list, () -> + getFileSystem().isDirectory(path)); + Assertions.assertThat(b) + .describedAs("isDirectory(%s)", path) + .isEqualTo(expected); + } + + /** + * Probe for a path being a file. + * Metrics are only checked on unguarded stores. + * @param path path + * @param expected expected outcome + * @param head head count (unguarded) + * @param list listCount (unguarded) + */ + protected void isFile(Path path, boolean expected, + int head, int list) throws Exception { + boolean b = verifyRawHeadList(head, list, () -> + getFileSystem().isFile(path)); + Assertions.assertThat(b) + .describedAs("isFile(%s)", path) + .isEqualTo(expected); + } + + /** + * A metric diff which must always hold. + * @param Statistic metric source + * @param expected expected value. + * @return the diff. + */ + protected OperationCostValidator.ExpectedProbe always( + final Statistic Statistic, final int expected) { + return probe(Statistic, expected); + } + + /** + * A metric diff which must hold when the fs is unguarded. + * @param Statistic metric source + * @param expected expected value. + * @return the diff. + */ + protected OperationCostValidator.ExpectedProbe raw( + final Statistic Statistic, final int expected) { + return probe(isRaw(), Statistic, expected); + } + + /** + * A metric diff which must hold when the fs is guarded. + * @param Statistic metric source + * @param expected expected value. + * @return the diff. + */ + protected OperationCostValidator.ExpectedProbe guarded( + final Statistic Statistic, + final int expected) { + return probe(isGuarded(), Statistic, expected); + } + + /** + * A metric diff which must hold when the fs is guarded + authoritative. + * @param Statistic metric source + * @param expected expected value. + * @return the diff. + */ + protected OperationCostValidator.ExpectedProbe authoritative( + final Statistic Statistic, + final int expected) { + return probe(isAuthoritative(), Statistic, expected); + } + + /** + * A metric diff which must hold when the fs is guarded + authoritative. + * @param Statistic metric source + * @param expected expected value. + * @return the diff. + */ + protected OperationCostValidator.ExpectedProbe nonauth( + final Statistic Statistic, + final int expected) { + return probe(isNonAuth(), Statistic, expected); + } + + /** + * A metric diff which must hold when the fs is keeping markers + * @param Statistic metric source + * @param expected expected value. + * @return the diff. + */ + protected OperationCostValidator.ExpectedProbe keeping( + final Statistic Statistic, + final int expected) { + return probe(isKeepingMarkers(), Statistic, expected); + } + + /** + * A metric diff which must hold when the fs is keeping markers + * @param Statistic metric source + * @param expected expected value. + * @return the diff. + */ + protected OperationCostValidator.ExpectedProbe deleting( + final Statistic Statistic, + final int expected) { + return probe(isDeleting(), Statistic, expected); + } + + /** + * Assert the empty directory status of a file is as expected. + * @param status status to probe. + * @param expected expected value + */ + protected void assertEmptyDirStatus(final S3AFileStatus status, + final Tristate expected) { + Assertions.assertThat(status.isEmptyDirectory()) + .describedAs(dynamicDescription(() -> + "FileStatus says directory is not empty: " + status + + "\n" + ContractTestUtils.ls(getFileSystem(), status.getPath()))) + .isEqualTo(expected); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/ITestS3ADeleteCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/ITestS3ADeleteCost.java new file mode 100644 index 0000000000000..6303047cc904e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/ITestS3ADeleteCost.java @@ -0,0 +1,164 @@ +/* + * 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.costs; + + +import java.util.Arrays; +import java.util.Collection; + +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.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.Tristate; +import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; + +import static org.apache.hadoop.fs.s3a.Statistic.*; +import static org.apache.hadoop.fs.s3a.test.costs.HeadListCosts.*; +import static org.apache.hadoop.fs.s3a.test.costs.OperationCostValidator.probe; + + +/** + * Use metrics to assert about the cost of file API calls. + * Parameterized on guarded vs raw. and directory marker keep vs delete + */ +@RunWith(Parameterized.class) +public class ITestS3ADeleteCost extends AbstractS3ACostTest { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestS3ADeleteCost.class); + + /** + * Parameterization. + */ + @Parameterized.Parameters(name = "{0}") + public static Collection params() { + return Arrays.asList(new Object[][]{ + {"raw-keep-markers", false, true, false}, + {"raw-delete-markers", false, false, false}, + {"nonauth-keep-markers", true, true, false}, + {"auth-delete-markers", true, false, true} + }); + } + + public ITestS3ADeleteCost(final String name, + final boolean s3guard, + final boolean keepMarkers, + final boolean authoritative) { + super(s3guard, keepMarkers, authoritative); + } + + + /** + * This creates a directory with a child and then deletes it. + * The parent dir must be found and declared as empty. + */ + @Test + public void testDeleteFile() throws Throwable { + describe("performing getFileStatus on newly emptied directory"); + S3AFileSystem fs = getFileSystem(); + // creates the marker + Path dir = dir(methodPath()); + // file creation may have deleted that marker, but it may + // still be there + Path simpleFile = file(new Path(dir, "simple.txt")); + + boolean rawAndKeeping = isRaw() && isDeleting(); + boolean rawAndDeleting = isRaw() && isDeleting(); + verifyMetrics(() -> { + fs.delete(simpleFile, false); + return "after fs.delete(simpleFile) " + getMetricSummary(); + }, + // delete file. For keeping: that's it + probe(rawAndKeeping, OBJECT_METADATA_REQUESTS, + FILESTATUS_FILE_PROBE_H), + // if deleting markers, look for the parent too + probe(rawAndDeleting, OBJECT_METADATA_REQUESTS, + FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H), + raw(OBJECT_LIST_REQUESTS, + FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L), + always(DIRECTORIES_DELETED, 0), + always(FILES_DELETED, 1), + + // keeping: create no parent dirs or delete parents + keeping(DIRECTORIES_CREATED, 0), + keeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + + // deleting: create a parent and delete any of its parents + deleting(DIRECTORIES_CREATED, 1), + deleting(OBJECT_DELETE_REQUESTS, + DELETE_OBJECT_REQUEST + + DELETE_MARKER_REQUEST) + ); + // there is an empty dir for a parent + S3AFileStatus status = verifyRawInnerGetFileStatus(dir, true, + StatusProbeEnum.ALL, GETFILESTATUS_DIR_H, GETFILESTATUS_DIR_L); + assertEmptyDirStatus(status, Tristate.TRUE); + } + + @Test + public void testDirMarkersSubdir() throws Throwable { + describe("verify cost of deep subdir creation"); + + Path subDir = new Path(methodPath(), "1/2/3/4/5/6"); + // one dir created, possibly a parent removed + verifyMetrics(() -> { + mkdirs(subDir); + return "after mkdir(subDir) " + getMetricSummary(); + }, + always(DIRECTORIES_CREATED, 1), + always(DIRECTORIES_DELETED, 0), + keeping(OBJECT_DELETE_REQUESTS, 0), + keeping(FAKE_DIRECTORIES_DELETED, 0), + deleting(OBJECT_DELETE_REQUESTS, DELETE_MARKER_REQUEST), + // delete all possible fake dirs above the subdirectory + deleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(subDir) - 1)); + } + + @Test + public void testDirMarkersFileCreation() throws Throwable { + describe("verify cost of file creation"); + + Path srcBaseDir = dir(methodPath()); + + Path srcDir = dir(new Path(srcBaseDir, "1/2/3/4/5/6")); + + // creating a file should trigger demise of the src dir marker + // unless markers are being kept + + verifyMetrics(() -> { + file(new Path(srcDir, "source.txt")); + return "after touch(fs, srcFilePath) " + getMetricSummary(); + }, + always(DIRECTORIES_CREATED, 0), + always(DIRECTORIES_DELETED, 0), + // keeping: no delete operations. + keeping(OBJECT_DELETE_REQUESTS, 0), + keeping(FAKE_DIRECTORIES_DELETED, 0), + // delete all possible fake dirs above the file + deleting(OBJECT_DELETE_REQUESTS, 1), + deleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(srcDir))); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/ITestS3ARenameCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/ITestS3ARenameCost.java new file mode 100644 index 0000000000000..209bf0741f028 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/ITestS3ARenameCost.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.test.costs; + + +import java.util.Arrays; +import java.util.Collection; +import java.util.UUID; + +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.fs.Path; +import org.apache.hadoop.fs.s3a.S3AFileSystem; + +import static org.apache.hadoop.fs.s3a.Statistic.*; +import static org.apache.hadoop.fs.s3a.test.costs.HeadListCosts.*; + +/** + * Use metrics to assert about the cost of file API calls. + * Parameterized on guarded vs raw. and directory marker keep vs delete + */ +@RunWith(Parameterized.class) +public class ITestS3ARenameCost extends AbstractS3ACostTest { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestS3ARenameCost.class); + + /** + * Parameterization. + */ + @Parameterized.Parameters(name = "{0}") + public static Collection params() { + return Arrays.asList(new Object[][]{ + {"raw-keep-markers", false, true, false}, + {"raw-delete-markers", false, false, false}, + {"nonauth-keep-markers", true, true, false}, + {"auth-delete-markers", true, false, true} + }); + } + + public ITestS3ARenameCost(final String name, + final boolean s3guard, + final boolean keepMarkers, + final boolean authoritative) { + super(s3guard, keepMarkers, authoritative); + } + + @Test + public void testRenameFileToDifferentDirectory() throws Throwable { + describe("rename a file to a different directory, " + + "keeping the source dir present"); + + Path baseDir = dir(methodPath()); + + Path srcDir = new Path(baseDir, "1/2/3/4/5/6"); + final Path srcFilePath = file(new Path(srcDir, "source.txt")); + + // create a new source file. + // Explicitly use a new path object to guarantee that the parent paths + // are different object instances and so equals() rather than == + // is + Path parent2 = srcFilePath.getParent(); + Path srcFile2 = file(new Path(parent2, "source2.txt")); + Assertions.assertThat(srcDir) + .isNotSameAs(parent2); + Assertions.assertThat(srcFilePath.getParent()) + .isEqualTo(srcFile2.getParent()); + + // create a directory tree, expect the dir to be created and + // possibly a request to delete all parent directories made. + Path destBaseDir = new Path(baseDir, "dest"); + Path destDir = dir(new Path(destBaseDir, "a/b/c/d")); + Path destFilePath = new Path(destDir, "dest.txt"); + + // rename the source file to the destination file. + // this tests file rename, not dir rename + // as srcFile2 exists, the parent dir of srcFilePath must not be created. + verifyMetrics(() -> + execRename(srcFilePath, destFilePath), + rawHeadList(RENAME_SINGLE_FILE_RENAME_H, + RENAME_SINGLE_FILE_RENAME_DIFFERENT_DIR_L), + always(DIRECTORIES_CREATED, 0), + always(DIRECTORIES_DELETED, 0), + // keeping: only the core delete operation is issued. + keeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + keeping(FAKE_DIRECTORIES_DELETED, 0), + // deleting: delete any fake marker above the destination. + deleting(OBJECT_DELETE_REQUESTS, + DELETE_OBJECT_REQUEST + DELETE_MARKER_REQUEST), + deleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(destDir))); + + assertIsFile(destFilePath); + assertIsDirectory(srcDir); + assertPathDoesNotExist("should have gone in the rename", srcFilePath); + } + + /** + * Same directory rename is lower cost as there's no need to + * look for the parent dir of the dest path or worry about + * deleting markers. + */ + @Test + public void testRenameSameDirectory() throws Throwable { + describe("rename a file to a different directory, " + + "keeping the source dir present"); + + Path baseDir = dir(methodPath()); + final Path sourceFile = file(new Path(baseDir, "source.txt")); + + // create a new source file. + // Explicitly use a new path object to guarantee that the parent paths + // are different object instances and so equals() rather than == + // is + Path parent2 = sourceFile.getParent(); + Path destFile = new Path(parent2, "dest"); + verifyMetrics(() -> + execRename(sourceFile, destFile), + rawHeadList(RENAME_SINGLE_FILE_RENAME_H, + RENAME_SINGLE_FILE_RENAME_SAME_DIR_L), + always(OBJECT_COPY_REQUESTS, 1), + always(DIRECTORIES_CREATED, 0), + always(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + always(FAKE_DIRECTORIES_DELETED, 0)); + } + + @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 = file(new Path("/src-" + uuid)); + Path dest = new Path("/dest-" + uuid); + try { + + verifyMetrics(() -> { + fs.rename(src, dest); + return "after fs.rename(/src,/dest) " + getMetricSummary(); + }, + // TWO HEAD for exists, one for source MD in copy + rawHeadList(RENAME_SINGLE_FILE_RENAME_H, + GETFILESTATUS_FNFE_L), + // here we expect there to be no fake directories + always(DIRECTORIES_CREATED, 0), + // one for the renamed file only + always(OBJECT_DELETE_REQUESTS, + DELETE_OBJECT_REQUEST), + // no directories are deleted: This is root + always(DIRECTORIES_DELETED, 0), + // no fake directories are deleted: This is root + always(FAKE_DIRECTORIES_DELETED, 0), + always(FILES_DELETED, 1)); + + // delete that destination file, assert only the file delete was issued + verifyMetrics(() -> { + fs.delete(dest, false); + return "after fs.delete(/dest) " + getMetricSummary(); + }, + always(DIRECTORIES_CREATED, 0), + always(DIRECTORIES_DELETED, 0), + always(FAKE_DIRECTORIES_DELETED, 0), + always(FILES_DELETED, 1), + always(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + rawHeadList(FILESTATUS_FILE_PROBE_H, + 0)); /* no need to look at parent. */ + + } finally { + fs.delete(src, false); + fs.delete(dest, false); + } + } + + +} From c0320ef0bf9a8f31b9f766aa38f37364c4646272 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 22 Jul 2020 14:47:05 +0100 Subject: [PATCH 05/20] HADOOP-13230 Marker Tool getting into a testable/tested state Change-Id: I241202cefeb63ed26a19703b97a54bfe2a024912 --- .../hadoop/fs/s3a/impl/DirMarkerTracker.java | 10 + .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 8 + .../hadoop/fs/s3a/tools/MarkerTool.java | 294 +++++++++++++++--- .../hadoop/fs/s3a/tools/package-info.java | 2 +- .../s3a/test/costs/AbstractS3ACostTest.java | 2 +- .../hadoop/fs/s3a/tools/ITestMarkerTool.java | 284 +++++++++++++++++ 6 files changed, 556 insertions(+), 44 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java index af785eb9b268e..4d6807d2e76ae 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java @@ -151,4 +151,14 @@ public Path getLastDirChecked() { public int getScanCount() { return scanCount; } + + @Override + public String toString() { + return "DirMarkerTracker{" + + "leafMarkers=" + leafMarkers.size() + + ", surplusMarkers=" + surplusMarkers.size() + + ", lastDirChecked=" + lastDirChecked + + ", scanCount=" + scanCount + + '}'; + } } 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 9a906439a7449..44c9ac81c77c7 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 @@ -475,6 +475,14 @@ protected void setStore(MetadataStore store) { this.store = store; } + /** + * Reset the store and filesystem bindings. + */ + protected void resetBindings() { + store = null; + filesystem = null; + } + protected CommandFormat getCommandFormat() { return commandFormat; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index b8a4b5c47ece4..30fcfe87b8b60 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -20,10 +20,17 @@ import java.io.IOException; import java.io.PrintStream; +import java.util.ArrayList; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import com.amazonaws.AmazonClientException; +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; @@ -33,6 +40,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.StorageStatistics; +import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; @@ -43,6 +51,7 @@ import org.apache.hadoop.fs.shell.CommandFormat; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.ExitUtil; +import org.apache.hadoop.util.OperationDuration; import static org.apache.hadoop.fs.s3a.Invoker.once; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE; @@ -71,7 +80,7 @@ public final class MarkerTool extends S3GuardTool { "view and manipulate S3 directory markers"; private static final String USAGE = NAME - + " [OPTIONS]" + + " [-verbose] [-expected ]" + " (audit || report || clean)" // + " [-out ]" + " [-" + VERBOSE + "]" @@ -80,11 +89,11 @@ public final class MarkerTool extends S3GuardTool { public static final String OPT_EXPECTED = "expected"; - public static final String OPT_AUDIT = "audit"; + public static final String AUDIT = "audit"; - public static final String OPT_CLEAN = "clean"; + public static final String CLEAN = "clean"; - public static final String OPT_REPORT = "report"; + public static final String REPORT = "report"; public static final String OPT_OUTPUT = "output"; @@ -92,10 +101,23 @@ public final class MarkerTool extends S3GuardTool { static final String TOO_FEW_ARGUMENTS = "Too few arguments"; - private PrintStream out; + /** Will be overridden in run(), but during tests needs to avoid NPEs */ + private PrintStream out = System.out; + + private boolean verbose; + + private boolean purge; + + private int expected; + + private OperationCallbacks operationCallbacks; + + private StoreContext storeContext; public MarkerTool(final Configuration conf) { - super(conf, OPT_VERBOSE); + super(conf, + OPT_VERBOSE + ); getCommandFormat().addOptionWithValue(OPT_EXPECTED); // getCommandFormat().addOptionWithValue(OPT_OUTPUT); } @@ -110,6 +132,18 @@ public String getName() { return NAME; } + @Override + public void resetBindings() { + super.resetBindings(); + storeContext = null; + operationCallbacks = null; + } + + @Override + public void close() throws IOException { + super.close(); + } + @Override public int run(final String[] args, final PrintStream stream) throws ExitUtil.ExitException, Exception { @@ -127,21 +161,21 @@ public int run(final String[] args, final PrintStream stream) } // read arguments CommandFormat commandFormat = getCommandFormat(); - boolean verbose = commandFormat.getOpt(VERBOSE); + verbose = commandFormat.getOpt(VERBOSE); - boolean purge = false; - int expected = 0; + expected = 0; + // argument 0 is the action String action = parsedArgs.get(0); switch (action) { - case OPT_AUDIT: + case AUDIT: purge = false; expected = 0; break; - case OPT_CLEAN: + case CLEAN: purge = true; expected = -1; break; - case OPT_REPORT: + case REPORT: purge = false; expected = -1; break; @@ -152,33 +186,83 @@ public int run(final String[] args, final PrintStream stream) final String file = parsedArgs.get(1); final Path path = new Path(file); - S3AFileSystem fs = bindFilesystem(path.getFileSystem(getConf())); - final StoreContext context = fs.createStoreContext(); - final OperationCallbacks operations = fs.getOperationCallbacks(); - + ScanResult result = execute(path.getFileSystem(getConf()), path, purge, expected); + return result.exitCode; + } - boolean finalPurge = purge; - int finalExpected = expected; - int result = once("action", path.toString(), - () -> scan(path, finalPurge, finalExpected, verbose, context, - operations)); + /** + * Execute the scan/purge. + * @param sourceFS source FS; must be or wrap an S3A FS. + * @param path path to scan. + * @param doPurge purge? + * @param expectedMarkerCount expected marker count + * @return scan+purge result. + * @throws IOException failure + */ + @VisibleForTesting + ScanResult execute( + final FileSystem sourceFS, + final Path path, + final boolean doPurge, + final int expectedMarkerCount) + throws IOException { + S3AFileSystem fs = bindFilesystem(sourceFS); + storeContext = fs.createStoreContext(); + operationCallbacks = fs.getOperationCallbacks(); + + ScanResult result = once("action", path.toString(), + () -> scan(path, doPurge, expectedMarkerCount)); if (verbose) { dumpFileSystemStatistics(fs); } return result; } - private int scan(final Path path, - final boolean purge, - final int expected, - final boolean verbose, - final StoreContext context, - final OperationCallbacks operations) + /** + * Result of the scan operation. + */ + static final class ScanResult { + + /** Exit code to report. */ + int exitCode; + + /** Tracker which did the scan. */ + DirMarkerTracker tracker; + + /** Summary of purge. Null if none took place. */ + MarkerPurgeSummary purgeSummary; + + @Override + public String toString() { + return "ScanResult{" + + "exitCode=" + exitCode + + ", tracker=" + tracker + + ", purgeSummary=" + purgeSummary + + '}'; + } + } + + /** + * Do the scan. + * @param path path to scan. + * @param doPurge purge? + * @param expectedMarkerCount expected marker count + * @return scan+purge result. + * @throws IOException + * @throws ExitUtil.ExitException + */ + private ScanResult scan( + final Path path, + final boolean doPurge, + final int expectedMarkerCount) throws IOException, ExitUtil.ExitException { + ScanResult result = new ScanResult(); + DirMarkerTracker tracker = new DirMarkerTracker(); + result.tracker = tracker; try (DurationInfo ignored = new DurationInfo(LOG, "marker scan %s", path)) { - scanDirectoryTree(path, expected, context, operations, tracker); + scanDirectoryTree(path, tracker); } // scan done. what have we got? Map> surplusMarkers @@ -198,36 +282,66 @@ private int scan(final Path path, println(out, " %s", markers); } - if (size > expected) { - // failure - println(out, "Expected %d marker%s", expected, suffix(size)); - return EXIT_NOT_ACCEPTABLE; - } + } + if (verbose && !leafMarkers.isEmpty()) { + println(out, "Found %d empty directory 'leaf' marker%s under %s", + leafMarkers.size(), + suffix(size), + path); + println(out, "These are required to indicate empty directories"); + } + if (size > expectedMarkerCount) { + // failure + println(out, "Expected %d marker%s", expectedMarkerCount, suffix(size)); + result.exitCode = EXIT_NOT_ACCEPTABLE; + return result; + } + if (doPurge) { + int deletePageSize = storeContext.getConfiguration() + .getInt(Constants.BULK_DELETE_PAGE_SIZE, + Constants.BULK_DELETE_PAGE_SIZE_DEFAULT); + result.purgeSummary = purgeMarkers(tracker, + deletePageSize); } - return EXIT_SUCCESS; + result.exitCode = EXIT_SUCCESS; + return result; } + /** + * Suffix for plurals. + * @param size size to generate a suffix for + * @return "" or "s", depending on size + */ private String suffix(final int size) { return size == 1 ? "" : "s"; } + public static Logger getLOG() { + return LOG; + } + + /** + * Scan a directory tree + * @param path path to scan + * @param tracker tracker to update + * @throws IOException + */ private void scanDirectoryTree(final Path path, - final int expected, - final StoreContext context, - final OperationCallbacks operations, final DirMarkerTracker tracker) throws IOException { - RemoteIterator listing = operations - .listObjects(path, - context.pathToKey(path)); + RemoteIterator listing = operationCallbacks + .listObjects(path, storeContext.pathToKey(path)); while (listing.hasNext()) { S3AFileStatus status = listing.next(); Path p = status.getPath(); S3ALocatedFileStatus lfs = new S3ALocatedFileStatus( status, null); - String key = context.pathToKey(p); + String key = storeContext.pathToKey(p); if (status.isDirectory()) { - LOG.info("{}", key); + if (verbose) { + println(out, "Directory Marker %s", key); + } + LOG.debug("{}", key); tracker.markerFound(p, key + "/", lfs); @@ -236,11 +350,99 @@ private void scanDirectoryTree(final Path path, key, lfs); } + } + } + + /** + * Result of a call of {@link #purgeMarkers(DirMarkerTracker, int)}; + * included in {@link ScanResult} so must share visibility. + */ + static final class MarkerPurgeSummary { + + /** Number of markers deleted. */ + private int markersDeleted; + + /** Number of delete requests issued. */ + private int deleteRequests; + + /** + * Total duration of delete requests. + * If this is ever parallelized, this will + * be greater than the elapsed time of the + * operation. + */ + private long totalDeleteRequestDuration; + + @Override + public String toString() { + return "MarkerPurgeSummary{" + + "markersDeleted=" + markersDeleted + + ", deleteRequests=" + deleteRequests + + ", totalDeleteRequestDuration=" + totalDeleteRequestDuration + + '}'; } } + /** + * Purge the markers. + * @param tracker tracker with the details + * @param deletePageSize page size of deletes + * @return summary + * @throws MultiObjectDeleteException + * @throws AmazonClientException + * @throws IOException + */ + private MarkerPurgeSummary purgeMarkers(DirMarkerTracker tracker, + int deletePageSize) + throws MultiObjectDeleteException, AmazonClientException, IOException { + + MarkerPurgeSummary summary = new MarkerPurgeSummary(); + // we get a map of surplus markers to delete. + Map> markers + = tracker.getSurplusMarkers(); + int size = markers.size(); + // build a list from the strings in the map + List collect = + markers.values().stream() + .map(p -> new DeleteObjectsRequest.KeyVersion(p.getLeft())) + .collect(Collectors.toList()); + // as an array list so .sublist is straightforward + List markerKeys = new ArrayList<>( + collect); + + // now randomize. Why so? if the list spans multiple S3 partitions, + // it should reduce the IO load on each part. + Collections.shuffle(markerKeys); + int pages = size / deletePageSize; + if (size % deletePageSize > 0) { + pages += 1; + } + if (verbose) { + println(out, "%d markers to delete in %d pages of %d keys/page", + size, pages, deletePageSize); + } + int start = 0; + while (start < size) { + // end is one past the end of the page + int end = Math.min(start + deletePageSize, size); + List page = markerKeys.subList(start, + end); + List undeleted = new ArrayList<>(); + // currently no attempt at doing this in pages. + OperationDuration duration = new OperationDuration(); + operationCallbacks.removeKeys(page, true, undeleted, null, false); + duration.finished(); + summary.deleteRequests++; + summary.totalDeleteRequestDuration += duration.value(); + // and move to the start of the next page + start = end; + } + summary.markersDeleted = size; + return summary; + } + /** * Dump the filesystem Storage Statistics. * @param fs filesystem; can be null @@ -258,4 +460,12 @@ private void dumpFileSystemStatistics(FileSystem fs) { println(out, "%s\t%s", next.getName(), next.getValue()); } } + + public boolean isVerbose() { + return verbose; + } + + public void setVerbose(final boolean verbose) { + this.verbose = verbose; + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/package-info.java index ee0b56a548229..cb3a3749b658c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/package-info.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/package-info.java @@ -17,7 +17,7 @@ */ /** - * Command line tools. + * S3A Command line tools independent of S3Guard. */ @InterfaceAudience.Private @InterfaceStability.Unstable diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/AbstractS3ACostTest.java index e5f098e58480c..970cac805e1e7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/AbstractS3ACostTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/AbstractS3ACostTest.java @@ -102,11 +102,11 @@ public Configuration createConfiguration() { removeBaseAndBucketOverrides(bucketName, conf, S3_METADATA_STORE_IMPL); } - // directory marker options removeBaseAndBucketOverrides(bucketName, conf, DIRECTORY_MARKER_POLICY, METADATASTORE_AUTHORITATIVE, AUTHORITATIVE_PATH); + // directory marker options conf.set(DIRECTORY_MARKER_POLICY, keepMarkers ? DIRECTORY_MARKER_POLICY_KEEP diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java new file mode 100644 index 0000000000000..3a5cbfdf8d414 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java @@ -0,0 +1,284 @@ +/* + * 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.tools; + +import java.io.IOException; +import java.net.URI; +import java.util.ArrayList; +import java.util.List; + +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +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.io.IOUtils; + +import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP; +import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE; +import static org.apache.hadoop.fs.s3a.Constants.S3A_BUCKET_PROBE; +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.getTestBucketName; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; +import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Test the marker tool and use it to compare the behavior + * of keeping vs legacy S3A FS instances. + */ +public class ITestMarkerTool extends AbstractS3ATestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestMarkerTool.class); + + + private S3AFileSystem keepingFS; + private S3AFileSystem mixedFS; + + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + String bucketName = getTestBucketName(conf); + removeBaseAndBucketOverrides(bucketName, conf, + S3A_BUCKET_PROBE, + DIRECTORY_MARKER_POLICY, + METADATASTORE_AUTHORITATIVE, + AUTHORITATIVE_PATH); + // base FS is legacy + conf.set(DIRECTORY_MARKER_POLICY, DIRECTORY_MARKER_POLICY_DELETE); + // turn off bucket probes for a bit of speedup in the connectors we create. + conf.setInt(S3A_BUCKET_PROBE, 0); + return conf; + } + + @Override + public void setup() throws Exception { + super.setup(); + setKeepingFS(createFS(DIRECTORY_MARKER_POLICY_KEEP, null)); + } + + @Override + public void teardown() throws Exception { + IOUtils.cleanupWithLogger(LOG, getKeepingFS(), getMixedFS()); + super.teardown(); + } + + /** + * Create and initialize a new filesystem. + * This filesystem MUST be closed in test teardown. + * @param uri FS URI + * @param config config. + * @return new instance + * @throws IOException failure + */ + private S3AFileSystem createFS(final URI uri, final Configuration config) + throws IOException { + S3AFileSystem fs2 = new S3AFileSystem(); + fs2.initialize(uri, config); + return fs2; + } + + /** + * Create a new FS with given marker policy and path. + * This filesystem MUST be closed in test teardown. + * @param markerPolicy markers + * @param authPath authoritative path. If null: no path. + * @return a new FS. + */ + private S3AFileSystem createFS(String markerPolicy, + String authPath) throws Exception { + S3AFileSystem testFS = getFileSystem(); + Configuration conf = new Configuration(testFS.getConf()); + URI uri = testFS.getUri(); + String bucketName = getTestBucketName(conf); + removeBucketOverrides(bucketName, conf, + DIRECTORY_MARKER_POLICY, + S3_METADATA_STORE_IMPL, + AUTHORITATIVE_PATH); + if (authPath != null) { + + conf.set(AUTHORITATIVE_PATH, authPath); + } + conf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL); + conf.set(DIRECTORY_MARKER_POLICY, markerPolicy); + final S3AFileSystem newFS = createFS(uri, conf); + return newFS; + } + + /** + * FS which deletes markers. + */ + private S3AFileSystem getLegacyFS() { + return getFileSystem(); + } + + + /** + * FS which keeps markers. + */ + private S3AFileSystem getKeepingFS() { + return keepingFS; + } + + private void setKeepingFS(S3AFileSystem keepingFS) { + this.keepingFS = keepingFS; + } + + /** only created on demand. */ + private S3AFileSystem getMixedFS() { + return mixedFS; + } + + private void setMixedFS(S3AFileSystem mixedFS) { + this.mixedFS = mixedFS; + } + + private static class CreatedPaths { + Path base; + List files = new ArrayList<>(); + List dirs = new ArrayList<>(); + List emptyDirs = new ArrayList<>(); + } + + private CreatedPaths createPaths(FileSystem fs, Path base) + throws IOException { + CreatedPaths r = new CreatedPaths(); + r.base = base; + Path dir1 = mkdir(r, fs, base, "dir1"); + Path subdir2 = mkdir(r, fs, dir1, "subdir2"); + Path subdir3 = mkdir(r, fs, subdir2, "subdir3"); + + // create the emtpy dir + Path empty = mkdir(r, fs, base, "empty"); + r.emptyDirs.add(empty); + r.dirs.remove(empty); + + // files + mkfile(r, fs, dir1, "file1"); + mkfile(r, fs, subdir2, "file2"); + mkfile(r, fs, subdir3, "file3"); + return r; + } + + private Path mkdir(CreatedPaths r, FileSystem fs, Path base, String name) + throws IOException { + Path dir = new Path(base, name); + fs.mkdirs(dir); + r.dirs.add(dir); + return dir; + } + + private Path mkfile(CreatedPaths r, + FileSystem fs, Path base, String name) throws IOException { + Path file = new Path(base, name); + ContractTestUtils.touch(fs, file); + r.files.add(file); + return file; + } + + private MarkerTool.ScanResult markerTool( + final FileSystem sourceFS, + final Path path, + final boolean doPurge, + final int expectedMarkerCount) + throws IOException { + return markerTool(0, sourceFS, path, doPurge, expectedMarkerCount); + } + + @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") + private MarkerTool.ScanResult markerTool(final int expected, + final FileSystem sourceFS, + final Path path, + final boolean doPurge, + final int expectedMarkerCount) throws IOException { + MarkerTool tool = new MarkerTool(sourceFS.getConf()); + tool.setVerbose(true); + + MarkerTool.ScanResult result = tool.execute(sourceFS, path, doPurge, + expectedMarkerCount); + Assertions.assertThat(result.exitCode) + .describedAs("Exit code of marker(%s, %s, %d) -> %s", + path, doPurge, expectedMarkerCount, result) + .isEqualTo(expected); + return result; + } + + @Test + public void testAuditPruneMarkersLegacyDir() throws Throwable { + CreatedPaths createdPaths = createPaths(getLegacyFS(), methodPath()); + markerTool(getLegacyFS(), createdPaths.base, false, 0); + markerTool(getLegacyFS(), createdPaths.base, true, 0); + } + + @Test + public void testAuditPruneMarkersKeepingDir() throws Throwable { + CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); + + // audit will find three entries + int expectedMarkerCount = createdPaths.dirs.size(); + S3AFileSystem fs = getLegacyFS(); + markerTool(EXIT_NOT_ACCEPTABLE, fs, + createdPaths.base, false, 0); + + markerTool(fs, createdPaths.base, false, + expectedMarkerCount); + // we know a purge didn't take place + markerTool(fs, createdPaths.base, false, + expectedMarkerCount); + // purge cleans up + markerTool(fs, createdPaths.base, true, expectedMarkerCount); + // and a rerun doesn't find markers + markerTool(fs, createdPaths.base, true, 0); + } + + @Test + public void testRenameKeepingFS() throws Throwable { + describe("Rename with the keeping FS"); + Path base = methodPath(); + Path source = new Path(base, "source"); + Path dest = new Path(base, "dest"); + + S3AFileSystem fs = getKeepingFS(); + CreatedPaths createdPaths = createPaths(fs, source); + + // audit will find three entries + int expectedMarkerCount = createdPaths.dirs.size(); + + markerTool(fs, source, false, + expectedMarkerCount); + fs.rename(source, dest); + assertIsDirectory(dest); + + // there are no markers + markerTool(fs, dest, false, 0); + } + +} From cf8689947ad73ba5ef43cb6cd66bbcae25d1a9df Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 22 Jul 2020 17:44:29 +0100 Subject: [PATCH 06/20] HADOOP-13230 Marker Tool testing; improving API of marker tracker class and integration with rename. Be nice to be able to collect/report IOStats here, but it's not yet going to happen Change-Id: I7d5f450fa0f89ce7c1f44daa6d33e67466061d8e --- .../hadoop/fs/s3a/impl/DirMarkerTracker.java | 129 ++++++++--- .../hadoop/fs/s3a/impl/RenameOperation.java | 163 +++++++++----- .../hadoop/fs/s3a/tools/MarkerTool.java | 50 ++++- .../hadoop/fs/s3a/tools/ITestMarkerTool.java | 205 ++++++++++++++---- 4 files changed, 416 insertions(+), 131 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java index 4d6807d2e76ae..0e5fbf0b39a87 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java @@ -18,12 +18,11 @@ package org.apache.hadoop.fs.s3a.impl; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.TreeMap; -import com.google.common.annotations.VisibleForTesting; - -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; @@ -50,13 +49,13 @@ public class DirMarkerTracker { /** * all leaf markers. */ - private final Map> leafMarkers + private final Map leafMarkers = new TreeMap<>(); /** * all surplus markers. */ - private final Map> surplusMarkers + private final Map surplusMarkers = new TreeMap<>(); /** @@ -69,19 +68,24 @@ public class DirMarkerTracker { */ private int scanCount; + private int filesFound; + + private int markersFound; + /** * A marker has been found; this may or may not be a leaf. * Trigger a move of all markers above it into the surplus map. * @param path marker path * @param key object key * @param source listing source - * @return the number of surplus markers found. + * @return the surplus markers found. */ - public int markerFound(Path path, + public List markerFound(Path path, final String key, final S3ALocatedFileStatus source) { - leafMarkers.put(path, Pair.of(key, source)); - return fileFound(path, key, source); + markersFound++; + leafMarkers.put(path, new Marker(path, key, source)); + return pathFound(path, key, source); } /** @@ -90,47 +94,65 @@ public int markerFound(Path path, * @param path marker path * @param key object key * @param source listing source - * @return the number of surplus markers found. + * @return the surplus markers found. */ - public int fileFound(Path path, + public List fileFound(Path path, final String key, final S3ALocatedFileStatus source) { + filesFound++; + return pathFound(path, key, source); + } + + /** + * A path has been found. Trigger a move of all + * markers above it into the surplus map. + * @param path marker path + * @param key object key + * @param source listing source + * @return the surplus markers found. + */ + public List pathFound(Path path, + final String key, + final S3ALocatedFileStatus source) { + List removed = new ArrayList<>(); + // all parent entries are superfluous final Path parent = path.getParent(); if (parent == null || parent.equals(lastDirChecked)) { // short cut exit - return 0; + return removed; } - final int markers = removeParentMarkers(parent); + removeParentMarkers(parent, removed); lastDirChecked = parent; - return markers; + return removed; } /** * Remove all markers from the path and its parents. * @param path path to start at - * @return number of markers removed. + * @param removed list of markers removed; is built up during the + * recursive operation. */ - private int removeParentMarkers(final Path path) { + private void removeParentMarkers(final Path path, + List removed) { if (path == null || path.isRoot()) { - return 0; + return; } scanCount++; - int parents = removeParentMarkers(path.getParent()); - final Pair value = leafMarkers.remove(path); + removeParentMarkers(path.getParent(), removed); + final Marker value = leafMarkers.remove(path); if (value != null) { // marker is surplus surplusMarkers.put(path, value); - parents++; + removed.add(value); } - return parents; } /** * get the map of leaf markers. * @return all leaf markers. */ - public Map> getLeafMarkers() { + public Map getLeafMarkers() { return leafMarkers; } @@ -138,27 +160,84 @@ public Map> getLeafMarkers() { * get the map of surplus markers. * @return all surplus markers. */ - public Map> getSurplusMarkers() { + public Map getSurplusMarkers() { return surplusMarkers; } - @VisibleForTesting public Path getLastDirChecked() { return lastDirChecked; } - @VisibleForTesting public int getScanCount() { return scanCount; } + public int getFilesFound() { + return filesFound; + } + + public int getMarkersFound() { + return markersFound; + } + @Override public String toString() { return "DirMarkerTracker{" + "leafMarkers=" + leafMarkers.size() + ", surplusMarkers=" + surplusMarkers.size() + ", lastDirChecked=" + lastDirChecked + + ", filesFound=" + filesFound + ", scanCount=" + scanCount + '}'; } + + /** + * This is a marker entry stored in the map and + * returned as markers are deleted. + */ + public static final class Marker { + /** Path of the marker. */ + private final Path path; + + /** + * Key in the store. + */ + private final String key; + + /** + * The file status of the marker. + */ + private final S3ALocatedFileStatus status; + + public Marker(final Path path, + final String key, + final S3ALocatedFileStatus status) { + this.path = path; + this.key = key; + this.status = status; + } + + public Path getPath() { + return path; + } + + public String getKey() { + return key; + } + + public S3ALocatedFileStatus getStatus() { + return status; + } + + @Override + public String toString() { + return "Marker{" + + "path=" + path + + ", key='" + key + '\'' + + ", status=" + status + + '}'; + } + + } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java index c6b14655c7891..69586cbde3f36 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java @@ -32,7 +32,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.s3a.RenameFailedException; @@ -45,6 +44,7 @@ import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.fs.s3a.s3guard.RenameTracker; import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.OperationDuration; import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_BLOCK_SIZE; @@ -186,11 +186,23 @@ private void completeActiveCopies(String reason) throws IOException { /** * Queue an object for deletion. * @param path path to the object + * @param s * @param key key of the object. */ - private void queueToDelete(Path path, String key) { + private void queueToDelete(Path path, String key, String version) { pathsToDelete.add(path); - keysToDelete.add(new DeleteObjectsRequest.KeyVersion(key)); + keysToDelete.add(new DeleteObjectsRequest.KeyVersion(key, version)); + } + + /** + * Queue a list of markers for deletion. + * no-op if the list is empty. + * @param markersToDelete markers + */ + private void queueToDelete( + List markersToDelete) { + markersToDelete.forEach(m -> + queueToDelete(m.getPath(), m.getKey(), m.getStatus().getVersionId())); } /** @@ -331,6 +343,8 @@ protected void recursiveDirectoryRename() throws IOException { // TODO: dir marker policy doesn't always need to do this. callbacks.deleteObjectAtPath(destStatus.getPath(), dstKey, false, null); } + // Track directory markers so that we know which leaf directories need to be + // recreated DirMarkerTracker dirMarkerTracker = new DirMarkerTracker(); Path parentPath = storeContext.keyToPath(srcKey); @@ -353,58 +367,41 @@ protected void recursiveDirectoryRename() throws IOException { Path childSourcePath = storeContext.keyToPath(key); // mark for deletion on a successful copy. - queueToDelete(childSourcePath, key); - - + queueToDelete(childSourcePath, key, child.getVersionId()); + List markersToDelete; boolean isMarker = key.endsWith("/"); if (isMarker) { - // markers are not replicated until we know - // that they are leaf markers. - dirMarkerTracker.markerFound(childSourcePath, key, child); + // add the marker to the tracker. + // it will not be deleted _yet_ but it may find a list of parent + // markers which may now be deleted. + markersToDelete = dirMarkerTracker.markerFound(childSourcePath, key, child); } else { - // its a file, note - dirMarkerTracker.fileFound(childSourcePath, key, child); + // it is a file. + // note that it has been found -ths may find a list of parent + // markers which may now be deleted. + markersToDelete = dirMarkerTracker.fileFound(childSourcePath, key, child); // the destination key is that of the key under the source tree, // remapped under the new destination path. String newDestKey = dstKey + key.substring(srcKey.length()); Path childDestPath = storeContext.keyToPath(newDestKey); + // now begin the single copy - activeCopies.add(initiateCopy(child, key, - childSourcePath, newDestKey, childDestPath)); - bytesCopied.addAndGet(sourceStatus.getLen()); + CompletableFuture copy = initiateCopy(child, key, + childSourcePath, newDestKey, childDestPath); + activeCopies.add(copy); + bytesCopied.addAndGet(sourceStatus.getLen()); } + // add any markers to delete to the operation so they get cleaned + // incrementally + queueToDelete(markersToDelete); + // and trigger any end of loop operations endOfLoopActions(); } // end of iteration through the list - // directory marker work. - // for all leaf markers: copy the original - Map> leafMarkers = - dirMarkerTracker.getLeafMarkers(); - Map> surplus = - dirMarkerTracker.getSurplusMarkers(); - LOG.debug("copying {} leaf markers; {} surplus", - leafMarkers.size(), surplus.size()); - for (Map.Entry> entry : - leafMarkers.entrySet()) { - Path source = entry.getKey(); - String key = entry.getValue().getLeft(); - S3ALocatedFileStatus stat = entry.getValue().getRight(); - String newDestKey = - dstKey + key.substring(srcKey.length()); - Path childDestPath = storeContext.keyToPath(newDestKey); - LOG.debug("copying dir marker from {} to {}", key, newDestKey); - activeCopies.add(initiateCopy(stat, key, - source, newDestKey, childDestPath)); - endOfLoopActions(); - } - // the surplus ones are also explicitly deleted - for (Map.Entry> entry : - surplus.entrySet()) { - queueToDelete(entry.getKey(), entry.getValue().getLeft()); - endOfLoopActions(); - } + // finally process remaining directory markers + copyEmptyDirectoryMarkers(srcKey, dstKey, dirMarkerTracker); // await the final set of copies and their deletion // This will notify the renameTracker that these objects @@ -416,20 +413,86 @@ protected void recursiveDirectoryRename() throws IOException { renameTracker.moveSourceDirectory(); } + /** + * Operations to perform at the end of every loop iteration. + * This may block the thread waiting for copies to complete + * and/or delete a page of data. + */ private void endOfLoopActions() throws IOException { - if (activeCopies.size() == RENAME_PARALLEL_LIMIT) { - // the limit of active copies has been reached; - // wait for completion or errors to surface. - LOG.debug("Waiting for active copies to complete"); - completeActiveCopies("batch threshold reached"); - } if (keysToDelete.size() == pageSize) { // 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("paged delete"); + } else { + if (activeCopies.size() == RENAME_PARALLEL_LIMIT) { + // the limit of active copies has been reached; + // wait for completion or errors to surface. + LOG.debug("Waiting for active copies to complete"); + completeActiveCopies("batch threshold reached"); + } + } + + } + + /** + * Process all directory markers at the end of the rename. + * All leaf markers are queued to be copied in the store; + * this updates the metastore tracker as it does so. + *

+ * Why not simply create new markers? All the metadata + * gets copied too, so if there was anything relevant then + * it would be preserved. + *

+ * At the same time: markers aren't valued much and may + * be deleted without any safety checks -so if there was relevant + * data it is at risk of destruction at any point. + * If there are lots of empty directory rename operations taking place, + * the decision to copy the source may need revisiting. + *

+ * The duration returned is the time to initiate all copy/delete operations, + * including any blocking waits for active copies and paged deletes + * to execute. There may still be outstanding operations + * queued by this method -the duration may be an underestimate + * of the time this operation actually takes. + * + * @param srcKey source key with trailing / + * @param dstKey dest key with trailing / + * @param dirMarkerTracker tracker of markers + * @return how long it took. + */ + private OperationDuration copyEmptyDirectoryMarkers( + final String srcKey, + final String dstKey, + final DirMarkerTracker dirMarkerTracker) throws IOException { + // directory marker work. + LOG.debug("Copying markers from {}", dirMarkerTracker); + final StoreContext storeContext = getStoreContext(); + Map leafMarkers = + dirMarkerTracker.getLeafMarkers(); + Map surplus = + dirMarkerTracker.getSurplusMarkers(); + // for all leaf markers: copy the original + DurationInfo duration = new DurationInfo(LOG, false, + "copying %d leaf markers with %d surplus not copied", + leafMarkers.size(), surplus.size()); + for (DirMarkerTracker.Marker entry: leafMarkers.values()) { + Path source = entry.getPath(); + String key = entry.getKey(); + String newDestKey = + dstKey + key.substring(srcKey.length()); + Path childDestPath = storeContext.keyToPath(newDestKey); + LOG.debug("copying dir marker from {} to {}", key, newDestKey); + activeCopies.add( + initiateCopy( + entry.getStatus(), + key, + source, + newDestKey, + childDestPath)); + // end of loop + endOfLoopActions(); } + duration.close(); + return duration; } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index 30fcfe87b8b60..ee950e3c322d5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -223,14 +223,11 @@ ScanResult execute( */ static final class ScanResult { - /** Exit code to report. */ - int exitCode; + private int exitCode; - /** Tracker which did the scan. */ - DirMarkerTracker tracker; + private DirMarkerTracker tracker; - /** Summary of purge. Null if none took place. */ - MarkerPurgeSummary purgeSummary; + private MarkerPurgeSummary purgeSummary; @Override public String toString() { @@ -240,6 +237,21 @@ public String toString() { ", purgeSummary=" + purgeSummary + '}'; } + + /** Exit code to report. */ + public int getExitCode() { + return exitCode; + } + + /** Tracker which did the scan. */ + public DirMarkerTracker getTracker() { + return tracker; + } + + /** Summary of purge. Null if none took place. */ + public MarkerPurgeSummary getPurgeSummary() { + return purgeSummary; + } } /** @@ -265,9 +277,9 @@ private ScanResult scan( scanDirectoryTree(path, tracker); } // scan done. what have we got? - Map> surplusMarkers + Map surplusMarkers = tracker.getSurplusMarkers(); - Map> leafMarkers + Map leafMarkers = tracker.getLeafMarkers(); int size = surplusMarkers.size(); if (size == 0) { @@ -286,8 +298,11 @@ private ScanResult scan( if (verbose && !leafMarkers.isEmpty()) { println(out, "Found %d empty directory 'leaf' marker%s under %s", leafMarkers.size(), - suffix(size), + suffix(leafMarkers.size()), path); + for (Path markers : leafMarkers.keySet()) { + println(out, " %s", markers); + } println(out, "These are required to indicate empty directories"); } if (size > expectedMarkerCount) { @@ -383,6 +398,19 @@ public String toString() { ", totalDeleteRequestDuration=" + totalDeleteRequestDuration + '}'; } + + + int getMarkersDeleted() { + return markersDeleted; + } + + int getDeleteRequests() { + return deleteRequests; + } + + long getTotalDeleteRequestDuration() { + return totalDeleteRequestDuration; + } } /** @@ -400,13 +428,13 @@ private MarkerPurgeSummary purgeMarkers(DirMarkerTracker tracker, MarkerPurgeSummary summary = new MarkerPurgeSummary(); // we get a map of surplus markers to delete. - Map> markers + Map markers = tracker.getSurplusMarkers(); int size = markers.size(); // build a list from the strings in the map List collect = markers.values().stream() - .map(p -> new DeleteObjectsRequest.KeyVersion(p.getLeft())) + .map(p -> new DeleteObjectsRequest.KeyVersion(p.getKey())) .collect(Collectors.toList()); // as an array list so .sublist is straightforward List markerKeys = new ArrayList<>( diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java index 3a5cbfdf8d414..5523b3de271e7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java @@ -36,19 +36,11 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.io.IOUtils; -import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH; -import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY; -import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE; -import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP; -import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE; -import static org.apache.hadoop.fs.s3a.Constants.S3A_BUCKET_PROBE; -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.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test the marker tool and use it to compare the behavior @@ -61,8 +53,17 @@ public class ITestMarkerTool extends AbstractS3ATestBase { private S3AFileSystem keepingFS; + private S3AFileSystem mixedFS; + private int expectedFiles; + + private int expectMarkersUnderDir1; + + private int expectMarkersUnderDir2; + + private int expectMarkers; + @Override protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); @@ -70,10 +71,13 @@ protected Configuration createConfiguration() { removeBaseAndBucketOverrides(bucketName, conf, S3A_BUCKET_PROBE, DIRECTORY_MARKER_POLICY, + S3_METADATA_STORE_IMPL, METADATASTORE_AUTHORITATIVE, AUTHORITATIVE_PATH); // base FS is legacy conf.set(DIRECTORY_MARKER_POLICY, DIRECTORY_MARKER_POLICY_DELETE); + conf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL); + // turn off bucket probes for a bit of speedup in the connectors we create. conf.setInt(S3A_BUCKET_PROBE, 0); return conf; @@ -117,19 +121,18 @@ private S3AFileSystem createFS(String markerPolicy, String authPath) throws Exception { S3AFileSystem testFS = getFileSystem(); Configuration conf = new Configuration(testFS.getConf()); - URI uri = testFS.getUri(); + URI testFSUri = testFS.getUri(); String bucketName = getTestBucketName(conf); removeBucketOverrides(bucketName, conf, DIRECTORY_MARKER_POLICY, S3_METADATA_STORE_IMPL, AUTHORITATIVE_PATH); if (authPath != null) { - conf.set(AUTHORITATIVE_PATH, authPath); } conf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL); conf.set(DIRECTORY_MARKER_POLICY, markerPolicy); - final S3AFileSystem newFS = createFS(uri, conf); + final S3AFileSystem newFS = createFS(testFSUri, conf); return newFS; } @@ -161,49 +164,90 @@ private void setMixedFS(S3AFileSystem mixedFS) { this.mixedFS = mixedFS; } + private static Path mkpath(Path base, final String name) { + return name.isEmpty() ? base : new Path(base, name); + } + + /** + * Tracker of created paths. + */ private static class CreatedPaths { + Path base; + List files = new ArrayList<>(); + List dirs = new ArrayList<>(); + List emptyDirs = new ArrayList<>(); + + List filesUnderBase = new ArrayList<>(); + + List dirsUnderBase = new ArrayList<>(); + + List emptyDirsUnderBase = new ArrayList<>(); + + + private Path mkdir(FileSystem fs, String name) + throws IOException { + Path dir = mkpath(base, name); + fs.mkdirs(dir); + dirs.add(dir); + dirsUnderBase.add(name); + return dir; + } + + private Path emptydir(FileSystem fs, String name) + throws IOException { + Path dir = mkpath(base, name); + fs.mkdirs(dir); + emptyDirs.add(dir); + emptyDirsUnderBase.add(name); + return dir; + } + + private Path mkfile(FileSystem fs, String name) + throws IOException { + Path file = mkpath(base, name); + ContractTestUtils.touch(fs, file); + files.add(file); + filesUnderBase.add(name); + return file; + } } + /** + * Create the "standard" test paths. + * @param fs filesystem + * @param base base dir + * @return the details on what was created. + */ private CreatedPaths createPaths(FileSystem fs, Path base) throws IOException { CreatedPaths r = new CreatedPaths(); r.base = base; - Path dir1 = mkdir(r, fs, base, "dir1"); - Path subdir2 = mkdir(r, fs, dir1, "subdir2"); - Path subdir3 = mkdir(r, fs, subdir2, "subdir3"); + // the directories under which we will create files, so expect to have markers + r.mkdir(fs, ""); + r.mkdir(fs, "dir1"); + r.mkdir(fs, "dir2"); + r.mkdir(fs, "dir2/dir3"); - // create the emtpy dir - Path empty = mkdir(r, fs, base, "empty"); - r.emptyDirs.add(empty); - r.dirs.remove(empty); + // create the empty dirs + r.emptydir(fs, "empty"); + r.emptydir(fs, "dir2/empty"); // files - mkfile(r, fs, dir1, "file1"); - mkfile(r, fs, subdir2, "file2"); - mkfile(r, fs, subdir3, "file3"); + r.mkfile(fs, "dir1/file1"); + r.mkfile(fs, "dir2/file2"); + r.mkfile(fs, "dir2/dir3/file3"); + + expectedFiles = 3; + expectMarkersUnderDir1 = 1; + expectMarkersUnderDir2 = 2; + expectMarkers = expectMarkersUnderDir1 + expectMarkersUnderDir2; return r; } - private Path mkdir(CreatedPaths r, FileSystem fs, Path base, String name) - throws IOException { - Path dir = new Path(base, name); - fs.mkdirs(dir); - r.dirs.add(dir); - return dir; - } - - private Path mkfile(CreatedPaths r, - FileSystem fs, Path base, String name) throws IOException { - Path file = new Path(base, name); - ContractTestUtils.touch(fs, file); - r.files.add(file); - return file; - } - private MarkerTool.ScanResult markerTool( final FileSystem sourceFS, final Path path, @@ -214,7 +258,8 @@ private MarkerTool.ScanResult markerTool( } @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") - private MarkerTool.ScanResult markerTool(final int expected, + private MarkerTool.ScanResult markerTool( + final int exitCode, final FileSystem sourceFS, final Path path, final boolean doPurge, @@ -224,10 +269,10 @@ private MarkerTool.ScanResult markerTool(final int expected, MarkerTool.ScanResult result = tool.execute(sourceFS, path, doPurge, expectedMarkerCount); - Assertions.assertThat(result.exitCode) + Assertions.assertThat(result.getExitCode()) .describedAs("Exit code of marker(%s, %s, %d) -> %s", path, doPurge, expectedMarkerCount, result) - .isEqualTo(expected); + .isEqualTo(exitCode); return result; } @@ -242,7 +287,7 @@ public void testAuditPruneMarkersLegacyDir() throws Throwable { public void testAuditPruneMarkersKeepingDir() throws Throwable { CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); - // audit will find three entries + // audit will find the expected entries int expectedMarkerCount = createdPaths.dirs.size(); S3AFileSystem fs = getLegacyFS(); markerTool(EXIT_NOT_ACCEPTABLE, fs, @@ -254,14 +299,17 @@ public void testAuditPruneMarkersKeepingDir() throws Throwable { markerTool(fs, createdPaths.base, false, expectedMarkerCount); // purge cleans up - markerTool(fs, createdPaths.base, true, expectedMarkerCount); + assertMarkersDeleted(expectedMarkerCount, + markerTool(fs, createdPaths.base, true, expectedMarkerCount)); // and a rerun doesn't find markers - markerTool(fs, createdPaths.base, true, 0); + assertMarkersDeleted(0, + markerTool(fs, createdPaths.base, true, 0)); } @Test public void testRenameKeepingFS() throws Throwable { - describe("Rename with the keeping FS"); + describe( + "Rename with the keeping FS -verify that no markers exist at far end"); Path base = methodPath(); Path source = new Path(base, "source"); Path dest = new Path(base, "dest"); @@ -279,6 +327,73 @@ public void testRenameKeepingFS() throws Throwable { // there are no markers markerTool(fs, dest, false, 0); + LOG.info("Auditing destination paths"); + verifyRenamed(dest, createdPaths); + } + void verifyRenamed(final Path dest, + final CreatedPaths createdPaths) throws IOException { + // all leaf directories exist + for (String p : createdPaths.emptyDirsUnderBase) { + assertIsDirectory(mkpath(dest, p)); + } + // non-empty dirs + for (String p : createdPaths.dirsUnderBase) { + assertIsDirectory(mkpath(dest, p)); + } + // all files exist + for (String p : createdPaths.filesUnderBase) { + assertIsFile(mkpath(dest, p)); + } + } + + /** + * Create a FS where only dir2 in the source tree keeps markers; + * verify all is good. + */ + @Test + public void testAuthPathIsMixed() throws Throwable { + describe("Create a source tree with mixed semantics"); + Path base = methodPath(); + Path source = new Path(base, "source"); + Path dest = new Path(base, "dest"); + Path dir2 = new Path(source, "dir2"); + S3AFileSystem mixedFS = createFS(DIRECTORY_MARKER_POLICY_AUTHORITATIVE, + dir2.toUri().toString()); + // some of these paths will retain markers, some will not + CreatedPaths createdPaths = createPaths(mixedFS, source); + + // markers are only under dir2 + markerTool(mixedFS, mkpath(source, "dir1"), false, 0); + markerTool(mixedFS, source, false, expectMarkersUnderDir2); + + // if we now rename, all will be good + mixedFS.rename(source, dest); + assertIsDirectory(dest); + + // there are no markers + MarkerTool.ScanResult scanResult = markerTool(mixedFS, dest, false, 0); + // there are exactly the files we want + Assertions.assertThat(scanResult) + .describedAs("Scan result %s", scanResult) + .extracting(s -> s.getTracker().getFilesFound()) + .isEqualTo(expectedFiles); + verifyRenamed(dest, createdPaths); + } + + /** + * Assert that an expected number of markers were deleted. + * @param expected expected count. + * @param result scan result + */ + private static void assertMarkersDeleted(int expected, + MarkerTool.ScanResult result) { + + Assertions.assertThat(result.getPurgeSummary()) + .describedAs("Purge result of scan %s", result) + .isNotNull() + .extracting(f -> f.getMarkersDeleted()) + .isEqualTo(expected); + } } From 0eb7a6544fc81a188650222931233b9c90e49f37 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 27 Jul 2020 16:51:48 +0100 Subject: [PATCH 07/20] HADOOP-13230 directory marker tests 1. move new test code from s3a/test/costs to s3a/performance 2. new standalone test suite for semantics of list and other fs operations the new test ITestDirectoryMarkerListing is intended to be backportable to older hadoop versions and so verify correctness there. So -no use of new classes -or assertj -or anything known to be recent in the codebase Todo: the rename test cases. Always trouble, especially as markers will be copied. Change-Id: I7a3d577012bdd2d4137db0a214a01e68c24ef2e0 --- .../fs/s3a/ITestS3AFileOperationCost.java | 4 +- .../AbstractS3ACostTest.java | 8 +- .../costs => performance}/HeadListCosts.java | 2 +- .../ITestDirectoryMarkerListing.java | 445 ++++++++++++++++++ .../ITestS3ADeleteCost.java | 6 +- .../ITestS3ARenameCost.java | 4 +- .../OperationCostValidator.java | 2 +- 7 files changed, 458 insertions(+), 13 deletions(-) rename hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/{test/costs => performance}/AbstractS3ACostTest.java (98%) rename hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/{test/costs => performance}/HeadListCosts.java (98%) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java rename hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/{test/costs => performance}/ITestS3ADeleteCost.java (96%) rename hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/{test/costs => performance}/ITestS3ARenameCost.java (98%) rename hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/{test/costs => performance}/OperationCostValidator.java (99%) 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 95e1feeb50a4c..2dc7efafee654 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 @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; -import org.apache.hadoop.fs.s3a.test.costs.AbstractS3ACostTest; +import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest; import org.junit.Test; @@ -43,7 +43,7 @@ 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.fs.s3a.test.costs.HeadListCosts.*; +import static org.apache.hadoop.fs.s3a.performance.HeadListCosts.*; import static org.apache.hadoop.test.GenericTestUtils.getTestDir; /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java similarity index 98% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/AbstractS3ACostTest.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java index 970cac805e1e7..f47e03360ecc9 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/AbstractS3ACostTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.test.costs; +package org.apache.hadoop.fs.s3a.performance; import java.io.FileNotFoundException; import java.io.IOException; @@ -40,9 +40,9 @@ import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.*; -import static org.apache.hadoop.fs.s3a.test.costs.HeadListCosts.*; -import static org.apache.hadoop.fs.s3a.test.costs.OperationCostValidator.probe; -import static org.apache.hadoop.fs.s3a.test.costs.OperationCostValidator.probes; +import static org.apache.hadoop.fs.s3a.performance.HeadListCosts.*; +import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.probe; +import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.probes; import static org.apache.hadoop.test.AssertExtensions.dynamicDescription; /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/HeadListCosts.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/HeadListCosts.java similarity index 98% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/HeadListCosts.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/HeadListCosts.java index ef15a725d7acf..5ab94f150e866 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/HeadListCosts.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/HeadListCosts.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.test.costs; +package org.apache.hadoop.fs.s3a.performance; /** * Declaration of the costs of head and list calls for various FS IO operations. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java new file mode 100644 index 0000000000000..619f4c0395a73 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java @@ -0,0 +1,445 @@ +/* + * 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.performance; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.stream.Collectors; + +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.model.AmazonS3Exception; +import com.amazonaws.services.s3.model.ObjectMetadata; +import org.junit.Assume; +import org.junit.FixMethodOrder; +import org.junit.Test; +import org.junit.runners.MethodSorters; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; +import org.apache.hadoop.fs.s3a.Invoker; +import org.apache.hadoop.fs.s3a.S3AFileSystem; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.createTestPath; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * This is a test suite designed to verify that directory markers do + * not get misconstrued as empty directories during operations + * which explicitly or implicitly list directory trees. + *

+ * It is also intended it to be backported to all releases + * which are enhanced to read directory trees where markers have + * been retained. + * Hence: it does not use any of the new helper classes to + * measure the cost of operations or attempt to create markers + * through the FS APIs. + *

+ * Instead, the directory structure to test is created through + * low-level S3 SDK API calls. + * We also skip any probes to measure/assert metrics. + * We're testing the semantics here, not the cost of the operations. + * Doing that makes it a lot easier to backport. + * + *

+ * Similarly: JUnit assertions over AssertJ. + *

+ * The tests work with unguarded buckets only. + */ +@FixMethodOrder(MethodSorters.NAME_ASCENDING) +public class ITestDirectoryMarkerListing extends AbstractS3ATestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestDirectoryMarkerListing.class); + + private static final String NAME = "ITestDirectoryMarkerListing"; + + private static final String FILENAME = "fileUnderMarker"; + + private static final String HELLO = "hello"; + + private Path markerDir; + + private String markerKey; + + private String markerKeySlash; + + private String bucket; + + private AmazonS3 s3client; + + private String fileUnderMarker; + + private Path pathUnderMarker; + + private Path basePath; + + private Path markerPeer; + + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + String bucketName = getTestBucketName(conf); + + removeBaseAndBucketOverrides(bucketName, conf, + S3_METADATA_STORE_IMPL); + return conf; + } + + @Override + public void setup() throws Exception { + super.setup(); + S3AFileSystem fs = getFileSystem(); + assume("unguarded FS only", + !fs.hasMetadataStore()); + + basePath = fs.qualify(createTestPath(new Path("/base-" + NAME))); + markerDir = new Path(basePath, "marker"); + // peer path has the same initial name to make sure there + // is no confusion there. + markerPeer = new Path(basePath, "markerpeer"); + markerKey = fs.pathToKey(markerDir); + markerKeySlash = markerKey + "/"; + fileUnderMarker = markerKeySlash + FILENAME; + pathUnderMarker = new Path(markerDir, FILENAME); + bucket = fs.getBucket(); + s3client = fs.getAmazonS3ClientForTesting("markers"); + } + + + @Test + public void test_010_createMarker() throws Throwable { + describe("Create the test markers for the suite"); + createTestObjects(); + head(markerKeySlash); + assertIsDirectory(markerDir); + } + + @Test + public void test_020_objectUnderMarker() throws Throwable { + assertIsFile(pathUnderMarker); + assertIsDirectory(markerDir); + head(fileUnderMarker); + + } + + @Test + public void test_030_listStatus() throws Throwable { + assumeTestObjectsExist(); + containsFileUnderMarkerOnly( + toList(getFileSystem().listStatus(markerDir))); + } + + + /* + ================================================================= + The listing operations + ================================================================= + */ + + @Test + public void test_040_listStatus() throws Throwable { + assumeTestObjectsExist(); + List statuses = toList( + getFileSystem().listFiles(markerDir, false)); + containsFileUnderMarkerOnly(statuses); + } + + @Test + public void test_050_listStatusRecursive() throws Throwable { + assumeTestObjectsExist(); + List statuses = toList( + getFileSystem().listFiles(markerDir, true)); + containsFileUnderMarkerOnly(statuses); + } + + /** + * Path listing above the base dir MUST only find the file + * and not the marker. + */ + @Test + public void test_060_listStatusBaseRecursive() throws Throwable { + assumeTestObjectsExist(); + List statuses = toList( + getFileSystem().listFiles(basePath, true)); + containsStatusOfPaths(statuses, pathUnderMarker, markerPeer); + } + + @Test + public void test_070_globStatusBaseRecursive() throws Throwable { + assumeTestObjectsExist(); + List statuses = + exec("glob", () -> + toList(getFileSystem().globStatus(new Path(basePath, "*")))); + containsStatusOfPaths(statuses, markerDir, markerPeer); + isFileAtPath(markerPeer, statuses.get(1)); + } + + @Test + public void test_080_globStatusMarkerDir() throws Throwable { + assumeTestObjectsExist(); + List statuses = + exec("glob", () -> + toList(getFileSystem().globStatus(new Path(markerDir, "*")))); + containsFileUnderMarkerOnly(statuses); + } + + /** + * The list here returns the marker peer and dir in a different order. Wny? + * + */ + @Test + public void test_090_listLocatedStatusBaseDir() throws Throwable { + assumeTestObjectsExist(); + List statuses = + exec("listLocatedStatus", () -> + toList(getFileSystem().listLocatedStatus(basePath))); + + containsStatusOfPaths(statuses, markerPeer, markerDir); + } + + @Test + public void test_100_listLocatedStatusMarkerDir() throws Throwable { + assumeTestObjectsExist(); + List statuses = + exec("listLocatedStatus", () -> + toList(getFileSystem().listLocatedStatus(markerDir))); + + containsFileUnderMarkerOnly(statuses); + } + + + /* + ================================================================= + Creation Rejection + ================================================================= + */ + + @Test + public void test_200_create_no_overwrite_marker() throws Throwable { + assumeTestObjectsExist(); + head(markerKeySlash); + intercept(FileAlreadyExistsException.class, () -> + exec("create", () -> + getFileSystem().create(markerDir, false))); + // dir is still there. + head(markerKeySlash); + } + + @Test + public void test_210_create_no_overwrite_file() throws Throwable { + head(fileUnderMarker); + intercept(FileAlreadyExistsException.class, () -> + exec("create", () -> + getFileSystem().create(pathUnderMarker, false))); + verifyTestObjectsExist(); + } + + @Test + public void test_220_createfile_no_overwrite() throws Throwable { + head(fileUnderMarker); + intercept(FileAlreadyExistsException.class, () -> + exec("create", () -> + getFileSystem().createFile(pathUnderMarker) + .overwrite(false) + .build())); + verifyTestObjectsExist(); + } + + /* + ================================================================= + Rename. + These tests use methodPaths for src and test + ================================================================= + */ + + @Test + public void test_300_rename_base() throws Throwable { + + } + + /* + ================================================================= + Delete. + ================================================================= + */ + + @Test + public void test_900_delete() throws Throwable { + assumeTestObjectsExist(); + S3AFileSystem fs = getFileSystem(); + intercept(PathIsNotEmptyDirectoryException.class, () -> + fs.delete(markerDir, false)); + head(fileUnderMarker); + fs.delete(markerDir, true); + intercept(AmazonS3Exception.class, () -> + head(fileUnderMarker)); + fs.delete(basePath, true); + } + + /* + ================================================================= + Utility methods and assertions. + ================================================================= + */ + + /** + * Creates the test objects at the well known paths; + * no probes for existence before or after. + */ + private void createTestObjects() throws Exception { + S3AFileSystem fs = getFileSystem(); + // put the empty dir + fs.mkdirs(markerDir); + touch(fs, markerPeer); + put(fileUnderMarker, HELLO); + } + + private void assumeTestObjectsExist() throws Exception { + assumeExists(fileUnderMarker); + assumeExists(markerKeySlash); + } + + private void verifyTestObjectsExist() throws Exception { + head(fileUnderMarker); + head(markerKeySlash); + } + + private void put(final String key, final String content) throws Exception { + exec("PUT " + key, () -> + s3client.putObject(bucket, key, content)); + } + + + private ObjectMetadata head(final String key) throws Exception { + return exec("HEAD " + key, () -> + s3client.getObjectMetadata(bucket, key)); + } + + private void assumeExists(final String key) throws Exception { + try { + head(key); + } catch (AmazonS3Exception e) { + Assume.assumeTrue("object " + key + " not found", false); + } + } + + private String marker(Path path) throws Exception { + String key = getFileSystem().pathToKey(path) + "/"; + put(key, ""); + return key; + } + + + private ObjectMetadata head(final Path path) throws Exception { + return head(getFileSystem().pathToKey(path)); + } + + private T exec(String op, Callable call) throws Exception { + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); + try { + return call.call(); + } finally { + timer.end(op); + } + } + + private void containsFileUnderMarkerOnly(final List statuses) { + assertEquals("Status length", + 1, statuses.size()); + isFileUnderMarker(statuses.get(0)); + + } + + /** + * Expect the list of status to match that of the paths + * @param statuses + * @param paths + * @param + */ + private void containsStatusOfPaths( + List statuses, Path... paths) { + String summary = statuses.stream() + .map(Object::toString) + .collect(Collectors.joining(";")); + assertEquals("mismatch in size of listing " + summary, + paths.length, statuses.size()); + for (int i = 0; i < statuses.size(); i++) { + assertEquals("Path mismatch at element " + i + " in " + summary, + paths[i], statuses.get(i).getPath()); + } + } + + private void isFileUnderMarker(final FileStatus stat) { + isFileAtPath(pathUnderMarker, stat); + } + + private void isFileAtPath(final Path path, final FileStatus stat) { + assertTrue("Is not file " + stat, + stat.isFile()); + assertName(path, stat); + } + + private void assertName(final Path path, final FileStatus stat) { + assertEquals("filename is not the expected path :" + stat, + path, stat.getPath()); + } + + private List + toList(RemoteIterator status) throws IOException { + + List l = new ArrayList<>(); + while (status.hasNext()) { + l.add(status.next()); + } + return dump(l); + + } + + private List + toList(T[] status) throws IOException { + List l = Arrays.asList(status); + return dump(l); + } + + private List dump(List l) { + int c = 1; + for (T t : l) { + LOG.info("{}\t{}", c++, t); + } + return l; + } + + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/ITestS3ADeleteCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java similarity index 96% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/ITestS3ADeleteCost.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java index 6303047cc904e..72dece4951851 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/ITestS3ADeleteCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.test.costs; +package org.apache.hadoop.fs.s3a.performance; import java.util.Arrays; @@ -35,8 +35,8 @@ import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import static org.apache.hadoop.fs.s3a.Statistic.*; -import static org.apache.hadoop.fs.s3a.test.costs.HeadListCosts.*; -import static org.apache.hadoop.fs.s3a.test.costs.OperationCostValidator.probe; +import static org.apache.hadoop.fs.s3a.performance.HeadListCosts.*; +import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.probe; /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/ITestS3ARenameCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java similarity index 98% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/ITestS3ARenameCost.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java index 209bf0741f028..e752187589bad 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/ITestS3ARenameCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.test.costs; +package org.apache.hadoop.fs.s3a.performance; import java.util.Arrays; @@ -34,7 +34,7 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import static org.apache.hadoop.fs.s3a.Statistic.*; -import static org.apache.hadoop.fs.s3a.test.costs.HeadListCosts.*; +import static org.apache.hadoop.fs.s3a.performance.HeadListCosts.*; /** * Use metrics to assert about the cost of file API calls. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/OperationCostValidator.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java similarity index 99% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/OperationCostValidator.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java index 63f9f77cb6eb2..91605802b016f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/costs/OperationCostValidator.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.test.costs; +package org.apache.hadoop.fs.s3a.performance; import java.util.ArrayList; import java.util.Arrays; From 62dfba394a185dbff4b6545a8ef648c66f7e7a0b Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 28 Jul 2020 14:42:05 +0100 Subject: [PATCH 08/20] HADOOP-13230 checkstyle warnings (ignoring the test_000 ones) Change-Id: Ia64fd7f0b4670a69d1c174d0c5b67211a1a1ae35 --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 87 +++++++++++-------- .../fs/s3a/impl/OperationCallbacks.java | 19 +--- .../hadoop/fs/s3a/impl/RenameOperation.java | 16 ++-- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 5 +- .../hadoop/fs/s3a/tools/MarkerTool.java | 14 +-- .../fs/s3a/ITestS3ABucketExistence.java | 5 +- .../fs/s3a/ITestS3AFileOperationCost.java | 6 +- .../s3a/ITestS3GuardOutOfBandOperations.java | 1 - .../s3a/impl/TestDirectoryMarkerPolicy.java | 7 -- .../s3a/performance/AbstractS3ACostTest.java | 65 +++++++------- .../fs/s3a/performance/HeadListCosts.java | 4 +- .../ITestDirectoryMarkerListing.java | 13 ++- .../s3a/performance/ITestS3ADeleteCost.java | 12 +-- .../s3a/performance/ITestS3ARenameCost.java | 7 +- .../hadoop/fs/s3a/tools/ITestMarkerTool.java | 29 ++++--- 15 files changed, 143 insertions(+), 147 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 1069972cfafbc..96ab61fc22cf4 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 @@ -462,12 +462,10 @@ public void initialize(URI name, Configuration originalConf) DEFAULT_S3GUARD_DISABLED_WARN_LEVEL); S3Guard.logS3GuardDisabled(LOG, warnLevel, bucket); } - // directory policy, which will look at authoritative paths - // if needed + // directory policy, which may look at authoritative paths directoryPolicy = new DirectoryPolicyImpl(conf, this::allowAuthoritative); - LOG.debug("Directory marker retention policy is {}", - directoryPolicy); + LOG.debug("Directory marker retention policy is {}", directoryPolicy); initMultipartUploads(conf); @@ -1460,7 +1458,7 @@ private Pair initiateRename( // Parent must exist Path parent = dst.getParent(); if (!pathToKey(parent).isEmpty() - && !parent.equals(src.getParent()) ) { + && !parent.equals(src.getParent())) { try { // only look against S3 for directories; saves // a HEAD request on all normal operations. @@ -1571,17 +1569,6 @@ public void deleteObjectAtPath(final Path path, operationState)); } - @Override - @Retries.RetryTranslated - public void deleteDirectoryMarkers(final Path path, - final String key, - final BulkOperationState operationState) - throws IOException { - if (!keepDirectoryMarkers(path)) { - deleteUnnecessaryFakeDirectories(path, operationState); - } - } - @Override @Retries.RetryTranslated public RemoteIterator listFilesAndEmptyDirectories( @@ -2950,18 +2937,27 @@ S3AFileStatus innerGetFileStatus(final Path f, // a file has been found in a non-auth path and the caller has not said // they only care about directories LOG.debug("Metadata for {} found in the non-auth metastore.", path); - // TODO: if the timestamp of the pm is close to "now", we don't need to - // TODO: bother with a check of S3. that means: - // TODO: one of : status modtime is close to now, - // TODO: or pm.getLastUpdated() == now - long validTime = ttlTimeProvider.getNow() - ttlTimeProvider.getMetadataTtl(); + // If the timestamp of the pm is close to "now", we don't need to + // bother with a check of S3. that means: + // one of : status modtime is close to now, + // or pm.getLastUpdated() == now + + // get the time in which a status modtime is considered valid + // in a non-auth metastore + long validTime = + ttlTimeProvider.getNow() - ttlTimeProvider.getMetadataTtl(); final long msModTime = msStatus.getModificationTime(); if (msModTime < validTime) { - LOG.debug("Metastore entry is out of date, probing S3"); + LOG.debug("Metastore entry of {} is out of date, probing S3", path); try { - S3AFileStatus s3AFileStatus = s3GetFileStatus(path, key, probes, - tombstones, needEmptyDirectoryFlag); + S3AFileStatus s3AFileStatus = s3GetFileStatus(path, + key, + probes, + tombstones, + needEmptyDirectoryFlag); + // if the new status is more current than that in the metastore, + // it means S3 has changed and the store needs updating final long s3ModTime = s3AFileStatus.getModificationTime(); if (s3ModTime > msModTime) { @@ -2969,10 +2965,8 @@ S3AFileStatus innerGetFileStatus(final Path f, LOG.debug("S3Guard metadata for {} is outdated;" + " s3modtime={}; msModTime={} updating metastore", path, s3ModTime, msModTime); - // add to S3Guard and return the value - // note that the checks for empty dir status below can be skipped - // because the call to s3GetFileStatus include the checks there - return S3Guard.putAndReturn(metadataStore, s3AFileStatus, + // add to S3Guard + S3Guard.putAndReturn(metadataStore, s3AFileStatus, ttlTimeProvider); } else { // the modtime of the data is the same as/older than the s3guard @@ -2982,6 +2976,10 @@ S3AFileStatus innerGetFileStatus(final Path f, S3Guard.refreshEntry(metadataStore, pm, s3AFileStatus, ttlTimeProvider); } + // return the value + // note that the checks for empty dir status below can be skipped + // because the call to s3GetFileStatus include the checks there + return s3AFileStatus; } catch (FileNotFoundException fne) { // the attempt to refresh the record failed because there was // no entry. Either it is a new file not visible, or it @@ -3018,7 +3016,10 @@ S3AFileStatus innerGetFileStatus(final Path f, // now issue the S3 getFileStatus call. try { - S3AFileStatus s3FileStatus = s3GetFileStatus(path, key, probes, tombstones, + S3AFileStatus s3FileStatus = s3GetFileStatus(path, + key, + probes, + tombstones, true); // entry was found, so save in S3Guard and return the final value. return S3Guard.putAndReturn(metadataStore, s3FileStatus, @@ -3033,7 +3034,11 @@ S3AFileStatus innerGetFileStatus(final Path f, // there was no entry in S3Guard // retrieve the data and update the metadata store in the process. return S3Guard.putAndReturn(metadataStore, - s3GetFileStatus(path, key, probes, tombstones, needEmptyDirectoryFlag), + s3GetFileStatus(path, + key, + probes, + tombstones, + needEmptyDirectoryFlag), ttlTimeProvider); } } @@ -3169,7 +3174,8 @@ S3AFileStatus s3GetFileStatus(final Path path, listResult.logAtDebug(LOG); } // At least one entry has been found. - // If looking for an empty directory, the marker must exist but no children. + // If looking for an empty directory, the marker must exist but no + // children. // So the listing must contain the marker entry only. if (needEmptyDirectoryFlag && listResult.representsEmptyDirectory( @@ -3747,16 +3753,21 @@ private Optional generateSSECustomerKey() { /** * Perform post-write actions. - * Calls {@link #deleteUnnecessaryFakeDirectories(Path, BulkOperationState)} and then - * updates any metastore. + *

* This operation MUST be called after any PUT/multipart PUT completes * successfully. - * - * The operations actions include + *

+ * The actions include: *
    - *
  1. Calling {@link #deleteUnnecessaryFakeDirectories(Path, BulkOperationState)}
  2. - *
  3. Updating any metadata store with details on the newly created - * object.
  4. + *
  5. + * Calling + * {@link #deleteUnnecessaryFakeDirectories(Path, BulkOperationState)} + * if directory markers are not being retained. + *
  6. + *
  7. + * Updating any metadata store with details on the newly created + * object. + *
  8. *
* @param key key written to * @param length total length of file written diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java index 7e74a08e8f99e..61c04d503c7bb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.impl; +import javax.annotation.Nullable; import java.io.IOException; import java.io.InterruptedIOException; import java.util.List; @@ -104,24 +105,6 @@ void deleteObjectAtPath(Path path, BulkOperationState operationState) throws IOException; - /** - * Delete a directory marker also updating the metastore. - * If the marker retention policy is to keep markers under this - * path, the marker is not deleted. - * This call does not create any mock parent entries. - * Retry policy: retry untranslated; delete considered idempotent. - * @param path path to delete - * @param key key of entry - * @param operationState (nullable) operational state for a bulk update - * @throws AmazonClientException problems working with S3 - * @throws IOException IO failure in the metastore - */ - @Retries.RetryTranslated - void deleteDirectoryMarkers(final Path path, - final String key, - final BulkOperationState operationState) - throws IOException; - /** * Recursive list of files and empty directories. * diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java index 69586cbde3f36..08ed363bd7ddc 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java @@ -375,23 +375,25 @@ protected void recursiveDirectoryRename() throws IOException { // add the marker to the tracker. // it will not be deleted _yet_ but it may find a list of parent // markers which may now be deleted. - markersToDelete = dirMarkerTracker.markerFound(childSourcePath, key, child); + markersToDelete = dirMarkerTracker.markerFound( + childSourcePath, key, child); } else { // it is a file. // note that it has been found -ths may find a list of parent // markers which may now be deleted. - markersToDelete = dirMarkerTracker.fileFound(childSourcePath, key, child); + markersToDelete = dirMarkerTracker.fileFound( + childSourcePath, key, child); // the destination key is that of the key under the source tree, // remapped under the new destination path. String newDestKey = dstKey + key.substring(srcKey.length()); Path childDestPath = storeContext.keyToPath(newDestKey); - // now begin the single copy - CompletableFuture copy = initiateCopy(child, key, - childSourcePath, newDestKey, childDestPath); - activeCopies.add(copy); - bytesCopied.addAndGet(sourceStatus.getLen()); + // now begin the single copy + CompletableFuture copy = initiateCopy(child, key, + childSourcePath, newDestKey, childDestPath); + activeCopies.add(copy); + bytesCopied.addAndGet(sourceStatus.getLen()); } // add any markers to delete to the operation so they get cleaned // incrementally 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 44c9ac81c77c7..7c92fbe728717 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 @@ -108,8 +108,8 @@ public abstract class S3GuardTool extends Configured implements Tool, "\t" + Prune.NAME + " - " + Prune.PURPOSE + "\n" + "\t" + SetCapacity.NAME + " - " + SetCapacity.PURPOSE + "\n" + "\t" + SelectTool.NAME + " - " + SelectTool.PURPOSE + "\n" + - "\t" + Uploads.NAME + " - " + Uploads.PURPOSE + "\n" - ; + "\t" + Uploads.NAME + " - " + Uploads.PURPOSE + "\n"; + private static final String DATA_IN_S3_IS_PRESERVED = "(all data in S3 is preserved)"; @@ -119,6 +119,7 @@ public abstract class S3GuardTool extends Configured implements Tool, static final int SUCCESS = EXIT_SUCCESS; static final int INVALID_ARGUMENT = EXIT_COMMAND_ARGUMENT_ERROR; static final int E_USAGE = EXIT_USAGE; + static final int ERROR = EXIT_FAIL; static final int E_BAD_STATE = EXIT_NOT_ACCEPTABLE; static final int E_NOT_FOUND = EXIT_NOT_FOUND; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index ee950e3c322d5..134f0195d9861 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -34,7 +34,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -101,7 +100,7 @@ public final class MarkerTool extends S3GuardTool { static final String TOO_FEW_ARGUMENTS = "Too few arguments"; - /** Will be overridden in run(), but during tests needs to avoid NPEs */ + /** Will be overridden in run(), but during tests needs to avoid NPEs. */ private PrintStream out = System.out; private boolean verbose; @@ -181,12 +180,17 @@ public int run(final String[] args, final PrintStream stream) break; default: errorln(getUsage()); - throw new ExitUtil.ExitException(EXIT_USAGE, "Unknown action: " + action); + throw new ExitUtil.ExitException(EXIT_USAGE, + "Unknown action: " + action); } final String file = parsedArgs.get(1); final Path path = new Path(file); - ScanResult result = execute(path.getFileSystem(getConf()), path, purge, expected); + ScanResult result = execute( + path.getFileSystem(getConf()), + path, + purge, + expected); return result.exitCode; } @@ -337,7 +341,7 @@ public static Logger getLOG() { } /** - * Scan a directory tree + * Scan a directory tree. * @param path path to scan * @param tracker tracker to update * @throws IOException diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java index f6c824e6d23bf..8c215d79ea680 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java @@ -76,8 +76,9 @@ public void testNoBucketProbing() throws Exception { // the exception must not be caught and marked down to an FNFE expectUnknownStore(() -> fs.exists(src)); // now that isFile() only does a HEAD, it will get a 404 without - // the no-such-bucket error (really) - assertFalse("isFile(" + src + " was expected to complete by returning false", + // the no-such-bucket error. + assertFalse("isFile(" + src + ")" + + " was expected to complete by returning false", fs.isFile(src)); expectUnknownStore(() -> fs.isDirectory(src)); expectUnknownStore(() -> fs.mkdirs(src)); 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 2dc7efafee654..820534e031ecd 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 @@ -266,9 +266,9 @@ public void testCostOfCopyFromLocalFile() throws Throwable { Path remotePath = methodPath(); verifyMetrics(() -> { - s3a.copyFromLocalFile(false, true, localPath, remotePath); - return "copy"; - }, + s3a.copyFromLocalFile(false, true, localPath, remotePath); + return "copy"; + }, always(INVOCATION_COPY_FROM_LOCAL_FILE, 1), always(OBJECT_PUT_REQUESTS, 1), always(OBJECT_PUT_BYTES, len)); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java index 1b3f62d185b61..2d4173d1c2ad4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java @@ -58,7 +58,6 @@ import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH; import static org.apache.hadoop.fs.s3a.Constants.CHANGE_DETECT_MODE; import static org.apache.hadoop.fs.s3a.Constants.CHANGE_DETECT_MODE_NONE; -import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_METADATASTORE_METADATA_TTL; import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE; import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_METADATA_TTL; import static org.apache.hadoop.fs.s3a.Constants.RETRY_INTERVAL; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java index 362fcf2f6e6a6..9bfe0f58090de 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java @@ -85,13 +85,6 @@ private DirectoryPolicyImpl retention( return new DirectoryPolicyImpl(c, authoritativeness); } - private static final Predicate authPathOnly = - (p) -> p.toUri().getPath().startsWith("/auth/"); - - private static final Predicate failIfInvoked = (p) -> { - throw new RuntimeException("failed"); - }; - private final Path nonAuthPath = new Path("s3a://bucket/nonauth/data"); private final Path authPath = new Path("s3a://bucket/auth/data1"); private final Path deepAuth = new Path("s3a://bucket/auth/d1/d2/data2"); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java index f47e03360ecc9..1f9ee1a800b64 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java @@ -55,30 +55,30 @@ public class AbstractS3ACostTest extends AbstractS3ATestBase { /** * Parameter: should the stores be guarded? */ - protected final boolean s3guard; + private final boolean s3guard; /** * Parameter: should directory markers be retained? */ - protected final boolean keepMarkers; + private final boolean keepMarkers; /** * Is this an auth mode test run? */ - protected final boolean authoritative; + private final boolean authoritative; /** probe states calculated from the configuration options. */ - boolean isGuarded; + private boolean isGuarded; - boolean isRaw; + private boolean isRaw; - boolean isAuthoritative; + private boolean isAuthoritative; - boolean isNonAuth; + private boolean isNonAuth; - boolean isKeeping; + private boolean isKeeping; - boolean isDeleting; + private boolean isDeleting; private OperationCostValidator costValidator; @@ -134,7 +134,7 @@ public void setup() throws Exception { isAuthoritative = isGuarded && authoritative; isNonAuth = isGuarded && !authoritative; - isKeeping = isKeepingMarkers (); + isKeeping = isKeepingMarkers(); isDeleting = !isKeeping; @@ -301,7 +301,8 @@ protected Path create(Path path, boolean overwrite, public String execRename(final Path source, final Path dest) throws IOException { getFileSystem().rename(source, dest); - return String.format("rename(%s, %s): %s", dest, source, getMetricSummary()); + return String.format("rename(%s, %s): %s", + dest, source, getMetricSummary()); } /** @@ -544,84 +545,84 @@ protected void isFile(Path path, boolean expected, /** * A metric diff which must always hold. - * @param Statistic metric source + * @param stat metric source * @param expected expected value. * @return the diff. */ protected OperationCostValidator.ExpectedProbe always( - final Statistic Statistic, final int expected) { - return probe(Statistic, expected); + final Statistic stat, final int expected) { + return probe(stat, expected); } /** * A metric diff which must hold when the fs is unguarded. - * @param Statistic metric source + * @param stat metric source * @param expected expected value. * @return the diff. */ protected OperationCostValidator.ExpectedProbe raw( - final Statistic Statistic, final int expected) { - return probe(isRaw(), Statistic, expected); + final Statistic stat, final int expected) { + return probe(isRaw(), stat, expected); } /** * A metric diff which must hold when the fs is guarded. - * @param Statistic metric source + * @param stat metric source * @param expected expected value. * @return the diff. */ protected OperationCostValidator.ExpectedProbe guarded( - final Statistic Statistic, + final Statistic stat, final int expected) { - return probe(isGuarded(), Statistic, expected); + return probe(isGuarded(), stat, expected); } /** * A metric diff which must hold when the fs is guarded + authoritative. - * @param Statistic metric source + * @param stat metric source * @param expected expected value. * @return the diff. */ protected OperationCostValidator.ExpectedProbe authoritative( - final Statistic Statistic, + final Statistic stat, final int expected) { - return probe(isAuthoritative(), Statistic, expected); + return probe(isAuthoritative(), stat, expected); } /** * A metric diff which must hold when the fs is guarded + authoritative. - * @param Statistic metric source + * @param stat metric source * @param expected expected value. * @return the diff. */ protected OperationCostValidator.ExpectedProbe nonauth( - final Statistic Statistic, + final Statistic stat, final int expected) { - return probe(isNonAuth(), Statistic, expected); + return probe(isNonAuth(), stat, expected); } /** * A metric diff which must hold when the fs is keeping markers - * @param Statistic metric source + * @param stat metric source * @param expected expected value. * @return the diff. */ protected OperationCostValidator.ExpectedProbe keeping( - final Statistic Statistic, + final Statistic stat, final int expected) { - return probe(isKeepingMarkers(), Statistic, expected); + return probe(isKeepingMarkers(), stat, expected); } /** * A metric diff which must hold when the fs is keeping markers - * @param Statistic metric source + * @param stat metric source * @param expected expected value. * @return the diff. */ protected OperationCostValidator.ExpectedProbe deleting( - final Statistic Statistic, + final Statistic stat, final int expected) { - return probe(isDeleting(), Statistic, expected); + return probe(isDeleting(), stat, expected); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/HeadListCosts.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/HeadListCosts.java index 5ab94f150e866..db3d2ae3974ff 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/HeadListCosts.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/HeadListCosts.java @@ -21,7 +21,7 @@ /** * Declaration of the costs of head and list calls for various FS IO operations. */ -public class HeadListCosts { +public final class HeadListCosts { /** Head costs for getFileStatus() directory probe: {@value}. */ public static final int FILESTATUS_DIR_PROBE_H = 0; @@ -120,4 +120,6 @@ public class HeadListCosts { */ public static final int CREATE_FILE_NO_OVERWRITE_L = FILESTATUS_DIR_PROBE_L; + private HeadListCosts() { + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java index 619f4c0395a73..39d58bcf72780 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java @@ -43,7 +43,6 @@ import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; -import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.S3AFileSystem; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; @@ -383,10 +382,10 @@ private void containsFileUnderMarkerOnly(final List statuses) { } /** - * Expect the list of status to match that of the paths - * @param statuses - * @param paths - * @param + * Expect the list of status to match that of the paths. + * @param statuses status list + * @param paths ordered varargs list of paths + * @param type of status */ private void containsStatusOfPaths( List statuses, Path... paths) { @@ -417,7 +416,7 @@ private void assertName(final Path path, final FileStatus stat) { } private List - toList(RemoteIterator status) throws IOException { + toList(RemoteIterator status) throws IOException { List l = new ArrayList<>(); while (status.hasNext()) { @@ -428,7 +427,7 @@ private void assertName(final Path path, final FileStatus stat) { } private List - toList(T[] status) throws IOException { + toList(T[] status) throws IOException { List l = Arrays.asList(status); return dump(l); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java index 72dece4951851..fba6aef0c2915 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java @@ -124,9 +124,9 @@ public void testDirMarkersSubdir() throws Throwable { Path subDir = new Path(methodPath(), "1/2/3/4/5/6"); // one dir created, possibly a parent removed verifyMetrics(() -> { - mkdirs(subDir); - return "after mkdir(subDir) " + getMetricSummary(); - }, + mkdirs(subDir); + return "after mkdir(subDir) " + getMetricSummary(); + }, always(DIRECTORIES_CREATED, 1), always(DIRECTORIES_DELETED, 0), keeping(OBJECT_DELETE_REQUESTS, 0), @@ -148,9 +148,9 @@ public void testDirMarkersFileCreation() throws Throwable { // unless markers are being kept verifyMetrics(() -> { - file(new Path(srcDir, "source.txt")); - return "after touch(fs, srcFilePath) " + getMetricSummary(); - }, + file(new Path(srcDir, "source.txt")); + return "after touch(fs, srcFilePath) " + getMetricSummary(); + }, always(DIRECTORIES_CREATED, 0), always(DIRECTORIES_DELETED, 0), // keeping: no delete operations. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java index e752187589bad..32821915070ce 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java @@ -155,11 +155,10 @@ public void testCostOfRootRename() throws Throwable { Path src = file(new Path("/src-" + uuid)); Path dest = new Path("/dest-" + uuid); try { - verifyMetrics(() -> { - fs.rename(src, dest); - return "after fs.rename(/src,/dest) " + getMetricSummary(); - }, + fs.rename(src, dest); + return "after fs.rename(/src,/dest) " + getMetricSummary(); + }, // TWO HEAD for exists, one for source MD in copy rawHeadList(RENAME_SINGLE_FILE_RENAME_H, GETFILESTATUS_FNFE_L), diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java index 5523b3de271e7..cd5dd79eeb54d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java @@ -173,19 +173,19 @@ private static Path mkpath(Path base, final String name) { */ private static class CreatedPaths { - Path base; + private Path base; - List files = new ArrayList<>(); + private List files = new ArrayList<>(); - List dirs = new ArrayList<>(); + private List dirs = new ArrayList<>(); - List emptyDirs = new ArrayList<>(); + private List emptyDirs = new ArrayList<>(); - List filesUnderBase = new ArrayList<>(); + private List filesUnderBase = new ArrayList<>(); - List dirsUnderBase = new ArrayList<>(); + private List dirsUnderBase = new ArrayList<>(); - List emptyDirsUnderBase = new ArrayList<>(); + private List emptyDirsUnderBase = new ArrayList<>(); private Path mkdir(FileSystem fs, String name) @@ -226,7 +226,8 @@ private CreatedPaths createPaths(FileSystem fs, Path base) throws IOException { CreatedPaths r = new CreatedPaths(); r.base = base; - // the directories under which we will create files, so expect to have markers + // the directories under which we will create files, + // so expect to have markers r.mkdir(fs, ""); r.mkdir(fs, "dir1"); r.mkdir(fs, "dir2"); @@ -359,21 +360,21 @@ public void testAuthPathIsMixed() throws Throwable { Path source = new Path(base, "source"); Path dest = new Path(base, "dest"); Path dir2 = new Path(source, "dir2"); - S3AFileSystem mixedFS = createFS(DIRECTORY_MARKER_POLICY_AUTHORITATIVE, + S3AFileSystem mixedFSDir2 = createFS(DIRECTORY_MARKER_POLICY_AUTHORITATIVE, dir2.toUri().toString()); // some of these paths will retain markers, some will not - CreatedPaths createdPaths = createPaths(mixedFS, source); + CreatedPaths createdPaths = createPaths(mixedFSDir2, source); // markers are only under dir2 - markerTool(mixedFS, mkpath(source, "dir1"), false, 0); - markerTool(mixedFS, source, false, expectMarkersUnderDir2); + markerTool(mixedFSDir2, mkpath(source, "dir1"), false, 0); + markerTool(mixedFSDir2, source, false, expectMarkersUnderDir2); // if we now rename, all will be good - mixedFS.rename(source, dest); + mixedFSDir2.rename(source, dest); assertIsDirectory(dest); // there are no markers - MarkerTool.ScanResult scanResult = markerTool(mixedFS, dest, false, 0); + MarkerTool.ScanResult scanResult = markerTool(mixedFSDir2, dest, false, 0); // there are exactly the files we want Assertions.assertThat(scanResult) .describedAs("Scan result %s", scanResult) From 5e2d0250f7b7785d9ca487e7de268ea35973bb16 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 28 Jul 2020 21:39:52 +0100 Subject: [PATCH 09/20] HADOOP-13230 tests, docs and tuning marker tool Initial docs including lots of background and warning messages. Marker tool CLI is being tuned by docs/experiments. Change-Id: Ic6740437d3c5fbf6e1a27f3433cec138a9bef7ec --- .../org/apache/hadoop/fs/s3a/Constants.java | 14 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 5 + .../hadoop/fs/s3a/impl/DirectoryPolicy.java | 24 +- .../fs/s3a/impl/DirectoryPolicyImpl.java | 5 + .../hadoop/fs/s3a/tools/MarkerTool.java | 71 ++--- .../tools/hadoop-aws/directory_markers.md | 255 ++++++++++++++++++ .../s3a/impl/TestDirectoryMarkerPolicy.java | 13 +- .../hadoop/fs/s3a/tools/ITestMarkerTool.java | 138 +++++++--- 8 files changed, 439 insertions(+), 86 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md 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 a738fe2465880..d7cd2ebadf98a 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 @@ -961,21 +961,21 @@ private Constants() { * at the risk of backwards compatibility. */ public static final String DIRECTORY_MARKER_POLICY = - "fs.s3a.directory.markers"; + "fs.s3a.directory.marker.retention"; /** - * Retain directory markers. + * Delete directory markers. This is the backwards compatible option. * Value: {@value}. */ - public static final String DIRECTORY_MARKER_POLICY_KEEP = - "keep"; + public static final String DIRECTORY_MARKER_POLICY_DELETE = + "delete"; /** - * Delete directory markers. This is the backwards compatible option. + * Retain directory markers. * Value: {@value}. */ - public static final String DIRECTORY_MARKER_POLICY_DELETE = - "delete"; + public static final String DIRECTORY_MARKER_POLICY_KEEP = + "keep"; /** * Retain directory markers in authoritative directory trees only. 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 96ab61fc22cf4..507886455473a 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 @@ -4019,6 +4019,10 @@ public long getDefaultBlockSize() { return getConf().getLongBytes(FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE); } + public DirectoryPolicy getDirectoryPolicy() { + return directoryPolicy; + } + @Override public String toString() { final StringBuilder sb = new StringBuilder( @@ -4057,6 +4061,7 @@ public String toString() { sb.append(", credentials=").append(credentials); sb.append(", delegation tokens=") .append(delegationTokens.map(Objects::toString).orElse("disabled")); + sb.append(", ").append(directoryPolicy); sb.append(", statistics {") .append(statistics) .append("}"); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java index e81577f5486d2..4228b128faebe 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java @@ -20,6 +20,9 @@ import org.apache.hadoop.fs.Path; +/** + * Interface for Directory Marker policies to implement. + */ public interface DirectoryPolicy { /** @@ -29,23 +32,34 @@ public interface DirectoryPolicy { */ boolean keepDirectoryMarkers(Path path); + /** + * Describe the policy for marker tools and logs. + * @return description of the current policy. + */ + String describe(); + /** * Supported retention policies. */ enum MarkerPolicy { + + /** + * Delete markers. + *

+ * This is the classic S3A policy, + */ + Delete, + /** * Keep markers. + *

* This is Not backwards compatible. */ Keep, - /** - * Delete markers. - * This is what has been done since S3A was released. */ - Delete, - /** * Keep markers in authoritative paths only. + *

* This is Not backwards compatible within the * auth paths, but is outside these. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java index e3170f4f10eb3..99318d08dcb69 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java @@ -107,6 +107,11 @@ public boolean keepDirectoryMarkers(final Path path) { } } + @Override + public String describe() { + return markerPolicy.toString(); + } + @Override public String toString() { final StringBuilder sb = new StringBuilder( diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index 134f0195d9861..1a49cabe5ad85 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -52,6 +52,8 @@ import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.util.OperationDuration; +import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE; +import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE_DEFAULT; import static org.apache.hadoop.fs.s3a.Invoker.once; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_SUCCESS; @@ -76,30 +78,46 @@ public final class MarkerTool extends S3GuardTool { public static final String NAME = "markers"; public static final String PURPOSE = - "view and manipulate S3 directory markers"; + "View and manipulate S3 directory markers"; private static final String USAGE = NAME - + " [-verbose] [-expected ]" - + " (audit || report || clean)" -// + " [-out ]" + " [-" + VERBOSE + "]" + + " (audit | report | clean)" + " \n" + "\t" + PURPOSE + "\n\n"; - public static final String OPT_EXPECTED = "expected"; - + /** + * Audit sub-command: {@value}. + */ public static final String AUDIT = "audit"; + /** + * Clean Sub-command: {@value}. + */ public static final String CLEAN = "clean"; + /** + * Report Sub-command: {@value}. + */ public static final String REPORT = "report"; - public static final String OPT_OUTPUT = "output"; - + /** + * Verbose option: {@value}. + */ public static final String OPT_VERBOSE = "verbose"; + /** + * Error text when too few arguments are found. + */ + @VisibleForTesting static final String TOO_FEW_ARGUMENTS = "Too few arguments"; + /** + * Constant to use when there is no limit on the number of + * markers expected. + */ + private static final int UNLIMITED = -1; + /** Will be overridden in run(), but during tests needs to avoid NPEs. */ private PrintStream out = System.out; @@ -114,11 +132,7 @@ public final class MarkerTool extends S3GuardTool { private StoreContext storeContext; public MarkerTool(final Configuration conf) { - super(conf, - OPT_VERBOSE - ); - getCommandFormat().addOptionWithValue(OPT_EXPECTED); -// getCommandFormat().addOptionWithValue(OPT_OUTPUT); + super(conf, OPT_VERBOSE); } @Override @@ -138,11 +152,6 @@ public void resetBindings() { operationCallbacks = null; } - @Override - public void close() throws IOException { - super.close(); - } - @Override public int run(final String[] args, final PrintStream stream) throws ExitUtil.ExitException, Exception { @@ -162,21 +171,22 @@ public int run(final String[] args, final PrintStream stream) CommandFormat commandFormat = getCommandFormat(); verbose = commandFormat.getOpt(VERBOSE); - expected = 0; + expected = UNLIMITED; // argument 0 is the action String action = parsedArgs.get(0); switch (action) { case AUDIT: + // audit. no purge; fail if any marker is found purge = false; expected = 0; break; case CLEAN: + // clean -purge the markers purge = true; - expected = -1; break; case REPORT: + // report -no purge purge = false; - expected = -1; break; default: errorln(getUsage()); @@ -214,6 +224,9 @@ ScanResult execute( storeContext = fs.createStoreContext(); operationCallbacks = fs.getOperationCallbacks(); + println(out, "Directory Marker Policy is %s", + fs.getDirectoryPolicy().describe()); + ScanResult result = once("action", path.toString(), () -> scan(path, doPurge, expectedMarkerCount)); if (verbose) { @@ -311,15 +324,19 @@ private ScanResult scan( } if (size > expectedMarkerCount) { // failure - println(out, "Expected %d marker%s", expectedMarkerCount, suffix(size)); + if (expectedMarkerCount > UNLIMITED) { + println(out, "Expected %d marker%s", expectedMarkerCount, suffix(size)); + } + println(out, "Surplus markers were found -failing audit"); + result.exitCode = EXIT_NOT_ACCEPTABLE; return result; } if (doPurge) { int deletePageSize = storeContext.getConfiguration() - .getInt(Constants.BULK_DELETE_PAGE_SIZE, - Constants.BULK_DELETE_PAGE_SIZE_DEFAULT); + .getInt(BULK_DELETE_PAGE_SIZE, + BULK_DELETE_PAGE_SIZE_DEFAULT); result.purgeSummary = purgeMarkers(tracker, deletePageSize); } @@ -336,10 +353,6 @@ private String suffix(final int size) { return size == 1 ? "" : "s"; } - public static Logger getLOG() { - return LOG; - } - /** * Scan a directory tree. * @param path path to scan @@ -373,7 +386,6 @@ private void scanDirectoryTree(final Path path, } - /** * Result of a call of {@link #purgeMarkers(DirMarkerTracker, int)}; * included in {@link ScanResult} so must share visibility. @@ -462,7 +474,6 @@ private MarkerPurgeSummary purgeMarkers(DirMarkerTracker tracker, List page = markerKeys.subList(start, end); List undeleted = new ArrayList<>(); - // currently no attempt at doing this in pages. OperationDuration duration = new OperationDuration(); operationCallbacks.removeKeys(page, true, undeleted, null, false); duration.finished(); diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md new file mode 100644 index 0000000000000..b3d8b94f7e9f7 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md @@ -0,0 +1,255 @@ + + +# Altering the S3A Directory Marker Retention Policy + +## Critical: this is not backwards compatible! + +This document shows how the performance of S3 IO, especially applications +writing many files (hive) or working with versioned S3 buckets can +increase performance by changing the S3A directory marker retention policy. + +Changing the policy from the default value, `"delete"` _is not backwards compatible_. + +Older versions of Hadoop will + +Versions of Hadoop which are incompatible with other marker retention policies + + +| Branch | Incompatible releases | Future Fix Planned? | +|---------|-----------------------|--------------------| +| Hadoop 2.x | All releases | WONTFIX | +| Hadoop 3.0 | All releases | WONTFIX | +| Hadoop 3.1 | All releases | Yes | +| Hadoop 3.2 | All releases | Yes | +| Hadoop 3.3 | 3.3.0 | Yes | + +External Hadoop-based applications should also be assumed to be incompatible +unless otherwise stated/known. + +It is only safe change the directory marker policy if the following + conditions are met: + +1. You know exactly which applications are writing to and reading from + (including backing up) an S3 bucket. +2. You know all applications which read data from the bucket are listed as compatible. + +### Applications backing up data MUST ALSO be based on compatible products. + +It is not enough to have a version of Apache Hadoop which is compatible, any +application which backs up an S3 bucket or copies elsewhere must have an S3 +connector which is compatible. For the Hadoop codebase, that means that if +distcp is used, it _must_ be from a compatible hadoop version. + +### How will incompatible applications/versions fail? + +Applications using an incompatible version of the S3A connector will mistake directories +containing data for empty directories. This means that + +* Listing directories/directory trees may exclude files which exist. +* Queries across the data will miss data files. +* Renaming a directory to a new location may exclude files underneath. + +Now that this is all clear, let's explain the problem. + + +## Background: Directory Markers: what and why? + +Amazon S3 is not a filesystem, it is an object store. + +The S3A connector not only provides a hadoop-compatible API to interact with +data in S3, it tries to maintain the filesystem metaphor. + +One key aspect of the metaphor of a file system is "directories" + +#### The directory concept + +In normal Unix-style filesystems, the "filesystem" is really a "directory and +file tree" in which files are always stored in "directories" + + +* A directory may contain 0 or more files. +* A directory may contain 0 or more directories "subdirectories" +* At the base of a filesystem is the "root directory" +* All files MUST be in a directory "the parent directory" +* All directories other than the root directory must be in another directory. +* If a directory contains no files or directories, it is "empty" +* When a directory is _listed_, all files and directories in it are enumerated and returned to the caller + + +The S3A connector mocks this entire metaphor by grouping all objects which have +the same prefix as if they are in the same directory tree. + +If there are two objects Cand `a/b/file2` then S3A pretends that there is a +directory `/a/b` containing two files `file1` and `file2`. + +The directory itself does not exist. + +There's a bit of a complication here. + +#### What does `mkdirs()` do? + +1. In HDFS and other "real" filesystems, when mkdirs() is invoked on a path +whose parents are all directories, then an _empty directory_ is created. + +1. This directory can be probed for "it exists" and listed (an empty list is +returned) + +1. Files and other directories can be created in it. + + +Lots of code contains a big assumption here: after you create a directory it +exists. They also assume that after files in a directory are delted, the +directory still exists. + +Given filesystem mimics directories just by aggregating objects which share a +prefix, how can you have empty directories? + + +The original Hadoop `s3n://` connector created a Directory Marker -any path ending +in `_$folder$` was considered to be a sign that a directory existed. A call to +`mkdir(s3n://bucket/a/b)` would create a new marker object `a/b_$folder$` . + +The S3A also has directory markers, but it just appends a / to the directory +name, so `mkdir(s3a://bucket/a/b)` would create a new marker object `a/b/` . +When a file is created under a path, the directory marker is deleted. And when a +file is deleted, if it was the last file in the directory, the marker is +recreated. + +And, historically, When a path is listed, if a marker to that path is found, *it +has been interpreted as an empty directory.* + +## The Problem + +Creating, deleting and the listing directory markers adds overhead and can slow +down applications. + +Whenever a file is created we have to delete any marker which could exist in +parent directory _or any parent paths_. Rather than do a sequence of probes for +parent markers existing, the connector issues a single request to S3 to delete +all parents. For example, if a file `/a/b/file1` is created, a `DELETE` request +containing the keys `/a/` and `/a/b/` is issued. If no marker exists, this is +nominally a no-op. + +When a file is deleted, a check for the parent directory continuing to exist +(i.e. are there sibling files/directories?), and if not a marker is created. + +This all works well and has worked well for many years. + +However, it turns out to have some scale problems, especially from the delete +call made whenever a file is created. + +1. The number of the objects listed in each request is that of the number of +parent directories: deeper trees create longer requests. + +1. Every single object listed in the delete request is considered to be a write +operation. + +1. In versioned S3 buckets, tombstone markers are added to the S3 indices even +if no object was deleted. + +1. There's also the overhead of actually issuing the request and awaiting the +response. + +Issue #2 has turned out to cause significant problems on some interactions with +large hive tables: + +Because each object listed in a DELETE call is treated as one operation, and +there is (as of summer 2020) a limit of +[3500 write requests/second in a directory tree](https://docs.aws.amazon.com/AmazonS3/latest/dev/optimizing-performance.html), +when writing many files to a deep directory tree, it is the delete calls which +create throttling problems. +For a symptom of this, see HADOOP-16829, +[Large DeleteObject requests are their own Thundering Herd](https://issues.apache.org/jira/browse/HADOOP-16823). + +The tombstone markers have follow-on consequences -it makes listings slower. +This can have adverse effects on those large directories, again. + +## Solutions + +### Presto: there are no directories + +In the Presto S3 connectors: `mkdirs()` is a no-op. Instead, whenever you list +any path which isn't an object or a prefix of one more more objects, you get an +empty listing. That is:; by default, every path is an empty directory. + +Provided no code probes for a directory existing and fails if it is there, this +is very efficient. That's a big requirement however, -one Presto can pull off +because they know how their file uses + + +### Hadoop 3.3.1+: marker deletion is now optional + + +## Controlling marker retention with `fs.s3a.directory.marker.retention` + +There is now an option `fs.s3a.directory.marker.retention` which controls how +markers are managed when new files are created + +*Default* `delete`: a request is issued to delete any parental directory markers +whenever a file or directory is created. + +*New* `keep`: No delete request is issued. Any directory markers which exist are +not deleted. This is *not* backwards compatible + +*New* `authoritative`: directory markers are deleted _except for files created +in "authoritative" directories_. +This is backwards compatible _outside +authoritative directories_. +Until now, the notion of an "authoritative" +directory has only been used as a performance optimization for deployments +where it is known that all Applications are using the same S3Guard metastore +when writing and reading data. In such a deployment, if it is also known that +all applications are using a compatible version of the s3a connector, then they +can switch to the higher-performance mode for those specific directories. + + +Only the default setting, `fs.s3a.directory.marker.retention = delete` is compatible with +existing Hadoop releases. + + +## The marker tool: 'hadoop s3guard marker' + +The marker tool aims to help migration by scanning/auditing directory trees +for surplus markers, and for optionally deleting them. +Leaf-node markers for empty directories are not considered surplus and +will be retained. + +Syntax + +``` +> bin/hadoop s3guard markers +markers [-verbose] [-expected ] (audit || report || clean) + view and manipulate S3 directory markers +``` + +### `markers report` +Scan the path and simply report on the markers found. + +### `markers audit` + +Audit the path and fail if any markers were found. + +### `markers clean` + +The `markers clean` command will clean the directory tree + +``` +> hadoop s3guard markers clean s3a://ireland-bucket/ + +2020-07-28 18:58:36,612 [main] INFO tools.MarkerTool (DurationInfo.java:(77)) - Starting: marker scan s3a://ireland-bucket/ +2020-07-28 18:58:37,516 [main] INFO tools.MarkerTool (DurationInfo.java:close(98)) - marker scan s3a://ireland-bucket/: duration 0:00.906s +No surplus directory markers were found under s3a://ireland-bucket/ +``` + diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java index 9bfe0f58090de..f0a673254af80 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java @@ -41,17 +41,17 @@ public static Collection data() { return Arrays.asList(new Object[][]{ { DirectoryPolicy.MarkerPolicy.Delete, - failIfInvoked, + FAIL_IF_INVOKED, false, false }, { DirectoryPolicy.MarkerPolicy.Keep, - failIfInvoked, + FAIL_IF_INVOKED, true, true }, { DirectoryPolicy.MarkerPolicy.Authoritative, - authPathOnly, + AUTH_PATH_ONLY, false, true }}); } @@ -85,6 +85,13 @@ private DirectoryPolicyImpl retention( return new DirectoryPolicyImpl(c, authoritativeness); } + private static final Predicate AUTH_PATH_ONLY = + (p) -> p.toUri().getPath().startsWith("/auth/"); + + private static final Predicate FAIL_IF_INVOKED = (p) -> { + throw new RuntimeException("failed"); + }; + private final Path nonAuthPath = new Path("s3a://bucket/nonauth/data"); private final Path authPath = new Path("s3a://bucket/auth/data1"); private final Path deepAuth = new Path("s3a://bucket/auth/d1/d2/data2"); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java index cd5dd79eeb54d..7a0fd917a36b3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java @@ -34,13 +34,18 @@ import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.util.ExitUtil; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; +import static org.apache.hadoop.fs.s3a.tools.MarkerTool.*; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE; +import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_USAGE; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test the marker tool and use it to compare the behavior @@ -51,9 +56,16 @@ public class ITestMarkerTool extends AbstractS3ATestBase { private static final Logger LOG = LoggerFactory.getLogger(ITestMarkerTool.class); + /** the -verbose option. */ + private static final String V = "-" + OPT_VERBOSE; + /** FS which keeps markers. */ private S3AFileSystem keepingFS; + /** FS which deletes markers. */ + private S3AFileSystem deletingFS; + + /** FS which mixes markers; only created in some tests. */ private S3AFileSystem mixedFS; private int expectedFiles; @@ -87,27 +99,14 @@ protected Configuration createConfiguration() { public void setup() throws Exception { super.setup(); setKeepingFS(createFS(DIRECTORY_MARKER_POLICY_KEEP, null)); + setDeletingFS(createFS(DIRECTORY_MARKER_POLICY_DELETE, null)); } @Override public void teardown() throws Exception { - IOUtils.cleanupWithLogger(LOG, getKeepingFS(), getMixedFS()); super.teardown(); - } + IOUtils.cleanupWithLogger(LOG, getKeepingFS(), getMixedFS(), getDeletingFS()); - /** - * Create and initialize a new filesystem. - * This filesystem MUST be closed in test teardown. - * @param uri FS URI - * @param config config. - * @return new instance - * @throws IOException failure - */ - private S3AFileSystem createFS(final URI uri, final Configuration config) - throws IOException { - S3AFileSystem fs2 = new S3AFileSystem(); - fs2.initialize(uri, config); - return fs2; } /** @@ -126,23 +125,33 @@ private S3AFileSystem createFS(String markerPolicy, removeBucketOverrides(bucketName, conf, DIRECTORY_MARKER_POLICY, S3_METADATA_STORE_IMPL, + BULK_DELETE_PAGE_SIZE, AUTHORITATIVE_PATH); if (authPath != null) { conf.set(AUTHORITATIVE_PATH, authPath); } + // Use a very small page size to force the paging + // code to be tested. + conf.setInt(BULK_DELETE_PAGE_SIZE, 4); conf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL); conf.set(DIRECTORY_MARKER_POLICY, markerPolicy); - final S3AFileSystem newFS = createFS(testFSUri, conf); - return newFS; + S3AFileSystem fs2 = new S3AFileSystem(); + fs2.initialize(testFSUri, conf); + LOG.info("created new filesystem with policy {} and auth path {}: {}", + markerPolicy, authPath, fs2); + return fs2; } /** * FS which deletes markers. */ - private S3AFileSystem getLegacyFS() { - return getFileSystem(); + public S3AFileSystem getDeletingFS() { + return deletingFS; } + public void setDeletingFS(final S3AFileSystem deletingFS) { + this.deletingFS = deletingFS; + } /** * FS which keeps markers. @@ -277,11 +286,27 @@ private MarkerTool.ScanResult markerTool( return result; } + void verifyRenamed(final Path dest, + final CreatedPaths createdPaths) throws IOException { + // all leaf directories exist + for (String p : createdPaths.emptyDirsUnderBase) { + assertIsDirectory(mkpath(dest, p)); + } + // non-empty dirs + for (String p : createdPaths.dirsUnderBase) { + assertIsDirectory(mkpath(dest, p)); + } + // all files exist + for (String p : createdPaths.filesUnderBase) { + assertIsFile(mkpath(dest, p)); + } + } + @Test public void testAuditPruneMarkersLegacyDir() throws Throwable { - CreatedPaths createdPaths = createPaths(getLegacyFS(), methodPath()); - markerTool(getLegacyFS(), createdPaths.base, false, 0); - markerTool(getLegacyFS(), createdPaths.base, true, 0); + CreatedPaths createdPaths = createPaths(getDeletingFS(), methodPath()); + markerTool(getDeletingFS(), createdPaths.base, false, 0); + markerTool(getDeletingFS(), createdPaths.base, true, 0); } @Test @@ -290,7 +315,7 @@ public void testAuditPruneMarkersKeepingDir() throws Throwable { // audit will find the expected entries int expectedMarkerCount = createdPaths.dirs.size(); - S3AFileSystem fs = getLegacyFS(); + S3AFileSystem fs = getDeletingFS(); markerTool(EXIT_NOT_ACCEPTABLE, fs, createdPaths.base, false, 0); @@ -309,8 +334,8 @@ public void testAuditPruneMarkersKeepingDir() throws Throwable { @Test public void testRenameKeepingFS() throws Throwable { - describe( - "Rename with the keeping FS -verify that no markers exist at far end"); + describe("Rename with the keeping FS -verify that no markers" + + " exist at far end"); Path base = methodPath(); Path source = new Path(base, "source"); Path dest = new Path(base, "dest"); @@ -333,22 +358,6 @@ public void testRenameKeepingFS() throws Throwable { } - void verifyRenamed(final Path dest, - final CreatedPaths createdPaths) throws IOException { - // all leaf directories exist - for (String p : createdPaths.emptyDirsUnderBase) { - assertIsDirectory(mkpath(dest, p)); - } - // non-empty dirs - for (String p : createdPaths.dirsUnderBase) { - assertIsDirectory(mkpath(dest, p)); - } - // all files exist - for (String p : createdPaths.filesUnderBase) { - assertIsFile(mkpath(dest, p)); - } - } - /** * Create a FS where only dir2 in the source tree keeps markers; * verify all is good. @@ -397,4 +406,51 @@ private static void assertMarkersDeleted(int expected, .extracting(f -> f.getMarkersDeleted()) .isEqualTo(expected); } + + @Test + public void testRunNoArgs() throws Throwable { + runToFailure(EXIT_USAGE, NAME); + } + + @Test + public void testRunVerboseAudit() throws Throwable { + CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); + run(NAME, + V, + REPORT, + createdPaths.base.toString()); + run(NAME, + V, + AUDIT, + createdPaths.base.toString()); + } + + /** + * Run a S3GuardTool command from a varags list and the + * configuration returned by {@code getConfiguration()}. + * @param args argument list + * @return the return code + * @throws Exception any exception + */ + protected int run(String... args) throws Exception { + return S3GuardTool.run(getConfiguration(), args); + } + + /** + * Run a S3GuardTool command from a varags list, catch any raised + * ExitException and verify the status code matches that expected. + * @param status expected status code of the exception + * @param args argument list + * @throws Exception any exception + */ + protected void runToFailure(int status, String... args) + throws Exception { + ExitUtil.ExitException ex = + intercept(ExitUtil.ExitException.class, + () -> run(args)); + if (ex.status != status) { + throw ex; + } + } + } From 85c92c10c21eca2639933db8ccdc47e56193e6f5 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 29 Jul 2020 18:14:03 +0100 Subject: [PATCH 10/20] HADOOP-13230. Marker policy tests. Fixed bug: on small delete page sizes, markers were being deleted before being copied. Or should we just create them? Issue: when I rename a file into a dir, and dir has a marker, then dir marker is not deleted, even when policy == deleting. Change-Id: Iae17d98a5b6e34c52503830e9154e8db8bd65848 --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 97 +++- .../hadoop/fs/s3a/impl/DirMarkerTracker.java | 8 + .../hadoop/fs/s3a/impl/DirectoryPolicy.java | 6 + .../fs/s3a/impl/DirectoryPolicyImpl.java | 6 + .../hadoop/fs/s3a/impl/RenameOperation.java | 42 +- .../fs/s3a/s3guard/NullMetadataStore.java | 2 +- .../hadoop/fs/s3a/tools/MarkerTool.java | 35 +- .../ITestDirectoryMarkerListing.java | 420 +++++++++++----- .../hadoop/fs/s3a/tools/ITestMarkerTool.java | 472 +++++++++++------- 9 files changed, 774 insertions(+), 314 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 507886455473a..fc750d74b3626 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 @@ -1614,7 +1614,7 @@ public void finishRename(final Path sourceRenamed, final Path destCreated) Path destParent = destCreated.getParent(); if (!sourceRenamed.getParent().equals(destParent)) { LOG.debug("source & dest parents are different; fix up dir markers"); - if (!keepDirectoryMarkers(destCreated)) { + if (!keepDirectoryMarkers(destParent)) { deleteUnnecessaryFakeDirectories(destParent, null); } maybeCreateFakeParentDirectory(sourceRenamed); @@ -4019,8 +4019,12 @@ public long getDefaultBlockSize() { return getConf().getLongBytes(FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE); } - public DirectoryPolicy getDirectoryPolicy() { - return directoryPolicy; + /** + * Get the directory marker policy of this filesystem. + * @return the marker policy. + */ + public DirectoryPolicy.MarkerPolicy getDirectoryMarkerPolicy() { + return directoryPolicy.getMarkerPolicy(); } @Override @@ -4396,6 +4400,93 @@ private RemoteIterator innerListFiles( } } + /** + * List files under a path assuming the path to be a directory. + * @param path input path. + * @param recursive recursive listing? + * @param acceptor file status filter + * @param collectTombstones should tombstones be collected from S3Guard? + * @param forceNonAuthoritativeMS forces metadata store to act like non + * authoritative. This is useful when + * listFiles output is used by import tool. + * @return an iterator over listing. + * @throws IOException any exception. + */ + private RemoteIterator getListFilesAssumingDir( + Path path, + boolean recursive, Listing.FileStatusAcceptor acceptor, + boolean collectTombstones, + boolean forceNonAuthoritativeMS) throws IOException { + + String key = maybeAddTrailingSlash(pathToKey(path)); + String delimiter = recursive ? null : "/"; + if (recursive) { + LOG.debug("Recursive list of all entries under {}", key); + } else { + LOG.debug("Requesting all entries under {} with delimiter '{}'", + key, delimiter); + } + final RemoteIterator cachedFilesIterator; + final Set tombstones; + boolean allowAuthoritative = allowAuthoritative(path); + if (recursive) { + final PathMetadata pm = metadataStore.get(path, true); + if (pm != null) { + if (pm.isDeleted()) { + OffsetDateTime deletedAt = OffsetDateTime + .ofInstant(Instant.ofEpochMilli( + pm.getFileStatus().getModificationTime()), + ZoneOffset.UTC); + throw new FileNotFoundException("Path " + path + " is recorded as " + + "deleted by S3Guard at " + deletedAt); + } + } + MetadataStoreListFilesIterator metadataStoreListFilesIterator = + new MetadataStoreListFilesIterator(metadataStore, pm, + allowAuthoritative); + tombstones = metadataStoreListFilesIterator.listTombstones(); + // if all of the below is true + // - authoritative access is allowed for this metadatastore + // for this directory, + // - all the directory listings are authoritative on the client + // - the caller does not force non-authoritative access + // return the listing without any further s3 access + if (!forceNonAuthoritativeMS && + allowAuthoritative && + metadataStoreListFilesIterator.isRecursivelyAuthoritative()) { + S3AFileStatus[] statuses = S3Guard.iteratorToStatuses( + metadataStoreListFilesIterator, tombstones); + cachedFilesIterator = listing.createProvidedFileStatusIterator( + statuses, ACCEPT_ALL, acceptor); + return listing.createLocatedFileStatusIterator(cachedFilesIterator); + } + cachedFilesIterator = metadataStoreListFilesIterator; + } else { + DirListingMetadata meta = + S3Guard.listChildrenWithTtl(metadataStore, path, ttlTimeProvider, + allowAuthoritative); + if (meta != null) { + tombstones = meta.listTombstones(); + } else { + tombstones = null; + } + cachedFilesIterator = listing.createProvidedFileStatusIterator( + S3Guard.dirMetaToStatuses(meta), ACCEPT_ALL, acceptor); + if (allowAuthoritative && meta != null && meta.isAuthoritative()) { + // metadata listing is authoritative, so return it directly + return listing.createLocatedFileStatusIterator(cachedFilesIterator); + } + } + return listing.createTombstoneReconcilingIterator( + listing.createLocatedFileStatusIterator( + listing.createFileStatusListingIterator(path, + createListObjectsRequest(key, delimiter), + ACCEPT_ALL, + acceptor, + cachedFilesIterator)), + collectTombstones ? tombstones : null); + } + /** * Override superclass so as to add statistic collection. * {@inheritDoc} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java index 0e5fbf0b39a87..11341e352cafd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java @@ -229,6 +229,14 @@ public S3ALocatedFileStatus getStatus() { return status; } + /** + * Get the version ID of the status object; may be null. + * @return a version ID, if known. + */ + public String getVersionId() { + return status.getVersionId(); + } + @Override public String toString() { return "Marker{" + diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java index 4228b128faebe..1038f890a787b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java @@ -32,6 +32,12 @@ public interface DirectoryPolicy { */ boolean keepDirectoryMarkers(Path path); + /** + * Get the marker policy. + * @return policy. + */ + MarkerPolicy getMarkerPolicy(); + /** * Describe the policy for marker tools and logs. * @return description of the current policy. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java index 99318d08dcb69..072988ba7fefa 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java @@ -107,6 +107,12 @@ public boolean keepDirectoryMarkers(final Path path) { } } + + @Override + public MarkerPolicy getMarkerPolicy() { + return markerPolicy; + } + @Override public String describe() { return markerPolicy.toString(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java index 08ed363bd7ddc..064435ac68b00 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java @@ -185,18 +185,40 @@ private void completeActiveCopies(String reason) throws IOException { /** * Queue an object for deletion. + *

+ * This object will be deleted when the next page of objects to delete + * is posted to S3. Therefore, the COPY must have finished + * before that deletion operation takes place. + * This is managed by + *
    + *
  1. + * The delete operation only being executed once all active + * copies have completed. + *
  2. + *
  3. + * Only queuing objects here whose copy operation has + * been submitted and so is in that thread pool. + *
  4. + *
+ * This method must only be called from the primary thread * @param path path to the object - * @param s * @param key key of the object. + * @param version object version. */ private void queueToDelete(Path path, String key, String version) { + LOG.debug("Queueing to delete {}", path); pathsToDelete.add(path); keysToDelete.add(new DeleteObjectsRequest.KeyVersion(key, version)); } /** * Queue a list of markers for deletion. + *

* no-op if the list is empty. + *

+ * See {@link #queueToDelete(Path, String, String)} for + * details on safe use of this method. + * * @param markersToDelete markers */ private void queueToDelete( @@ -366,8 +388,6 @@ protected void recursiveDirectoryRename() throws IOException { // the source object to copy as a path. Path childSourcePath = storeContext.keyToPath(key); - // mark for deletion on a successful copy. - queueToDelete(childSourcePath, key, child.getVersionId()); List markersToDelete; boolean isMarker = key.endsWith("/"); @@ -379,7 +399,7 @@ protected void recursiveDirectoryRename() throws IOException { childSourcePath, key, child); } else { // it is a file. - // note that it has been found -ths may find a list of parent + // note that it has been found -this may find a list of parent // markers which may now be deleted. markersToDelete = dirMarkerTracker.fileFound( childSourcePath, key, child); @@ -389,6 +409,8 @@ protected void recursiveDirectoryRename() throws IOException { dstKey + key.substring(srcKey.length()); Path childDestPath = storeContext.keyToPath(newDestKey); + // mark the source file for deletion on a successful copy. + queueToDelete(childSourcePath, key, child.getVersionId()); // now begin the single copy CompletableFuture copy = initiateCopy(child, key, childSourcePath, newDestKey, childDestPath); @@ -483,6 +505,7 @@ private OperationDuration copyEmptyDirectoryMarkers( dstKey + key.substring(srcKey.length()); Path childDestPath = storeContext.keyToPath(newDestKey); LOG.debug("copying dir marker from {} to {}", key, newDestKey); + activeCopies.add( initiateCopy( entry.getStatus(), @@ -559,6 +582,7 @@ private Path copySourceAndUpdateTracker( copyResult = callbacks.copyFile(srcKey, destinationKey, srcAttributes, readContext); } + if (objectRepresentsDirectory(srcKey, len)) { renameTracker.directoryMarkerCopied( sourceFile, @@ -597,6 +621,16 @@ private void removeSourceObjects( List undeletedObjects = new ArrayList<>(); try { // remove the keys + + // first list what is being deleted for the interest of anyone + // who is trying to debug while objects are no longer there. + if (LOG.isDebugEnabled()) { + LOG.debug("Initiating delete operation for {} objects", keys.size()); + for (DeleteObjectsRequest.KeyVersion key : keys) { + LOG.debug(" {} {}", key.getKey(), + key.getVersion() != null ? key.getVersion() : ""); + } + } // this will update the metastore on a failure, but on // a successful operation leaves the store as is. callbacks.removeKeys( 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 666c233575ad6..d1dc37b0c73d1 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 @@ -173,7 +173,7 @@ private NullRenameTracker( final StoreContext storeContext, final Path source, final Path dest, MetadataStore metadataStore) { - super("null tracker", storeContext, metadataStore, source, dest, null); + super("NullRenameTracker", storeContext, metadataStore, source, dest, null); } @Override diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index 1a49cabe5ad85..388d5f55deefc 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.tools; +import java.io.FileNotFoundException; import java.io.IOException; import java.io.PrintStream; import java.util.ArrayList; @@ -39,19 +40,22 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.StorageStatistics; -import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; +import org.apache.hadoop.fs.s3a.UnknownStoreException; import org.apache.hadoop.fs.s3a.impl.DirMarkerTracker; +import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; import org.apache.hadoop.fs.s3a.impl.OperationCallbacks; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; import org.apache.hadoop.fs.shell.CommandFormat; +import org.apache.hadoop.service.launcher.LauncherExitCodes; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.util.OperationDuration; +import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH; import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE; import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE_DEFAULT; import static org.apache.hadoop.fs.s3a.Invoker.once; @@ -224,8 +228,14 @@ ScanResult execute( storeContext = fs.createStoreContext(); operationCallbacks = fs.getOperationCallbacks(); - println(out, "Directory Marker Policy is %s", - fs.getDirectoryPolicy().describe()); + DirectoryPolicy.MarkerPolicy policy = fs.getDirectoryMarkerPolicy(); + println(out, "The store's directory marker policy is \"%s\"", + policy); + if (policy == DirectoryPolicy.MarkerPolicy.Authoritative) { + // in auth mode, note the auth paths. + String authPath = fs.getConf().getTrimmed(AUTHORITATIVE_PATH, "unset"); + println(out, "Authoritative path list is %s", authPath); + } ScanResult result = once("action", path.toString(), () -> scan(path, doPurge, expectedMarkerCount)); @@ -277,14 +287,28 @@ public MarkerPurgeSummary getPurgeSummary() { * @param doPurge purge? * @param expectedMarkerCount expected marker count * @return scan+purge result. - * @throws IOException - * @throws ExitUtil.ExitException + * @throws IOException IO failure + * @throws ExitUtil.ExitException explicitly raised failure */ private ScanResult scan( final Path path, final boolean doPurge, final int expectedMarkerCount) throws IOException, ExitUtil.ExitException { + // initial safety check: does the path exist + try { + getFilesystem().getFileStatus(path); + } catch (UnknownStoreException ex) { + // bucket doesn't exist. + // replace the stack trace with an error code. + throw new ExitUtil.ExitException(LauncherExitCodes.EXIT_NOT_FOUND, + ex.toString(), ex); + + } catch (FileNotFoundException ex) { + throw new ExitUtil.ExitException(LauncherExitCodes.EXIT_NOT_FOUND, + "Not found: " + path, ex); + + } ScanResult result = new ScanResult(); DirMarkerTracker tracker = new DirMarkerTracker(); @@ -292,6 +316,7 @@ private ScanResult scan( try (DurationInfo ignored = new DurationInfo(LOG, "marker scan %s", path)) { scanDirectoryTree(path, tracker); + } // scan done. what have we got? Map surplusMarkers diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java index 39d58bcf72780..ccd687102e807 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.s3a.performance; +import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -25,10 +26,9 @@ import java.util.concurrent.Callable; import java.util.stream.Collectors; +import com.amazonaws.AmazonClientException; import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.model.AmazonS3Exception; import com.amazonaws.services.s3.model.ObjectMetadata; -import org.junit.Assume; import org.junit.FixMethodOrder; import org.junit.Test; import org.junit.runners.MethodSorters; @@ -44,11 +44,12 @@ 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.impl.DirectoryPolicy; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.createTestPath; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -82,12 +83,34 @@ public class ITestDirectoryMarkerListing extends AbstractS3ATestBase { private static final Logger LOG = LoggerFactory.getLogger(ITestDirectoryMarkerListing.class); - private static final String NAME = "ITestDirectoryMarkerListing"; - private static final String FILENAME = "fileUnderMarker"; private static final String HELLO = "hello"; + private static final String MARKER = "marker"; + + private static final String MARKER_PEER = "markerpeer"; + + /** + * Does rename copy markers? + * Value: {@value} + *

+ * Older releases: yes. + *

+ * The full marker-optimized releases: no. + */ + private static final boolean RENAME_COPIES_MARKERS = false; + + /** + * Does rename copy markers? + * Value: {@value} + *

+ * Older releases: yes. + *

+ * The full marker-optimized releases: no. + */ + private boolean renameDeletesParentMarkers = false; + private Path markerDir; private String markerKey; @@ -98,9 +121,9 @@ public class ITestDirectoryMarkerListing extends AbstractS3ATestBase { private AmazonS3 s3client; - private String fileUnderMarker; + private String fileKeyUnderMarker; - private Path pathUnderMarker; + private Path filePathUnderMarker; private Path basePath; @@ -122,45 +145,56 @@ public void setup() throws Exception { S3AFileSystem fs = getFileSystem(); assume("unguarded FS only", !fs.hasMetadataStore()); + s3client = fs.getAmazonS3ClientForTesting("markers"); + + bucket = fs.getBucket(); + renameDeletesParentMarkers = fs.getDirectoryMarkerPolicy() + == DirectoryPolicy.MarkerPolicy.Delete; + createTestObjects(new Path(methodPath(), "base")); + } - basePath = fs.qualify(createTestPath(new Path("/base-" + NAME))); - markerDir = new Path(basePath, "marker"); + /** + * Create the test objects under the given path, setting + * various fields in the process. + * @param path parent path of everything + */ + private void createTestObjects(final Path path) throws Exception { + S3AFileSystem fs = getFileSystem(); + basePath = path; + markerDir = new Path(basePath, MARKER); // peer path has the same initial name to make sure there // is no confusion there. - markerPeer = new Path(basePath, "markerpeer"); + markerPeer = new Path(basePath, MARKER_PEER); markerKey = fs.pathToKey(markerDir); markerKeySlash = markerKey + "/"; - fileUnderMarker = markerKeySlash + FILENAME; - pathUnderMarker = new Path(markerDir, FILENAME); - bucket = fs.getBucket(); - s3client = fs.getAmazonS3ClientForTesting("markers"); + fileKeyUnderMarker = markerKeySlash + FILENAME; + filePathUnderMarker = new Path(markerDir, FILENAME); + // put the empty dir + fs.mkdirs(markerDir); + touch(fs, markerPeer); + put(fileKeyUnderMarker, HELLO); } + /* + ================================================================= + Basic probes + ================================================================= + */ @Test - public void test_010_createMarker() throws Throwable { + public void testMarkerExists() throws Throwable { describe("Create the test markers for the suite"); - createTestObjects(); head(markerKeySlash); assertIsDirectory(markerDir); } @Test - public void test_020_objectUnderMarker() throws Throwable { - assertIsFile(pathUnderMarker); + public void testObjectUnderMarker() throws Throwable { + assertIsFile(filePathUnderMarker); assertIsDirectory(markerDir); - head(fileUnderMarker); - + head(fileKeyUnderMarker); } - @Test - public void test_030_listStatus() throws Throwable { - assumeTestObjectsExist(); - containsFileUnderMarkerOnly( - toList(getFileSystem().listStatus(markerDir))); - } - - /* ================================================================= The listing operations @@ -168,19 +202,24 @@ public void test_030_listStatus() throws Throwable { */ @Test - public void test_040_listStatus() throws Throwable { - assumeTestObjectsExist(); - List statuses = toList( - getFileSystem().listFiles(markerDir, false)); - containsFileUnderMarkerOnly(statuses); + public void testListStatusMarkerDir() throws Throwable { + describe("list the marker directory and expect to see the file"); + assertContainsFileUnderMarkerOnly( + toList(getFileSystem().listStatus(markerDir))); + } + + + @Test + public void testListFilesMarkerDirFlat() throws Throwable { + assertContainsFileUnderMarkerOnly(toList( + getFileSystem().listFiles(markerDir, false))); } @Test - public void test_050_listStatusRecursive() throws Throwable { - assumeTestObjectsExist(); + public void testListFilesMarkerDirRecursive() throws Throwable { List statuses = toList( getFileSystem().listFiles(markerDir, true)); - containsFileUnderMarkerOnly(statuses); + assertContainsFileUnderMarkerOnly(statuses); } /** @@ -188,54 +227,57 @@ public void test_050_listStatusRecursive() throws Throwable { * and not the marker. */ @Test - public void test_060_listStatusBaseRecursive() throws Throwable { - assumeTestObjectsExist(); + public void testListStatusBaseDirRecursive() throws Throwable { List statuses = toList( getFileSystem().listFiles(basePath, true)); - containsStatusOfPaths(statuses, pathUnderMarker, markerPeer); + assertContainsExactlyStatusOfPaths(statuses, filePathUnderMarker, markerPeer); } @Test - public void test_070_globStatusBaseRecursive() throws Throwable { - assumeTestObjectsExist(); + public void testGlobStatusBaseDirRecursive() throws Throwable { List statuses = exec("glob", () -> toList(getFileSystem().globStatus(new Path(basePath, "*")))); - containsStatusOfPaths(statuses, markerDir, markerPeer); - isFileAtPath(markerPeer, statuses.get(1)); + assertContainsExactlyStatusOfPaths(statuses, markerDir, markerPeer); + assertIsFileAtPath(markerPeer, statuses.get(1)); } @Test - public void test_080_globStatusMarkerDir() throws Throwable { - assumeTestObjectsExist(); + public void testGlobStatusMarkerDir() throws Throwable { List statuses = exec("glob", () -> toList(getFileSystem().globStatus(new Path(markerDir, "*")))); - containsFileUnderMarkerOnly(statuses); + assertContainsFileUnderMarkerOnly(statuses); } /** - * The list here returns the marker peer and dir in a different order. Wny? - * + * Call {@code listLocatedStatus(basePath)} + *

+ * The list here returns the marker peer before the + * dir. Reason: the listing iterators return + * the objects before the common prefixes, and the + * marker dir is coming back as a prefix. */ @Test - public void test_090_listLocatedStatusBaseDir() throws Throwable { - assumeTestObjectsExist(); + public void testListLocatedStatusBaseDir() throws Throwable { List statuses = exec("listLocatedStatus", () -> toList(getFileSystem().listLocatedStatus(basePath))); - containsStatusOfPaths(statuses, markerPeer, markerDir); + assertContainsExactlyStatusOfPaths(statuses, markerPeer, markerDir); } + /** + * Call {@code listLocatedStatus(markerDir)}; expect + * the file entry only. + */ @Test - public void test_100_listLocatedStatusMarkerDir() throws Throwable { - assumeTestObjectsExist(); + public void testListLocatedStatusMarkerDir() throws Throwable { List statuses = exec("listLocatedStatus", () -> toList(getFileSystem().listLocatedStatus(markerDir))); - containsFileUnderMarkerOnly(statuses); + assertContainsFileUnderMarkerOnly(statuses); } @@ -246,8 +288,8 @@ public void test_100_listLocatedStatusMarkerDir() throws Throwable { */ @Test - public void test_200_create_no_overwrite_marker() throws Throwable { - assumeTestObjectsExist(); + public void testCreateNoOverwriteMarkerDir() throws Throwable { + describe("create no-overwrite over the marker dir fails"); head(markerKeySlash); intercept(FileAlreadyExistsException.class, () -> exec("create", () -> @@ -257,37 +299,102 @@ public void test_200_create_no_overwrite_marker() throws Throwable { } @Test - public void test_210_create_no_overwrite_file() throws Throwable { - head(fileUnderMarker); + public void testCreateNoOverwriteFile() throws Throwable { + describe("create-no-overwrite on the file fails"); + + head(fileKeyUnderMarker); intercept(FileAlreadyExistsException.class, () -> exec("create", () -> - getFileSystem().create(pathUnderMarker, false))); - verifyTestObjectsExist(); + getFileSystem().create(filePathUnderMarker, false))); + assertTestObjectsExist(); } @Test - public void test_220_createfile_no_overwrite() throws Throwable { - head(fileUnderMarker); + public void testCreateFileNoOverwrite() throws Throwable { + describe("verify the createFile() API also fails"); + head(fileKeyUnderMarker); intercept(FileAlreadyExistsException.class, () -> exec("create", () -> - getFileSystem().createFile(pathUnderMarker) + getFileSystem().createFile(filePathUnderMarker) .overwrite(false) .build())); - verifyTestObjectsExist(); + assertTestObjectsExist(); } /* ================================================================= Rename. - These tests use methodPaths for src and test ================================================================= */ + /** + * Rename the base directory, expect the source files to move. + *

+ * Whether or not the marker itself is copied depends on whether + * the release's rename operation explicitly skips + * markers on renames. + */ @Test - public void test_300_rename_base() throws Throwable { + public void testRenameBase() throws Throwable { + describe("directory rename"); + + Path src = basePath; + Path dest = new Path(methodPath(), "dest"); + assertRenamed(src, dest); + + assertPathDoesNotExist("source", src); + assertPathDoesNotExist("source", filePathUnderMarker); + assertPathExists("dest not found", dest); + + // all the paths dest relative + Path destMarkerDir = new Path(dest, MARKER); + // peer path has the same initial name to make sure there + // is no confusion there. + Path destMarkerPeer = new Path(dest, MARKER_PEER); + String destMarkerKey = toKey(destMarkerDir); + String destMarkerKeySlash = destMarkerKey + "/"; + String destFileKeyUnderMarker = destMarkerKeySlash + FILENAME; + Path destFilePathUnderMarker = new Path(destMarkerDir, FILENAME); + assertIsFile(destFilePathUnderMarker); + assertIsFile(destMarkerPeer); + head(destFileKeyUnderMarker); + + // probe for the marker based on expected rename + // behavior + if (RENAME_COPIES_MARKERS) { + head(destMarkerKeySlash); + } else { + head404(destMarkerKeySlash); + } } + /** + * Rename under a marker. + * What does happen here? + */ + @Test + public void testRenameUnderMarker() throws Throwable { + describe("directory rename"); + Path base = methodPath(); + S3AFileSystem fs = getFileSystem(); + String name = "sourceFile"; + Path src = new Path(methodPath(), name); + String srcKey = toKey(src); + put(srcKey, "source"); + + + Path dest = markerDir; + // renamed into the dest dir + assertRenamed(src, dest); + assertIsFile(new Path(dest, name)); + if (renameDeletesParentMarkers) { + head404(markerKeySlash); + } else { + head(markerKeySlash); + } + } + /* ================================================================= Delete. @@ -295,15 +402,21 @@ public void test_300_rename_base() throws Throwable { */ @Test - public void test_900_delete() throws Throwable { - assumeTestObjectsExist(); + public void testDelete() throws Throwable { S3AFileSystem fs = getFileSystem(); + // a non recursive delete MUST fail because + // it is not empty intercept(PathIsNotEmptyDirectoryException.class, () -> fs.delete(markerDir, false)); - head(fileUnderMarker); + // file is still there + head(fileKeyUnderMarker); + + // recursive delete MUST succeed fs.delete(markerDir, true); - intercept(AmazonS3Exception.class, () -> - head(fileUnderMarker)); + // and the markers are gone + head404(fileKeyUnderMarker); + head404(markerKeySlash); + // just for completeness fs.delete(basePath, true); } @@ -314,81 +427,85 @@ public void test_900_delete() throws Throwable { */ /** - * Creates the test objects at the well known paths; - * no probes for existence before or after. + * Assert the test objects exist */ - private void createTestObjects() throws Exception { - S3AFileSystem fs = getFileSystem(); - // put the empty dir - fs.mkdirs(markerDir); - touch(fs, markerPeer); - put(fileUnderMarker, HELLO); - } - - private void assumeTestObjectsExist() throws Exception { - assumeExists(fileUnderMarker); - assumeExists(markerKeySlash); - } - - private void verifyTestObjectsExist() throws Exception { - head(fileUnderMarker); + private void assertTestObjectsExist() throws Exception { + head(fileKeyUnderMarker); head(markerKeySlash); } + /** + * Put a string to a path. + * @param key key + * @param content string + */ private void put(final String key, final String content) throws Exception { exec("PUT " + key, () -> s3client.putObject(bucket, key, content)); } + /** + * Issue a HEAD request. + * @param key + * @return the metadata + */ private ObjectMetadata head(final String key) throws Exception { return exec("HEAD " + key, () -> s3client.getObjectMetadata(bucket, key)); } - private void assumeExists(final String key) throws Exception { - try { - head(key); - } catch (AmazonS3Exception e) { - Assume.assumeTrue("object " + key + " not found", false); - } - } - - private String marker(Path path) throws Exception { - String key = getFileSystem().pathToKey(path) + "/"; - put(key, ""); - return key; - } - - - private ObjectMetadata head(final Path path) throws Exception { - return head(getFileSystem().pathToKey(path)); + /** + * Issue a HEAD request and expect a 404 back. + * @param key + * @return the metadata + */ + private void head404(final String key) throws Exception { + intercept(FileNotFoundException.class, "", + "Expected 404 of " + key, () -> + head(key)); } + /** + * Execute an operation; transate AWS exceptions. + * @param op operation + * @param call call to make + * @param returned type + * @return result of the call. + * @throws Exception failure + */ private T exec(String op, Callable call) throws Exception { ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); try { return call.call(); + } catch (AmazonClientException ex) { + throw S3AUtils.translateException(op, "", ex); } finally { timer.end(op); } } - private void containsFileUnderMarkerOnly(final List statuses) { - assertEquals("Status length", - 1, statuses.size()); - isFileUnderMarker(statuses.get(0)); + /** + * Assert that the listing contains only the status + * of the file under the marker. + * @param statuses status objects + */ + private void assertContainsFileUnderMarkerOnly( + final List statuses) { + assertContainsExactlyStatusOfPaths(statuses, filePathUnderMarker); + assertIsFileUnderMarker(statuses.get(0)); } /** - * Expect the list of status to match that of the paths. - * @param statuses status list + * Expect the list of status objects to match that of the paths. + * @param statuses status object list * @param paths ordered varargs list of paths - * @param type of status + * @param type of status objects */ - private void containsStatusOfPaths( + private void assertContainsExactlyStatusOfPaths( List statuses, Path... paths) { + String summary = statuses.stream() .map(Object::toString) .collect(Collectors.joining(";")); @@ -400,38 +517,73 @@ private void containsStatusOfPaths( } } - private void isFileUnderMarker(final FileStatus stat) { - isFileAtPath(pathUnderMarker, stat); + /** + * Assert the status object refers to the file created + * under the marker. + * @param stat status object + */ + private void assertIsFileUnderMarker(final FileStatus stat) { + assertIsFileAtPath(filePathUnderMarker, stat); } - private void isFileAtPath(final Path path, final FileStatus stat) { - assertTrue("Is not file " + stat, - stat.isFile()); - assertName(path, stat); + /** + * Assert the status object refers to a path at the given name + * @param path path + * @param stat status object + */ + private void assertIsFileAtPath(final Path path, final FileStatus stat) { + assertTrue("Is not file " + stat, stat.isFile()); + assertPathEquals(path, stat); } - private void assertName(final Path path, final FileStatus stat) { + /** + * Assert a status object's path matches expected. + * @param path path to expect + * @param stat status object + */ + private void assertPathEquals(final Path path, final FileStatus stat) { assertEquals("filename is not the expected path :" + stat, path, stat.getPath()); } - private List - toList(RemoteIterator status) throws IOException { + /** + * Given a remote iterator of status objects, + * build a list of the values. + * @param status status list + * @param actual type. + * @return source. + * @throws IOException + */ + private List toList( + RemoteIterator status) throws IOException { List l = new ArrayList<>(); while (status.hasNext()) { l.add(status.next()); } return dump(l); - } - private List - toList(T[] status) throws IOException { - List l = Arrays.asList(status); - return dump(l); + /** + * Given an array of status objects, + * build a list of the values. + * @param status status list + * @param actual type. + * @return source. + * @throws IOException + */ + private List toList( + T[] status) throws IOException { + return dump(Arrays.asList(status)); } + /** + * Dump the string values of a list to the log; return + * the list. + * @param l source. + * @param source type + * @return the list + */ private List dump(List l) { int c = 1; for (T t : l) { @@ -440,5 +592,19 @@ private List dump(List l) { return l; } + /** + * Rename: assert the outcome is true. + * @param src source path + * @param dest dest path + */ + private void assertRenamed(final Path src, final Path dest) + throws IOException { + assertTrue("rename(" + src + ", " + dest + ")", + getFileSystem().rename(src, dest)); + } + + private String toKey(final Path destMarkerDir) { + return getFileSystem().pathToKey(destMarkerDir); + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java index 7a0fd917a36b3..bfd6de8779e2e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java @@ -44,6 +44,7 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; import static org.apache.hadoop.fs.s3a.tools.MarkerTool.*; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE; +import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_FOUND; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_USAGE; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -68,13 +69,13 @@ public class ITestMarkerTool extends AbstractS3ATestBase { /** FS which mixes markers; only created in some tests. */ private S3AFileSystem mixedFS; - private int expectedFiles; + private int expectedFileCount; - private int expectMarkersUnderDir1; + private int expectedMarkersUnderDir1; - private int expectMarkersUnderDir2; + private int expectedMarkersUnderDir2; - private int expectMarkers; + private int expectedMarkers; @Override protected Configuration createConfiguration() { @@ -109,39 +110,6 @@ public void teardown() throws Exception { } - /** - * Create a new FS with given marker policy and path. - * This filesystem MUST be closed in test teardown. - * @param markerPolicy markers - * @param authPath authoritative path. If null: no path. - * @return a new FS. - */ - private S3AFileSystem createFS(String markerPolicy, - String authPath) throws Exception { - S3AFileSystem testFS = getFileSystem(); - Configuration conf = new Configuration(testFS.getConf()); - URI testFSUri = testFS.getUri(); - String bucketName = getTestBucketName(conf); - removeBucketOverrides(bucketName, conf, - DIRECTORY_MARKER_POLICY, - S3_METADATA_STORE_IMPL, - BULK_DELETE_PAGE_SIZE, - AUTHORITATIVE_PATH); - if (authPath != null) { - conf.set(AUTHORITATIVE_PATH, authPath); - } - // Use a very small page size to force the paging - // code to be tested. - conf.setInt(BULK_DELETE_PAGE_SIZE, 4); - conf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL); - conf.set(DIRECTORY_MARKER_POLICY, markerPolicy); - S3AFileSystem fs2 = new S3AFileSystem(); - fs2.initialize(testFSUri, conf); - LOG.info("created new filesystem with policy {} and auth path {}: {}", - markerPolicy, authPath, fs2); - return fs2; - } - /** * FS which deletes markers. */ @@ -173,161 +141,44 @@ private void setMixedFS(S3AFileSystem mixedFS) { this.mixedFS = mixedFS; } - private static Path mkpath(Path base, final String name) { - return name.isEmpty() ? base : new Path(base, name); - } - - /** - * Tracker of created paths. - */ - private static class CreatedPaths { - - private Path base; - - private List files = new ArrayList<>(); - - private List dirs = new ArrayList<>(); - - private List emptyDirs = new ArrayList<>(); - - private List filesUnderBase = new ArrayList<>(); - - private List dirsUnderBase = new ArrayList<>(); - - private List emptyDirsUnderBase = new ArrayList<>(); - - - private Path mkdir(FileSystem fs, String name) - throws IOException { - Path dir = mkpath(base, name); - fs.mkdirs(dir); - dirs.add(dir); - dirsUnderBase.add(name); - return dir; - } - - private Path emptydir(FileSystem fs, String name) - throws IOException { - Path dir = mkpath(base, name); - fs.mkdirs(dir); - emptyDirs.add(dir); - emptyDirsUnderBase.add(name); - return dir; - } - - private Path mkfile(FileSystem fs, String name) - throws IOException { - Path file = mkpath(base, name); - ContractTestUtils.touch(fs, file); - files.add(file); - filesUnderBase.add(name); - return file; - } - } - - /** - * Create the "standard" test paths. - * @param fs filesystem - * @param base base dir - * @return the details on what was created. - */ - private CreatedPaths createPaths(FileSystem fs, Path base) - throws IOException { - CreatedPaths r = new CreatedPaths(); - r.base = base; - // the directories under which we will create files, - // so expect to have markers - r.mkdir(fs, ""); - r.mkdir(fs, "dir1"); - r.mkdir(fs, "dir2"); - r.mkdir(fs, "dir2/dir3"); - - // create the empty dirs - r.emptydir(fs, "empty"); - r.emptydir(fs, "dir2/empty"); - - // files - r.mkfile(fs, "dir1/file1"); - r.mkfile(fs, "dir2/file2"); - r.mkfile(fs, "dir2/dir3/file3"); - - expectedFiles = 3; - expectMarkersUnderDir1 = 1; - expectMarkersUnderDir2 = 2; - expectMarkers = expectMarkersUnderDir1 + expectMarkersUnderDir2; - return r; - } - - private MarkerTool.ScanResult markerTool( - final FileSystem sourceFS, - final Path path, - final boolean doPurge, - final int expectedMarkerCount) - throws IOException { - return markerTool(0, sourceFS, path, doPurge, expectedMarkerCount); - } - - @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") - private MarkerTool.ScanResult markerTool( - final int exitCode, - final FileSystem sourceFS, - final Path path, - final boolean doPurge, - final int expectedMarkerCount) throws IOException { - MarkerTool tool = new MarkerTool(sourceFS.getConf()); - tool.setVerbose(true); - - MarkerTool.ScanResult result = tool.execute(sourceFS, path, doPurge, - expectedMarkerCount); - Assertions.assertThat(result.getExitCode()) - .describedAs("Exit code of marker(%s, %s, %d) -> %s", - path, doPurge, expectedMarkerCount, result) - .isEqualTo(exitCode); - return result; - } - - void verifyRenamed(final Path dest, - final CreatedPaths createdPaths) throws IOException { - // all leaf directories exist - for (String p : createdPaths.emptyDirsUnderBase) { - assertIsDirectory(mkpath(dest, p)); - } - // non-empty dirs - for (String p : createdPaths.dirsUnderBase) { - assertIsDirectory(mkpath(dest, p)); - } - // all files exist - for (String p : createdPaths.filesUnderBase) { - assertIsFile(mkpath(dest, p)); - } - } @Test - public void testAuditPruneMarkersLegacyDir() throws Throwable { + public void testCleanMarkersLegacyDir() throws Throwable { + describe("Clean markers under a deleting FS -expect none"); CreatedPaths createdPaths = createPaths(getDeletingFS(), methodPath()); markerTool(getDeletingFS(), createdPaths.base, false, 0); markerTool(getDeletingFS(), createdPaths.base, true, 0); } @Test - public void testAuditPruneMarkersKeepingDir() throws Throwable { + public void testCleanMarkersKeepingDir() throws Throwable { + describe("Audit then clean markers under a deleting FS " + + "-expect markers to be found and then cleaned up"); CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); // audit will find the expected entries int expectedMarkerCount = createdPaths.dirs.size(); S3AFileSystem fs = getDeletingFS(); + LOG.info("Auditing a directory with retained markers -expect failure"); markerTool(EXIT_NOT_ACCEPTABLE, fs, createdPaths.base, false, 0); + LOG.info("Auditing a directory expecting retained markers"); markerTool(fs, createdPaths.base, false, expectedMarkerCount); - // we know a purge didn't take place + + // we require that a purge didn't take place, so run the + // audit again. + LOG.info("Auditing a directory expecting retained markers"); markerTool(fs, createdPaths.base, false, expectedMarkerCount); + + LOG.info("Purging a directory of retained markers"); // purge cleans up assertMarkersDeleted(expectedMarkerCount, markerTool(fs, createdPaths.base, true, expectedMarkerCount)); // and a rerun doesn't find markers + LOG.info("Auditing a directory with retained markers -expect success"); assertMarkersDeleted(0, markerTool(fs, createdPaths.base, true, 0)); } @@ -346,8 +197,7 @@ public void testRenameKeepingFS() throws Throwable { // audit will find three entries int expectedMarkerCount = createdPaths.dirs.size(); - markerTool(fs, source, false, - expectedMarkerCount); + markerTool(fs, source, false, expectedMarkerCount); fs.rename(source, dest); assertIsDirectory(dest); @@ -355,7 +205,6 @@ public void testRenameKeepingFS() throws Throwable { markerTool(fs, dest, false, 0); LOG.info("Auditing destination paths"); verifyRenamed(dest, createdPaths); - } /** @@ -371,14 +220,17 @@ public void testAuthPathIsMixed() throws Throwable { Path dir2 = new Path(source, "dir2"); S3AFileSystem mixedFSDir2 = createFS(DIRECTORY_MARKER_POLICY_AUTHORITATIVE, dir2.toUri().toString()); + // line up for close in teardown + setMixedFS(mixedFSDir2); // some of these paths will retain markers, some will not CreatedPaths createdPaths = createPaths(mixedFSDir2, source); // markers are only under dir2 - markerTool(mixedFSDir2, mkpath(source, "dir1"), false, 0); - markerTool(mixedFSDir2, source, false, expectMarkersUnderDir2); + markerTool(mixedFSDir2, topath(source, "dir1"), false, 0); + markerTool(mixedFSDir2, source, false, expectedMarkersUnderDir2); // if we now rename, all will be good + LOG.info("Executing rename"); mixedFSDir2.rename(source, dest); assertIsDirectory(dest); @@ -388,7 +240,7 @@ public void testAuthPathIsMixed() throws Throwable { Assertions.assertThat(scanResult) .describedAs("Scan result %s", scanResult) .extracting(s -> s.getTracker().getFilesFound()) - .isEqualTo(expectedFiles); + .isEqualTo(expectedFileCount); verifyRenamed(dest, createdPaths); } @@ -412,8 +264,21 @@ public void testRunNoArgs() throws Throwable { runToFailure(EXIT_USAGE, NAME); } + @Test + public void testRunWrongBucket() throws Throwable { + runToFailure(EXIT_NOT_FOUND, NAME, AUDIT, + "s3a://this-bucket-does-not-exist-hopefully"); + } + + @Test + public void testRunWrongPath() throws Throwable { + runToFailure(EXIT_NOT_FOUND, NAME, AUDIT, + methodPath().toString()); + } + @Test public void testRunVerboseAudit() throws Throwable { + describe("Run a verbose audit"); CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); run(NAME, V, @@ -425,6 +290,265 @@ public void testRunVerboseAudit() throws Throwable { createdPaths.base.toString()); } + private static Path topath(Path base, final String name) { + return name.isEmpty() ? base : new Path(base, name); + } + + /** + * Create a new FS with given marker policy and path. + * This filesystem MUST be closed in test teardown. + * @param markerPolicy markers + * @param authPath authoritative path. If null: no path. + * @return a new FS. + */ + private S3AFileSystem createFS(String markerPolicy, + String authPath) throws Exception { + S3AFileSystem testFS = getFileSystem(); + Configuration conf = new Configuration(testFS.getConf()); + URI testFSUri = testFS.getUri(); + String bucketName = getTestBucketName(conf); + removeBucketOverrides(bucketName, conf, + DIRECTORY_MARKER_POLICY, + S3_METADATA_STORE_IMPL, + BULK_DELETE_PAGE_SIZE, + AUTHORITATIVE_PATH); + if (authPath != null) { + conf.set(AUTHORITATIVE_PATH, authPath); + } + // Use a very small page size to force the paging + // code to be tested. + conf.setInt(BULK_DELETE_PAGE_SIZE, 2); + conf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL); + conf.set(DIRECTORY_MARKER_POLICY, markerPolicy); + S3AFileSystem fs2 = new S3AFileSystem(); + fs2.initialize(testFSUri, conf); + LOG.info("created new filesystem with policy {} and auth path {}: {}", + markerPolicy, authPath, fs2); + return fs2; + } + + /** + * Tracker of created paths. + */ + private static final class CreatedPaths { + + private final FileSystem fs; + + private final Path base; + + private List files = new ArrayList<>(); + + private List dirs = new ArrayList<>(); + + private List emptyDirs = new ArrayList<>(); + + private List filesUnderBase = new ArrayList<>(); + + private List dirsUnderBase = new ArrayList<>(); + + private List emptyDirsUnderBase = new ArrayList<>(); + + /** + * Constructor. + * @param fs filesystem. + * @param base base directory for all creation operations. + */ + private CreatedPaths(final FileSystem fs, + final Path base) { + this.fs = fs; + this.base = base; + } + + /** + * Make a set of directories. + * @param names varargs list of paths under the base. + * @return number of entries created. + * @throws IOException failure + */ + private int dirs(String... names) throws IOException { + for (String name : names) { + mkdir(name); + } + return names.length; + } + + /** + * Create a single directory under the base. + * @param name name/relative names of the directory + * @return the path of the new entry. + */ + private Path mkdir(String name) throws IOException { + Path dir = topath(base, name); + fs.mkdirs(dir); + dirs.add(dir); + dirsUnderBase.add(name); + return dir; + } + + /** + * Make a set of empty directories. + * @param names varargs list of paths under the base. + * @return number of entries created. + * @throws IOException failure + */ + private int emptydirs(String... names) throws IOException { + for (String name : names) { + emptydir(name); + } + return names.length; + } + + /** + * Create an empty directory. + * @param name name under the base dir + * @return the path + * @throws IOException failure + */ + private Path emptydir(String name) throws IOException { + Path dir = topath(base, name); + fs.mkdirs(dir); + emptyDirs.add(dir); + emptyDirsUnderBase.add(name); + return dir; + } + + /** + * Make a set of files. + * @param names varargs list of paths under the base. + * @return number of entries created. + * @throws IOException failure + */ + private int files(String... names) throws IOException { + for (String name : names) { + mkfile(name); + } + return names.length; + } + + /** + * Create a 0-byte file. + * @param name name under the base dir + * @return the path + * @throws IOException failure + */ + private Path mkfile(String name) + throws IOException { + Path file = topath(base, name); + ContractTestUtils.touch(fs, file); + files.add(file); + filesUnderBase.add(name); + return file; + } + } + + /** + * Create the "standard" test paths. + * @param fs filesystem + * @param base base dir + * @return the details on what was created. + */ + private CreatedPaths createPaths(FileSystem fs, Path base) + throws IOException { + CreatedPaths r = new CreatedPaths(fs, base); + // the directories under which we will create files, + // so expect to have markers + r.mkdir(""); + + // create the empty dirs + r.emptydir("empty"); + + // dir 1 has a file underneath + r.mkdir("dir1"); + expectedFileCount = r.files("dir1/file1"); + + expectedMarkersUnderDir1 = 1; + + + // dir2 has a subdir + r.dirs("dir2", "dir2/dir3"); + // an empty subdir + r.emptydir("dir2/empty2"); + + // and a file under itself and dir3 + expectedFileCount += r.files( + "dir2/file2", + "dir2/dir3/file3"); + + + // wrap up the expectations. + expectedMarkersUnderDir2 = 2; + expectedMarkers = expectedMarkersUnderDir1 + expectedMarkersUnderDir2; + return r; + } + + /** + * Execute the marker tool, expecting the execution to succeed. + * @param sourceFS filesystem to use + * @param path path to scan + * @param doPurge should markers be purged + * @param expectedMarkers number of markers expected + * @return the result + */ + private MarkerTool.ScanResult markerTool( + final FileSystem sourceFS, + final Path path, + final boolean doPurge, + final int expectedMarkers) + throws IOException { + return markerTool(0, sourceFS, path, doPurge, expectedMarkers); + } + + /** + * Execute the marker tool, expecting the execution to + * return a specific exit code. + * + * @param sourceFS filesystem to use + * @param exitCode exit code to expect. + * @param path path to scan + * @param doPurge should markers be purged + * @param expectedMarkers number of markers expected + * @return the result + */ + @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") + private MarkerTool.ScanResult markerTool( + final int exitCode, + final FileSystem sourceFS, + final Path path, + final boolean doPurge, + final int expectedMarkers) throws IOException { + MarkerTool tool = new MarkerTool(sourceFS.getConf()); + tool.setVerbose(LOG.isDebugEnabled()); + + MarkerTool.ScanResult result = tool.execute(sourceFS, path, doPurge, + expectedMarkers); + Assertions.assertThat(result.getExitCode()) + .describedAs("Exit code of marker(%s, %s, %d) -> %s", + path, doPurge, expectedMarkers, result) + .isEqualTo(exitCode); + return result; + } + + /** + * Verify that all the paths renamed from the source exist + * under the destination, including all empty directories. + * @param dest destination to look under. + * @param createdPaths list of created paths. + */ + void verifyRenamed(final Path dest, + final CreatedPaths createdPaths) throws IOException { + // all leaf directories exist + for (String p : createdPaths.emptyDirsUnderBase) { + assertIsDirectory(topath(dest, p)); + } + // non-empty dirs + for (String p : createdPaths.dirsUnderBase) { + assertIsDirectory(topath(dest, p)); + } + // all files exist + for (String p : createdPaths.filesUnderBase) { + assertIsFile(topath(dest, p)); + } + } /** * Run a S3GuardTool command from a varags list and the * configuration returned by {@code getConfiguration()}. From 52f74839b3bd87c3af1245402809db715dd99ce7 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 30 Jul 2020 22:23:25 +0100 Subject: [PATCH 11/20] HADOOP-13230 Test fixing and rename debug/fix Test failures are a mix of tests being brittle to marker retention (including the InconsistentS3Client) to some real stabilization issues with rename. Still trying to get the behavior of dir rename right Change-Id: I3f22cf93416aa1f125ff651df2ffde5b3d84e9d1 --- hadoop-tools/hadoop-aws/pom.xml | 16 +-- .../fs/s3a/InconsistentAmazonS3Client.java | 3 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 8 +- .../hadoop/fs/s3a/impl/RenameOperation.java | 23 +++- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 21 +++ .../hadoop/fs/s3a/tools/MarkerTool.java | 127 ++++++++++-------- .../fs/s3a/tools/MarkerToolOperations.java | 92 +++++++++++++ .../s3a/tools/MarkerToolOperationsImpl.java | 69 ++++++++++ .../hadoop/fs/s3a/ITestS3GuardEmptyDirs.java | 15 ++- .../fs/s3a/ITestS3GuardListConsistency.java | 51 +++++-- .../ITestDirectoryMarkerListing.java | 114 +++++++++++----- .../hadoop/fs/s3a/tools/ITestMarkerTool.java | 2 +- 12 files changed, 425 insertions(+), 116 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperations.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperationsImpl.java diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index a887e80fb124d..f4b8ee34b35ed 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -52,7 +52,7 @@ 200000 - + @@ -124,7 +124,7 @@ ${fs.s3a.scale.test.huge.filesize} ${fs.s3a.scale.test.huge.partitionsize} ${fs.s3a.scale.test.timeout} - ${fs.s3a.directory.marker} + ${fs.s3a.directory.marker.retention} @@ -165,7 +165,7 @@ ${fs.s3a.s3guard.test.enabled} ${fs.s3a.s3guard.test.authoritative} ${fs.s3a.s3guard.test.implementation} - ${fs.s3a.directory.marker} + ${fs.s3a.directory.marker.retention} ${test.integration.timeout} @@ -218,7 +218,7 @@ ${fs.s3a.s3guard.test.enabled} ${fs.s3a.s3guard.test.implementation} ${fs.s3a.s3guard.test.authoritative} - ${fs.s3a.directory.marker} + ${fs.s3a.directory.marker.retention} @@ -273,7 +273,7 @@ ${fs.s3a.s3guard.test.enabled} ${fs.s3a.s3guard.test.implementation} ${fs.s3a.s3guard.test.authoritative} - ${fs.s3a.directory.marker} + ${fs.s3a.directory.marker.retention} ${fs.s3a.scale.test.timeout} @@ -347,7 +347,7 @@ - keep + keep @@ -359,7 +359,7 @@ - delete + delete @@ -371,7 +371,7 @@ - authoritative + authoritative diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java index 34c043be9cb73..d8e5d9eb2a91c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java @@ -330,7 +330,8 @@ private boolean isDescendant(String parent, String child, boolean recursive) { } else { Path actualParentPath = new Path(child).getParent(); Path expectedParentPath = new Path(parent); - return actualParentPath.equals(expectedParentPath); + // children which are directory markers are excluded here + return actualParentPath.equals(expectedParentPath) && !child.endsWith("/"); } } 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 fc750d74b3626..2412f237295a4 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 @@ -117,6 +117,8 @@ import org.apache.hadoop.fs.s3a.impl.statistics.S3AMultipartUploaderStatisticsImpl; import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; import org.apache.hadoop.fs.s3a.select.InternalSelectConstants; +import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations; +import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.token.DelegationTokenIssuer; @@ -4980,12 +4982,12 @@ public StoreContext createStoreContext() { } /** - * Get the operation callbacks for this FS. + * Create a marker tools operations binding for this store. * @return callbacks for operations. */ @InterfaceAudience.Private - public OperationCallbacks getOperationCallbacks() { - return operationCallbacks; + public MarkerToolOperations createMarkerToolOperations() { + return new MarkerToolOperationsImpl(operationCallbacks); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java index 064435ac68b00..53d10842ad840 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java @@ -224,7 +224,20 @@ private void queueToDelete(Path path, String key, String version) { private void queueToDelete( List markersToDelete) { markersToDelete.forEach(m -> - queueToDelete(m.getPath(), m.getKey(), m.getStatus().getVersionId())); + queueToDelete(m)); + } + + /** + * Queue a single marker for deletion. + *

+ * See {@link #queueToDelete(Path, String, String)} for + * details on safe use of this method. + * + * @param marker markers + */ + private void queueToDelete(final DirMarkerTracker.Marker marker) { + queueToDelete(marker.getPath(), marker.getKey(), + marker.getStatus().getVersionId()); } /** @@ -454,7 +467,6 @@ private void endOfLoopActions() throws IOException { completeActiveCopies("batch threshold reached"); } } - } /** @@ -513,6 +525,7 @@ private OperationDuration copyEmptyDirectoryMarkers( source, newDestKey, childDestPath)); + queueToDelete(entry); // end of loop endOfLoopActions(); } @@ -622,8 +635,8 @@ private void removeSourceObjects( try { // remove the keys - // first list what is being deleted for the interest of anyone - // who is trying to debug while objects are no longer there. + // list what is being deleted for the interest of anyone + // who is trying to debug why objects are no longer there. if (LOG.isDebugEnabled()) { LOG.debug("Initiating delete operation for {} objects", keys.size()); for (DeleteObjectsRequest.KeyVersion key : keys) { @@ -642,7 +655,7 @@ private void removeSourceObjects( // and clear the list. } catch (AmazonClientException | IOException e) { // Failed. - // Notify the rename operation. + // Notify the rename tracker. // 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 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 7c92fbe728717..fc1c73937eaaf 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 @@ -30,6 +30,7 @@ import java.util.Date; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Locale; import java.util.Map; @@ -50,6 +51,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FilterFileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.StorageStatistics; import org.apache.hadoop.fs.s3a.MultipartUtils; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; @@ -509,6 +511,25 @@ public final int run(String[] args) throws Exception { public abstract int run(String[] args, PrintStream out) throws Exception, ExitUtil.ExitException; + /** + * Dump the filesystem Storage Statistics if the FS is not null. + * @param stream output stream + */ + protected void dumpFileSystemStatistics(PrintStream stream) { + FileSystem fs = getFilesystem(); + if (fs == null) { + return; + } + println(stream, "Storage Statistics for %s", fs.getUri()); + StorageStatistics st = fs.getStorageStatistics(); + Iterator it + = st.getLongStatistics(); + while (it.hasNext()) { + StorageStatistics.LongStatistic next = it.next(); + println(stream, "%s\t%s", next.getName(), next.getValue()); + } + } + /** * Create the metadata store. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index 388d5f55deefc..a62e894cabadf 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -23,7 +23,6 @@ import java.io.PrintStream; import java.util.ArrayList; import java.util.Collections; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -39,14 +38,12 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; -import org.apache.hadoop.fs.StorageStatistics; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; import org.apache.hadoop.fs.s3a.UnknownStoreException; import org.apache.hadoop.fs.s3a.impl.DirMarkerTracker; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; -import org.apache.hadoop.fs.s3a.impl.OperationCallbacks; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; import org.apache.hadoop.fs.shell.CommandFormat; @@ -76,14 +73,22 @@ */ public final class MarkerTool extends S3GuardTool { - private static final Logger LOG = - LoggerFactory.getLogger(MarkerTool.class); + private static final Logger LOG = LoggerFactory.getLogger(MarkerTool.class); + /** + * Name of this tool: {@value}. + */ public static final String NAME = "markers"; + /** + * Purpose of this tool: {@value}. + */ public static final String PURPOSE = "View and manipulate S3 directory markers"; + /** + * Usage string: {@value}. + */ private static final String USAGE = NAME + " [-" + VERBOSE + "]" + " (audit | report | clean)" @@ -118,23 +123,43 @@ public final class MarkerTool extends S3GuardTool { /** * Constant to use when there is no limit on the number of - * markers expected. + * markers expected: {@value}. */ private static final int UNLIMITED = -1; /** Will be overridden in run(), but during tests needs to avoid NPEs. */ private PrintStream out = System.out; + /** + * Verbosity flag. + */ private boolean verbose; + /** + * Should the scan also purge surplus markers? + */ private boolean purge; + /** + * How many markers are expected; + * {@link #UNLIMITED} means no limit. + */ private int expected; - private OperationCallbacks operationCallbacks; - + /** + * Store context. + */ private StoreContext storeContext; + /** + * Operations during the scan. + */ + private MarkerToolOperations operations; + + /** + * Constructor. + * @param conf + */ public MarkerTool(final Configuration conf) { super(conf, OPT_VERBOSE); } @@ -153,7 +178,7 @@ public String getName() { public void resetBindings() { super.resetBindings(); storeContext = null; - operationCallbacks = null; + operations = null; } @Override @@ -205,6 +230,9 @@ public int run(final String[] args, final PrintStream stream) path, purge, expected); + if (verbose) { + dumpFileSystemStatistics(out); + } return result.exitCode; } @@ -226,22 +254,33 @@ ScanResult execute( throws IOException { S3AFileSystem fs = bindFilesystem(sourceFS); storeContext = fs.createStoreContext(); - operationCallbacks = fs.getOperationCallbacks(); + operations = fs.createMarkerToolOperations(); DirectoryPolicy.MarkerPolicy policy = fs.getDirectoryMarkerPolicy(); println(out, "The store's directory marker policy is \"%s\"", policy); if (policy == DirectoryPolicy.MarkerPolicy.Authoritative) { // in auth mode, note the auth paths. - String authPath = fs.getConf().getTrimmed(AUTHORITATIVE_PATH, "unset"); + String authPath = storeContext.getConfiguration() + .getTrimmed(AUTHORITATIVE_PATH, "unset"); println(out, "Authoritative path list is %s", authPath); } + // validate the FS + try { + getFilesystem().getFileStatus(path); + } catch (UnknownStoreException ex) { + // bucket doesn't exist. + // replace the stack trace with an error code. + throw new ExitUtil.ExitException(LauncherExitCodes.EXIT_NOT_FOUND, + ex.toString(), ex); + } catch (FileNotFoundException ex) { + throw new ExitUtil.ExitException(LauncherExitCodes.EXIT_NOT_FOUND, + "Not found: " + path, ex); + + } ScanResult result = once("action", path.toString(), () -> scan(path, doPurge, expectedMarkerCount)); - if (verbose) { - dumpFileSystemStatistics(fs); - } return result; } @@ -250,10 +289,19 @@ ScanResult execute( */ static final class ScanResult { + /** + * Exit code to return if an exception was not raised. + */ private int exitCode; + /** + * The tracker. + */ private DirMarkerTracker tracker; + /** + * Scan summary. + */ private MarkerPurgeSummary purgeSummary; @Override @@ -296,19 +344,7 @@ private ScanResult scan( final int expectedMarkerCount) throws IOException, ExitUtil.ExitException { // initial safety check: does the path exist - try { - getFilesystem().getFileStatus(path); - } catch (UnknownStoreException ex) { - // bucket doesn't exist. - // replace the stack trace with an error code. - throw new ExitUtil.ExitException(LauncherExitCodes.EXIT_NOT_FOUND, - ex.toString(), ex); - - } catch (FileNotFoundException ex) { - throw new ExitUtil.ExitException(LauncherExitCodes.EXIT_NOT_FOUND, - "Not found: " + path, ex); - } ScanResult result = new ScanResult(); DirMarkerTracker tracker = new DirMarkerTracker(); @@ -316,8 +352,8 @@ private ScanResult scan( try (DurationInfo ignored = new DurationInfo(LOG, "marker scan %s", path)) { scanDirectoryTree(path, tracker); - } + // scan done. what have we got? Map surplusMarkers = tracker.getSurplusMarkers(); @@ -386,29 +422,28 @@ private String suffix(final int size) { */ private void scanDirectoryTree(final Path path, final DirMarkerTracker tracker) throws IOException { - RemoteIterator listing = operationCallbacks + RemoteIterator listing = operations .listObjects(path, storeContext.pathToKey(path)); while (listing.hasNext()) { S3AFileStatus status = listing.next(); - Path p = status.getPath(); - S3ALocatedFileStatus lfs = new S3ALocatedFileStatus( + Path statusPath = status.getPath(); + S3ALocatedFileStatus locatedStatus = new S3ALocatedFileStatus( status, null); - String key = storeContext.pathToKey(p); + String key = storeContext.pathToKey(statusPath); if (status.isDirectory()) { if (verbose) { println(out, "Directory Marker %s", key); } LOG.debug("{}", key); - tracker.markerFound(p, + tracker.markerFound(statusPath, key + "/", - lfs); + locatedStatus); } else { - tracker.fileFound(p, + tracker.fileFound(statusPath, key, - lfs); + locatedStatus); } } - } /** @@ -500,7 +535,7 @@ private MarkerPurgeSummary purgeMarkers(DirMarkerTracker tracker, end); List undeleted = new ArrayList<>(); OperationDuration duration = new OperationDuration(); - operationCallbacks.removeKeys(page, true, undeleted, null, false); + operations.removeKeys(page, true, undeleted, null, false); duration.finished(); summary.deleteRequests++; summary.totalDeleteRequestDuration += duration.value(); @@ -511,24 +546,6 @@ private MarkerPurgeSummary purgeMarkers(DirMarkerTracker tracker, return summary; } - /** - * Dump the filesystem Storage Statistics. - * @param fs filesystem; can be null - */ - private void dumpFileSystemStatistics(FileSystem fs) { - if (fs == null) { - return; - } - println(out, "Storage Statistics"); - StorageStatistics st = fs.getStorageStatistics(); - Iterator it - = st.getLongStatistics(); - while (it.hasNext()) { - StorageStatistics.LongStatistic next = it.next(); - println(out, "%s\t%s", next.getName(), next.getValue()); - } - } - public boolean isVerbose() { return verbose; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperations.java new file mode 100644 index 0000000000000..ac2f8bca35b74 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperations.java @@ -0,0 +1,92 @@ +/* + * 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.tools; + +import java.io.IOException; +import java.util.List; + +import com.amazonaws.AmazonClientException; +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.amazonaws.services.s3.model.DeleteObjectsResult; +import com.amazonaws.services.s3.model.MultiObjectDeleteException; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.InvalidRequestException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.s3a.Retries; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; + +/** + * Operations which must be offered by the store for {@link MarkerTool}. + * These are a proper subset of {@code OperationCallbacks}; this interface + * strips down those provided to the tool. + */ +public interface MarkerToolOperations { + + /** + * Create an iterator over objects in S3 only; S3Guard + * is not involved. + * The listing includes the key itself, if found. + * @param path path of the listing. + * @param key object key + * @return iterator with the first listing completed. + * @throws IOException failure. + */ + @Retries.RetryTranslated + RemoteIterator listObjects( + Path path, + String key) + throws IOException; + + /** + * Remove keys from the store, updating the metastore on a + * partial delete represented as a MultiObjectDeleteException failure by + * deleting all those entries successfully deleted and then rethrowing + * the MultiObjectDeleteException. + * @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. + * @param operationState bulk operation state + * @param quiet should a bulk query be quiet, or should its result list + * all deleted keys + * @return the deletion result if a multi object delete was invoked + * and it returned without a failure, else null. + * @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. + */ + @Retries.RetryMixed + DeleteObjectsResult removeKeys( + List keysToDelete, + boolean deleteFakeDir, + List undeletedObjectsOnFailure, + BulkOperationState operationState, + boolean quiet) + throws MultiObjectDeleteException, AmazonClientException, + IOException; + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperationsImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperationsImpl.java new file mode 100644 index 0000000000000..18fc51db7787f --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperationsImpl.java @@ -0,0 +1,69 @@ +/* + * 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.tools; + +import java.io.IOException; +import java.util.List; + +import com.amazonaws.AmazonClientException; +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.amazonaws.services.s3.model.DeleteObjectsResult; +import com.amazonaws.services.s3.model.MultiObjectDeleteException; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.impl.OperationCallbacks; +import org.apache.hadoop.fs.s3a.s3guard.BulkOperationState; + +/** + * Implement the marker tool operations by forwarding to the + * {@link OperationCallbacks} instance provided in the constructor. + */ +public class MarkerToolOperationsImpl implements MarkerToolOperations { + + private final OperationCallbacks operationCallbacks; + + /** + * Constructor. + * @param operations implementation of the operations + */ + public MarkerToolOperationsImpl(final OperationCallbacks operations) { + this.operationCallbacks = operations; + } + + @Override + public RemoteIterator listObjects(final Path path, + final String key) + throws IOException { + return operationCallbacks.listObjects(path, key); + } + + @Override + public DeleteObjectsResult removeKeys(final List keysToDelete, + final boolean deleteFakeDir, + final List undeletedObjectsOnFailure, + final BulkOperationState operationState, + final boolean quiet) + throws MultiObjectDeleteException, AmazonClientException, IOException { + return operationCallbacks.removeKeys(keysToDelete, deleteFakeDir, + undeletedObjectsOnFailure, operationState, quiet); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java index ab81491c4cf90..db3c2b6c27462 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java @@ -26,6 +26,7 @@ import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.model.ListObjectsV2Request; import com.amazonaws.services.s3.model.ListObjectsV2Result; +import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.PutObjectRequest; import com.amazonaws.services.s3.model.S3ObjectSummary; import org.assertj.core.api.Assertions; @@ -57,6 +58,10 @@ */ public class ITestS3GuardEmptyDirs extends AbstractS3ATestBase { + /** + * Rename an empty directory, verify that the empty dir + * marker moves in both S3Guard and in the S3A FS. + */ @Test public void testRenameEmptyDir() throws Throwable { S3AFileSystem fs = getFileSystem(); @@ -67,7 +72,7 @@ public void testRenameEmptyDir() throws Throwable { String destDirMarker = fs.pathToKey(destDir) + "/"; // set things up. mkdirs(sourceDir); - // there'a source directory marker + // there's source directory marker fs.getObjectMetadata(sourceDirMarker); S3AFileStatus srcStatus = getEmptyDirStatus(sourceDir); assertEquals("Must be an empty dir: " + srcStatus, Tristate.TRUE, @@ -82,8 +87,12 @@ public void testRenameEmptyDir() throws Throwable { () -> getEmptyDirStatus(sourceDir)); // and verify that there's no dir marker hidden under a tombstone intercept(FileNotFoundException.class, - () -> Invoker.once("HEAD", sourceDirMarker, - () -> fs.getObjectMetadata(sourceDirMarker))); + () -> Invoker.once("HEAD", sourceDirMarker, () -> { + ObjectMetadata md = fs.getObjectMetadata(sourceDirMarker); + return String.format("Object %s of length %d", + sourceDirMarker, md.getInstanceLength()); + })); + // the parent dir mustn't be confused S3AFileStatus baseStatus = getEmptyDirStatus(basePath); assertEquals("Must not be an empty dir: " + baseStatus, Tristate.FALSE, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java index 3c67e252e6e69..3e3330696f19c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java @@ -31,6 +31,7 @@ import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.contract.s3a.S3AContract; +import com.amazonaws.services.s3.model.S3ObjectSummary; import com.google.common.collect.Lists; import org.assertj.core.api.Assertions; import org.junit.Assume; @@ -560,24 +561,23 @@ public void testInconsistentS3ClientDeletes() throws Throwable { + " paths"); ListObjectsV2Result postDeleteDelimited = listObjectsV2(fs, key, "/"); - assertListSizeEqual( + boolean stripTombstones = false; + assertObjectSummariesEqual( "InconsistentAmazonS3Client added back objects incorrectly " + "in a non-recursive listing", - preDeleteDelimited.getObjectSummaries(), - postDeleteDelimited.getObjectSummaries()); + preDeleteDelimited, postDeleteDelimited, + stripTombstones); assertListSizeEqual("InconsistentAmazonS3Client added back prefixes incorrectly " + "in a non-recursive listing", preDeleteDelimited.getCommonPrefixes(), - postDeleteDelimited.getCommonPrefixes() - ); + postDeleteDelimited.getCommonPrefixes()); LOG.info("Executing Deep listing"); ListObjectsV2Result postDeleteUndelimited = listObjectsV2(fs, key, null); - assertListSizeEqual("InconsistentAmazonS3Client added back objects incorrectly " + + assertObjectSummariesEqual("InconsistentAmazonS3Client added back objects incorrectly " + "in a recursive listing", - preDeleteUndelimited.getObjectSummaries(), - postDeleteUndelimited.getObjectSummaries() - ); + preDeleteUndelimited, postDeleteUndelimited, + stripTombstones); assertListSizeEqual("InconsistentAmazonS3Client added back prefixes incorrectly " + "in a recursive listing", @@ -586,6 +586,23 @@ public void testInconsistentS3ClientDeletes() throws Throwable { ); } + private void assertObjectSummariesEqual(final String message, + final ListObjectsV2Result expected, + final ListObjectsV2Result actual, + final boolean stripTombstones) { + assertCollectionsEqual( + message, + stringify(expected.getObjectSummaries(), stripTombstones), + stringify(actual.getObjectSummaries(), stripTombstones)); + } + + List stringify(List objects, boolean stripTombstones) { + return objects.stream() + .filter(s -> !stripTombstones || !(s.getKey().endsWith("/"))) + .map(s -> s.getKey()) + .collect(Collectors.toList()); + } + /** * Require the v2 S3 list API. */ @@ -682,6 +699,22 @@ public void testListingReturnsVersionMetadata() throws Throwable { versionId, locatedFileStatus.getVersionId()); } + /** + * Assert that the two collections match using + * object equality of the elements within. + * @param message text for the assertion + * @param expected expected list + * @param actual actual list + * @param type of list + */ + private void assertCollectionsEqual(String message, + Collection expected, + Collection actual) { + Assertions.assertThat(actual) + .describedAs(message) + .containsExactlyInAnyOrderElementsOf(expected); + } + /** * Assert that the two list sizes match; failure message includes the lists. * @param message text for the assertion diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java index ccd687102e807..7514f5ce1d089 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java @@ -129,6 +129,8 @@ public class ITestDirectoryMarkerListing extends AbstractS3ATestBase { private Path markerPeer; + private String markerPeerKey; + @Override protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); @@ -153,6 +155,17 @@ public void setup() throws Exception { createTestObjects(new Path(methodPath(), "base")); } + @Override + public void teardown() throws Exception { + if (s3client != null) { + delete(markerKey); + delete(markerKeySlash); + delete(markerPeerKey); + delete(fileKeyUnderMarker); + } + super.teardown(); + } + /** * Create the test objects under the given path, setting * various fields in the process. @@ -165,6 +178,7 @@ private void createTestObjects(final Path path) throws Exception { // peer path has the same initial name to make sure there // is no confusion there. markerPeer = new Path(basePath, MARKER_PEER); + markerPeerKey = fs.pathToKey(markerPeer); markerKey = fs.pathToKey(markerDir); markerKeySlash = markerKey + "/"; fileKeyUnderMarker = markerKeySlash + FILENAME; @@ -321,6 +335,31 @@ public void testCreateFileNoOverwrite() throws Throwable { assertTestObjectsExist(); } + /* + ================================================================= + Delete. + ================================================================= + */ + + @Test + public void testDelete() throws Throwable { + S3AFileSystem fs = getFileSystem(); + // a non recursive delete MUST fail because + // it is not empty + intercept(PathIsNotEmptyDirectoryException.class, () -> + fs.delete(markerDir, false)); + // file is still there + head(fileKeyUnderMarker); + + // recursive delete MUST succeed + fs.delete(markerDir, true); + // and the markers are gone + head404(fileKeyUnderMarker); + head404(markerKeySlash); + // just for completeness + fs.delete(basePath, true); + } + /* ================================================================= Rename. @@ -371,18 +410,17 @@ public void testRenameBase() throws Throwable { /** * Rename under a marker. - * What does happen here? + * The marker must no longer exist. */ @Test public void testRenameUnderMarker() throws Throwable { describe("directory rename"); - Path base = methodPath(); S3AFileSystem fs = getFileSystem(); String name = "sourceFile"; - Path src = new Path(methodPath(), name); + Path src = new Path(basePath, name); String srcKey = toKey(src); - put(srcKey, "source"); - + put(srcKey, name); + head(srcKey); Path dest = markerDir; // renamed into the dest dir @@ -395,29 +433,31 @@ public void testRenameUnderMarker() throws Throwable { } } - /* - ================================================================= - Delete. - ================================================================= - */ - + /** + * This test creates an empty dir and renames it over the directory marker. + * If the dest was considered to be empty, the rename would fail. + */ @Test - public void testDelete() throws Throwable { + public void testRenameEmptyDirOverMarker() throws Throwable { + describe("rename an empty directory over the marker"); S3AFileSystem fs = getFileSystem(); - // a non recursive delete MUST fail because - // it is not empty - intercept(PathIsNotEmptyDirectoryException.class, () -> - fs.delete(markerDir, false)); - // file is still there - head(fileKeyUnderMarker); - - // recursive delete MUST succeed - fs.delete(markerDir, true); - // and the markers are gone - head404(fileKeyUnderMarker); - head404(markerKeySlash); - // just for completeness - fs.delete(basePath, true); + String name = "sourceDir"; + Path src = new Path(basePath, name); + fs.mkdirs(src); + assertIsDirectory(src); + String srcKey = toKey(src) + "/"; + head(srcKey); + Path dest = markerDir; + // renamed into the dest dir + assertFalse("rename(" + src + ", " + dest + ") should have failed", + getFileSystem().rename(src, dest)); + // source is still there + assertIsDirectory(src); + head(srcKey); + // and a non-recursive delete lets us verify it is considered + // an empty dir + assertDeleted(src, false); + assertTestObjectsExist(); } /* @@ -443,16 +483,28 @@ private void put(final String key, final String content) throws Exception { exec("PUT " + key, () -> s3client.putObject(bucket, key, content)); } - + /** + * Delete an object. + * @param key key + * @param content string + */ + private void delete(final String key) throws Exception { + exec("DELETE " + key, () -> { + s3client.deleteObject(bucket, key); + return "deleted " + key; + }); + } /** * Issue a HEAD request. * @param key - * @return the metadata + * @return a description of the object. */ - private ObjectMetadata head(final String key) throws Exception { - return exec("HEAD " + key, () -> + private String head(final String key) throws Exception { + ObjectMetadata md = exec("HEAD " + key, () -> s3client.getObjectMetadata(bucket, key)); + return String.format("Object %s of length %d", + key, md.getInstanceLength()); } /** @@ -599,7 +651,7 @@ private List dump(List l) { */ private void assertRenamed(final Path src, final Path dest) throws IOException { - assertTrue("rename(" + src + ", " + dest + ")", + assertTrue("rename(" + src + ", " + dest + ") failed", getFileSystem().rename(src, dest)); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java index bfd6de8779e2e..bcd636fb08c00 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java @@ -510,7 +510,7 @@ private MarkerTool.ScanResult markerTool( * @return the result */ @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") - private MarkerTool.ScanResult markerTool( + public static MarkerTool.ScanResult markerTool( final int exitCode, final FileSystem sourceFS, final Path path, From c3e2a6b3fec1d6954dd1b7d3c8787eee56fc5959 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Sat, 1 Aug 2020 13:45:48 +0100 Subject: [PATCH 12/20] HADOOP-13230 yetus warnings; audit on teardown The audit on teardown looks to see if there are surplus markers in the test method dir. It always runs, but logs failures @ warn I think I'll make it something you explicitly turn on with a config option, and which then fails the test if the scan finds surplus markers. Change-Id: Ida70cb5cd0692165c8b0fca45a41e6a2bee9860b --- .../fs/s3a/InconsistentAmazonS3Client.java | 3 ++- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 8 ++++++ .../fs/s3a/impl/OperationCallbacks.java | 1 - .../fs/s3a/s3guard/NullMetadataStore.java | 6 +++-- .../hadoop/fs/s3a/tools/MarkerTool.java | 24 ++++++++++++++++- .../fs/s3a/tools/MarkerToolOperations.java | 1 - .../s3a/tools/MarkerToolOperationsImpl.java | 3 ++- .../hadoop/fs/s3a/AbstractS3ATestBase.java | 21 +++++++++++++++ .../fs/s3a/ITestS3AFileOperationCost.java | 6 ++--- .../fs/s3a/ITestS3GuardListConsistency.java | 7 ++--- .../s3a/performance/AbstractS3ACostTest.java | 19 +++++++------- .../ITestDirectoryMarkerListing.java | 7 ++--- .../s3a/performance/ITestS3ADeleteCost.java | 26 +++++++++---------- .../s3a/performance/ITestS3ARenameCost.java | 14 +++++----- .../hadoop/fs/s3a/tools/ITestMarkerTool.java | 17 +++++++----- 15 files changed, 112 insertions(+), 51 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java index d8e5d9eb2a91c..4cb05ae9e6512 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java @@ -331,7 +331,8 @@ private boolean isDescendant(String parent, String child, boolean recursive) { Path actualParentPath = new Path(child).getParent(); Path expectedParentPath = new Path(parent); // children which are directory markers are excluded here - return actualParentPath.equals(expectedParentPath) && !child.endsWith("/"); + return actualParentPath.equals(expectedParentPath) + && !child.endsWith("/"); } } 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 2412f237295a4..92f5aec29c7dd 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 @@ -2415,6 +2415,14 @@ private DeleteObjectsResult removeKeysS3( boolean quiet) throws MultiObjectDeleteException, AmazonClientException, IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("Initiating delete operation for {} objects", + keysToDelete.size()); + for (DeleteObjectsRequest.KeyVersion key : keysToDelete) { + LOG.debug(" {} {}", key.getKey(), + key.getVersion() != null ? key.getVersion() : ""); + } + } DeleteObjectsResult result = null; if (keysToDelete.isEmpty()) { // exit fast if there are no keys to delete diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java index 61c04d503c7bb..0fcf6454c11fb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java @@ -18,7 +18,6 @@ package org.apache.hadoop.fs.s3a.impl; -import javax.annotation.Nullable; import java.io.IOException; import java.io.InterruptedIOException; import java.util.List; 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 d1dc37b0c73d1..722f42176ef2f 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 @@ -172,8 +172,10 @@ private static final class NullRenameTracker extends RenameTracker { private NullRenameTracker( final StoreContext storeContext, final Path source, - final Path dest, MetadataStore metadataStore) { - super("NullRenameTracker", storeContext, metadataStore, source, dest, null); + final Path dest, + MetadataStore metadataStore) { + super("NullRenameTracker", storeContext, metadataStore, source, dest, + null); } @Override diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index a62e894cabadf..f84b75ad3eec6 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -287,7 +287,7 @@ ScanResult execute( /** * Result of the scan operation. */ - static final class ScanResult { + public static final class ScanResult { /** * Exit code to return if an exception was not raised. @@ -553,4 +553,26 @@ public boolean isVerbose() { public void setVerbose(final boolean verbose) { this.verbose = verbose; } + + /** + * Execute the marker tool, with no checks on return codes. + * + * @param sourceFS filesystem to use + * @param path path to scan + * @param doPurge should markers be purged + * @param expectedMarkers number of markers expected + * @return the result + */ + @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") + public static MarkerTool.ScanResult execMarkerTool( + final FileSystem sourceFS, + final Path path, + final boolean doPurge, + final int expectedMarkers) throws IOException { + MarkerTool tool = new MarkerTool(sourceFS.getConf()); + tool.setVerbose(LOG.isDebugEnabled()); + + return tool.execute(sourceFS, path, doPurge, + expectedMarkers); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperations.java index ac2f8bca35b74..9ab7636d6c99f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperations.java @@ -26,7 +26,6 @@ import com.amazonaws.services.s3.model.DeleteObjectsResult; import com.amazonaws.services.s3.model.MultiObjectDeleteException; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.InvalidRequestException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperationsImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperationsImpl.java index 18fc51db7787f..d14bb6b1d8ebb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperationsImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperationsImpl.java @@ -56,7 +56,8 @@ public RemoteIterator listObjects(final Path path, } @Override - public DeleteObjectsResult removeKeys(final List keysToDelete, + public DeleteObjectsResult removeKeys( + final List keysToDelete, final boolean deleteFakeDir, final List undeletedObjectsOnFailure, final BulkOperationState operationState, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java index d94288dfc307f..42ca823de4983 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java @@ -25,11 +25,15 @@ import org.apache.hadoop.fs.contract.AbstractFSContractTestBase; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.contract.s3a.S3AContract; +import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; +import org.apache.hadoop.fs.s3a.tools.ITestMarkerTool; +import org.apache.hadoop.fs.s3a.tools.MarkerTool; import org.apache.hadoop.io.IOUtils; import org.junit.Before; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.FileNotFoundException; import java.io.IOException; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; @@ -62,6 +66,23 @@ public void setup() throws Exception { @Override public void teardown() throws Exception { Thread.currentThread().setName("teardown"); + // Perform an audit of the directory tree with the marker tool + S3AFileSystem fs = getFileSystem(); + if (fs != null && + fs.getDirectoryMarkerPolicy() == DirectoryPolicy.MarkerPolicy.Delete + && fs.isDirectory(methodPath())) { + try { + MarkerTool.ScanResult result = MarkerTool.execMarkerTool(fs, + methodPath(), true, 0); + if (result.getExitCode() != 0) { + LOG.warn("Marker Tool Exit Code indicated failure {}", result); + } + } catch (FileNotFoundException ignored) { + } catch (Exception e) { + LOG.warn("Marker Tool Failure", e); + } + } + super.teardown(); describe("closing file system"); IOUtils.closeStream(getFileSystem()); 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 820534e031ecd..1ad4380169cf6 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 @@ -129,8 +129,8 @@ public void testCostOfListFilesOnFile() throws Throwable { fs.listFiles(file, true), rawHeadList(GETFILESTATUS_SINGLE_FILE_H, LIST_LOCATED_STATUS_L), - authoritative(OBJECT_LIST_REQUESTS, 0), - nonauth(OBJECT_LIST_REQUESTS, + whenAuthoritative(OBJECT_LIST_REQUESTS, 0), + whenNonauth(OBJECT_LIST_REQUESTS, LIST_FILES_L)); } @@ -268,7 +268,7 @@ public void testCostOfCopyFromLocalFile() throws Throwable { verifyMetrics(() -> { s3a.copyFromLocalFile(false, true, localPath, remotePath); return "copy"; - }, + }, always(INVOCATION_COPY_FROM_LOCAL_FILE, 1), always(OBJECT_PUT_REQUESTS, 1), always(OBJECT_PUT_BYTES, len)); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java index 3e3330696f19c..0246b5415f18f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java @@ -574,8 +574,8 @@ public void testInconsistentS3ClientDeletes() throws Throwable { postDeleteDelimited.getCommonPrefixes()); LOG.info("Executing Deep listing"); ListObjectsV2Result postDeleteUndelimited = listObjectsV2(fs, key, null); - assertObjectSummariesEqual("InconsistentAmazonS3Client added back objects incorrectly " + - "in a recursive listing", + assertObjectSummariesEqual("InconsistentAmazonS3Client added back objects" + + " incorrectly in a recursive listing", preDeleteUndelimited, postDeleteUndelimited, stripTombstones); @@ -596,7 +596,8 @@ private void assertObjectSummariesEqual(final String message, stringify(actual.getObjectSummaries(), stripTombstones)); } - List stringify(List objects, boolean stripTombstones) { + List stringify(List objects, + boolean stripTombstones) { return objects.stream() .filter(s -> !stripTombstones || !(s.getKey().endsWith("/"))) .map(s -> s.getKey()) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java index 1f9ee1a800b64..d448c09f91d72 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java @@ -560,7 +560,7 @@ protected OperationCostValidator.ExpectedProbe always( * @param expected expected value. * @return the diff. */ - protected OperationCostValidator.ExpectedProbe raw( + protected OperationCostValidator.ExpectedProbe whenRraw( final Statistic stat, final int expected) { return probe(isRaw(), stat, expected); } @@ -571,7 +571,7 @@ protected OperationCostValidator.ExpectedProbe raw( * @param expected expected value. * @return the diff. */ - protected OperationCostValidator.ExpectedProbe guarded( + protected OperationCostValidator.ExpectedProbe whenGuarded( final Statistic stat, final int expected) { return probe(isGuarded(), stat, expected); @@ -583,7 +583,7 @@ protected OperationCostValidator.ExpectedProbe guarded( * @param expected expected value. * @return the diff. */ - protected OperationCostValidator.ExpectedProbe authoritative( + protected OperationCostValidator.ExpectedProbe whenAuthoritative( final Statistic stat, final int expected) { return probe(isAuthoritative(), stat, expected); @@ -595,31 +595,31 @@ protected OperationCostValidator.ExpectedProbe authoritative( * @param expected expected value. * @return the diff. */ - protected OperationCostValidator.ExpectedProbe nonauth( + protected OperationCostValidator.ExpectedProbe whenNonauth( final Statistic stat, final int expected) { return probe(isNonAuth(), stat, expected); } /** - * A metric diff which must hold when the fs is keeping markers + * A metric diff which must hold when the fs is keeping markers. * @param stat metric source * @param expected expected value. * @return the diff. */ - protected OperationCostValidator.ExpectedProbe keeping( + protected OperationCostValidator.ExpectedProbe whenKeeping( final Statistic stat, final int expected) { return probe(isKeepingMarkers(), stat, expected); } /** - * A metric diff which must hold when the fs is keeping markers + * A metric diff which must hold when the fs is keeping markers. * @param stat metric source * @param expected expected value. * @return the diff. */ - protected OperationCostValidator.ExpectedProbe deleting( + protected OperationCostValidator.ExpectedProbe whenDeleting( final Statistic stat, final int expected) { return probe(isDeleting(), stat, expected); @@ -635,7 +635,8 @@ protected void assertEmptyDirStatus(final S3AFileStatus status, Assertions.assertThat(status.isEmptyDirectory()) .describedAs(dynamicDescription(() -> "FileStatus says directory is not empty: " + status - + "\n" + ContractTestUtils.ls(getFileSystem(), status.getPath()))) + + "\n" + ContractTestUtils.ls( + getFileSystem(), status.getPath()))) .isEqualTo(expected); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java index 7514f5ce1d089..13c691e9c00c4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java @@ -244,7 +244,8 @@ public void testListFilesMarkerDirRecursive() throws Throwable { public void testListStatusBaseDirRecursive() throws Throwable { List statuses = toList( getFileSystem().listFiles(basePath, true)); - assertContainsExactlyStatusOfPaths(statuses, filePathUnderMarker, markerPeer); + assertContainsExactlyStatusOfPaths(statuses, filePathUnderMarker, + markerPeer); } @Test @@ -467,7 +468,7 @@ public void testRenameEmptyDirOverMarker() throws Throwable { */ /** - * Assert the test objects exist + * Assert the test objects exist. */ private void assertTestObjectsExist() throws Exception { head(fileKeyUnderMarker); @@ -579,7 +580,7 @@ private void assertIsFileUnderMarker(final FileStatus stat) { } /** - * Assert the status object refers to a path at the given name + * Assert the status object refers to a path at the given name. * @param path path * @param stat status object */ diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java index fba6aef0c2915..b897e5198af19 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java @@ -96,18 +96,18 @@ public void testDeleteFile() throws Throwable { // if deleting markers, look for the parent too probe(rawAndDeleting, OBJECT_METADATA_REQUESTS, FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H), - raw(OBJECT_LIST_REQUESTS, + whenRraw(OBJECT_LIST_REQUESTS, FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L), always(DIRECTORIES_DELETED, 0), always(FILES_DELETED, 1), // keeping: create no parent dirs or delete parents - keeping(DIRECTORIES_CREATED, 0), - keeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + whenKeeping(DIRECTORIES_CREATED, 0), + whenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), // deleting: create a parent and delete any of its parents - deleting(DIRECTORIES_CREATED, 1), - deleting(OBJECT_DELETE_REQUESTS, + whenDeleting(DIRECTORIES_CREATED, 1), + whenDeleting(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST + DELETE_MARKER_REQUEST) ); @@ -129,11 +129,11 @@ public void testDirMarkersSubdir() throws Throwable { }, always(DIRECTORIES_CREATED, 1), always(DIRECTORIES_DELETED, 0), - keeping(OBJECT_DELETE_REQUESTS, 0), - keeping(FAKE_DIRECTORIES_DELETED, 0), - deleting(OBJECT_DELETE_REQUESTS, DELETE_MARKER_REQUEST), + whenKeeping(OBJECT_DELETE_REQUESTS, 0), + whenKeeping(FAKE_DIRECTORIES_DELETED, 0), + whenDeleting(OBJECT_DELETE_REQUESTS, DELETE_MARKER_REQUEST), // delete all possible fake dirs above the subdirectory - deleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(subDir) - 1)); + whenDeleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(subDir) - 1)); } @Test @@ -154,11 +154,11 @@ public void testDirMarkersFileCreation() throws Throwable { always(DIRECTORIES_CREATED, 0), always(DIRECTORIES_DELETED, 0), // keeping: no delete operations. - keeping(OBJECT_DELETE_REQUESTS, 0), - keeping(FAKE_DIRECTORIES_DELETED, 0), + whenKeeping(OBJECT_DELETE_REQUESTS, 0), + whenKeeping(FAKE_DIRECTORIES_DELETED, 0), // delete all possible fake dirs above the file - deleting(OBJECT_DELETE_REQUESTS, 1), - deleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(srcDir))); + whenDeleting(OBJECT_DELETE_REQUESTS, 1), + whenDeleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(srcDir))); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java index 32821915070ce..fe5986adb43af 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java @@ -103,12 +103,12 @@ public void testRenameFileToDifferentDirectory() throws Throwable { always(DIRECTORIES_CREATED, 0), always(DIRECTORIES_DELETED, 0), // keeping: only the core delete operation is issued. - keeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), - keeping(FAKE_DIRECTORIES_DELETED, 0), + whenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + whenKeeping(FAKE_DIRECTORIES_DELETED, 0), // deleting: delete any fake marker above the destination. - deleting(OBJECT_DELETE_REQUESTS, + whenDeleting(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST + DELETE_MARKER_REQUEST), - deleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(destDir))); + whenDeleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(destDir))); assertIsFile(destFilePath); assertIsDirectory(srcDir); @@ -175,9 +175,9 @@ public void testCostOfRootRename() throws Throwable { // delete that destination file, assert only the file delete was issued verifyMetrics(() -> { - fs.delete(dest, false); - return "after fs.delete(/dest) " + getMetricSummary(); - }, + fs.delete(dest, false); + return "after fs.delete(/dest) " + getMetricSummary(); + }, always(DIRECTORIES_CREATED, 0), always(DIRECTORIES_DELETED, 0), always(FAKE_DIRECTORIES_DELETED, 0), diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java index bcd636fb08c00..b2861a1da273f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java @@ -39,6 +39,7 @@ import org.apache.hadoop.util.ExitUtil; import static org.apache.hadoop.fs.s3a.Constants.*; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; @@ -105,8 +106,12 @@ public void setup() throws Exception { @Override public void teardown() throws Exception { + // do this ourselves to avoid the marker checks in the superclass getting + // upset that even when the test FS delete markers, surplus markers are found + deleteTestDirInTeardown(); super.teardown(); - IOUtils.cleanupWithLogger(LOG, getKeepingFS(), getMixedFS(), getDeletingFS()); + IOUtils.cleanupWithLogger(LOG, getKeepingFS(), + getMixedFS(), getDeletingFS()); } @@ -509,17 +514,15 @@ private MarkerTool.ScanResult markerTool( * @param expectedMarkers number of markers expected * @return the result */ - @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") public static MarkerTool.ScanResult markerTool( final int exitCode, final FileSystem sourceFS, final Path path, final boolean doPurge, final int expectedMarkers) throws IOException { - MarkerTool tool = new MarkerTool(sourceFS.getConf()); - tool.setVerbose(LOG.isDebugEnabled()); - MarkerTool.ScanResult result = tool.execute(sourceFS, path, doPurge, + MarkerTool.ScanResult result = MarkerTool.execMarkerTool(sourceFS, path, + doPurge, expectedMarkers); Assertions.assertThat(result.getExitCode()) .describedAs("Exit code of marker(%s, %s, %d) -> %s", @@ -557,7 +560,9 @@ void verifyRenamed(final Path dest, * @throws Exception any exception */ protected int run(String... args) throws Exception { - return S3GuardTool.run(getConfiguration(), args); + Configuration conf = new Configuration(getConfiguration()); + disableFilesystemCaching(conf); + return S3GuardTool.run(conf, args); } /** From 5273e87f60ec593a30315316ec110487bd6c5429 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 3 Aug 2020 20:59:24 +0100 Subject: [PATCH 13/20] HADOOP-13230 Test Tuning, docs, audit Tuning tests by factoring out list/head contants into a class to describe it for different FS operations, with aggregation. Should make it much easier to maintain asserts from now on. Some tweaking of marker tool -mainly just code review. Audit of paths when -Dfs.s3a.directory.marker.audit=true Production code seems complete Change-Id: Id8052846422eed7154bdda559535b10773775b47 --- hadoop-tools/hadoop-aws/pom.xml | 9 + .../apache/hadoop/fs/s3a/S3AFileSystem.java | 4 +- .../hadoop/fs/s3a/impl/DirMarkerTracker.java | 22 +- .../hadoop/fs/s3a/impl/RenameOperation.java | 5 +- .../hadoop/fs/s3a/tools/MarkerTool.java | 38 ++-- .../tools/hadoop-aws/directory_markers.md | 85 +++++--- .../site/markdown/tools/hadoop-aws/index.md | 1 + .../site/markdown/tools/hadoop-aws/s3guard.md | 16 +- .../site/markdown/tools/hadoop-aws/testing.md | 43 ++++ .../hadoop/fs/s3a/AbstractS3ATestBase.java | 43 ++-- .../fs/s3a/ITestS3AFileOperationCost.java | 132 ++++++------ .../hadoop/fs/s3a/S3ATestConstants.java | 6 + .../apache/hadoop/fs/s3a/S3ATestUtils.java | 2 + .../s3a/performance/AbstractS3ACostTest.java | 191 ++++++++--------- .../fs/s3a/performance/HeadListCosts.java | 125 ----------- .../ITestDirectoryMarkerListing.java | 21 +- .../s3a/performance/ITestS3ADeleteCost.java | 59 ++++- .../s3a/performance/ITestS3ARenameCost.java | 40 ++-- .../fs/s3a/performance/OperationCost.java | 202 ++++++++++++++++++ .../performance/OperationCostValidator.java | 21 +- 20 files changed, 661 insertions(+), 404 deletions(-) delete mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/HeadListCosts.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index f4b8ee34b35ed..dc29d66a6ce2f 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -52,6 +52,9 @@ 200000 + + false + @@ -124,7 +127,9 @@ ${fs.s3a.scale.test.huge.filesize} ${fs.s3a.scale.test.huge.partitionsize} ${fs.s3a.scale.test.timeout} + ${fs.s3a.directory.marker.retention} + ${fs.s3a.directory.marker.audit} @@ -218,7 +223,9 @@ ${fs.s3a.s3guard.test.enabled} ${fs.s3a.s3guard.test.implementation} ${fs.s3a.s3guard.test.authoritative} + ${fs.s3a.directory.marker.retention} + ${fs.s3a.directory.marker.audit} @@ -273,7 +280,9 @@ ${fs.s3a.s3guard.test.enabled} ${fs.s3a.s3guard.test.implementation} ${fs.s3a.s3guard.test.authoritative} + ${fs.s3a.directory.marker.retention} + ${fs.s3a.directory.marker.audit} ${fs.s3a.scale.test.timeout} 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 92f5aec29c7dd..c881eb1a2e19b 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 @@ -4033,8 +4033,8 @@ public long getDefaultBlockSize() { * Get the directory marker policy of this filesystem. * @return the marker policy. */ - public DirectoryPolicy.MarkerPolicy getDirectoryMarkerPolicy() { - return directoryPolicy.getMarkerPolicy(); + public DirectoryPolicy getDirectoryMarkerPolicy() { + return directoryPolicy; } @Override diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java index 11341e352cafd..cf6586822271d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java @@ -58,6 +58,8 @@ public class DirMarkerTracker { private final Map surplusMarkers = new TreeMap<>(); + private final Path basePath; + /** * last parent directory checked. */ @@ -72,6 +74,24 @@ public class DirMarkerTracker { private int markersFound; + /** + * Construct. + * Base path is currently only used for information rather than validating + * paths supplied in other mathods. + * @param basePath base path of track + */ + public DirMarkerTracker(final Path basePath) { + this.basePath = basePath; + } + + /** + * Get the base path of the tracker. + * @return the path + */ + public Path getBasePath() { + return basePath; + } + /** * A marker has been found; this may or may not be a leaf. * Trigger a move of all markers above it into the surplus map. @@ -209,7 +229,7 @@ public static final class Marker { */ private final S3ALocatedFileStatus status; - public Marker(final Path path, + private Marker(final Path path, final String key, final S3ALocatedFileStatus status) { this.path = path; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java index 53d10842ad840..37168c5417f2a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java @@ -378,11 +378,12 @@ protected void recursiveDirectoryRename() throws IOException { // TODO: dir marker policy doesn't always need to do this. callbacks.deleteObjectAtPath(destStatus.getPath(), dstKey, false, null); } + Path parentPath = storeContext.keyToPath(srcKey); + // Track directory markers so that we know which leaf directories need to be // recreated - DirMarkerTracker dirMarkerTracker = new DirMarkerTracker(); + DirMarkerTracker dirMarkerTracker = new DirMarkerTracker(parentPath); - Path parentPath = storeContext.keyToPath(srcKey); final RemoteIterator iterator = callbacks.listFilesAndEmptyDirectories(parentPath, sourceStatus, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index f84b75ad3eec6..44866351e0bf6 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -38,6 +38,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.s3a.Retries; import org.apache.hadoop.fs.s3a.S3AFileStatus; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; @@ -253,10 +254,12 @@ ScanResult execute( final int expectedMarkerCount) throws IOException { S3AFileSystem fs = bindFilesystem(sourceFS); + + // extract the callbacks needed for the rest of the work storeContext = fs.createStoreContext(); operations = fs.createMarkerToolOperations(); - - DirectoryPolicy.MarkerPolicy policy = fs.getDirectoryMarkerPolicy(); + DirectoryPolicy.MarkerPolicy policy = fs.getDirectoryMarkerPolicy() + .getMarkerPolicy(); println(out, "The store's directory marker policy is \"%s\"", policy); if (policy == DirectoryPolicy.MarkerPolicy.Authoritative) { @@ -265,7 +268,7 @@ ScanResult execute( .getTrimmed(AUTHORITATIVE_PATH, "unset"); println(out, "Authoritative path list is %s", authPath); } - // validate the FS + // initial safety check: does the path exist? try { getFilesystem().getFileStatus(path); } catch (UnknownStoreException ex) { @@ -279,8 +282,8 @@ ScanResult execute( "Not found: " + path, ex); } - ScanResult result = once("action", path.toString(), - () -> scan(path, doPurge, expectedMarkerCount)); + + ScanResult result = scan(path, doPurge, expectedMarkerCount); return result; } @@ -304,6 +307,9 @@ public static final class ScanResult { */ private MarkerPurgeSummary purgeSummary; + private ScanResult() { + } + @Override public String toString() { return "ScanResult{" + @@ -330,7 +336,7 @@ public MarkerPurgeSummary getPurgeSummary() { } /** - * Do the scan. + * Do the scan/purge. * @param path path to scan. * @param doPurge purge? * @param expectedMarkerCount expected marker count @@ -338,16 +344,16 @@ public MarkerPurgeSummary getPurgeSummary() { * @throws IOException IO failure * @throws ExitUtil.ExitException explicitly raised failure */ + @Retries.RetryTranslated private ScanResult scan( final Path path, final boolean doPurge, final int expectedMarkerCount) throws IOException, ExitUtil.ExitException { - // initial safety check: does the path exist ScanResult result = new ScanResult(); - DirMarkerTracker tracker = new DirMarkerTracker(); + DirMarkerTracker tracker = new DirMarkerTracker(path); result.tracker = tracker; try (DurationInfo ignored = new DurationInfo(LOG, "marker scan %s", path)) { @@ -398,8 +404,7 @@ private ScanResult scan( int deletePageSize = storeContext.getConfiguration() .getInt(BULK_DELETE_PAGE_SIZE, BULK_DELETE_PAGE_SIZE_DEFAULT); - result.purgeSummary = purgeMarkers(tracker, - deletePageSize); + result.purgeSummary = purgeMarkers(tracker, deletePageSize); } result.exitCode = EXIT_SUCCESS; return result; @@ -418,10 +423,12 @@ private String suffix(final int size) { * Scan a directory tree. * @param path path to scan * @param tracker tracker to update - * @throws IOException + * @throws IOException IO failure */ + @Retries.RetryTranslated private void scanDirectoryTree(final Path path, final DirMarkerTracker tracker) throws IOException { + RemoteIterator listing = operations .listObjects(path, storeContext.pathToKey(path)); while (listing.hasNext()) { @@ -494,10 +501,9 @@ long getTotalDeleteRequestDuration() { * @param tracker tracker with the details * @param deletePageSize page size of deletes * @return summary - * @throws MultiObjectDeleteException - * @throws AmazonClientException - * @throws IOException + * @throws IOException IO failure */ + @Retries.RetryTranslated private MarkerPurgeSummary purgeMarkers(DirMarkerTracker tracker, int deletePageSize) throws MultiObjectDeleteException, AmazonClientException, IOException { @@ -535,7 +541,9 @@ private MarkerPurgeSummary purgeMarkers(DirMarkerTracker tracker, end); List undeleted = new ArrayList<>(); OperationDuration duration = new OperationDuration(); - operations.removeKeys(page, true, undeleted, null, false); + once("Remove S3 Keys", + tracker.getBasePath().toString(), () -> + operations.removeKeys(page, true, undeleted, null, false)); duration.finished(); summary.deleteRequests++; summary.totalDeleteRequestDuration += duration.value(); diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md index b3d8b94f7e9f7..d6f42736c7b0f 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md @@ -12,9 +12,9 @@ limitations under the License. See accompanying LICENSE file. --> -# Altering the S3A Directory Marker Retention Policy +# Altering the S3A Directory Marker Behavior -## Critical: this is not backwards compatible! +## Critical: this is not backwards compatible! This document shows how the performance of S3 IO, especially applications writing many files (hive) or working with versioned S3 buckets can @@ -22,18 +22,17 @@ increase performance by changing the S3A directory marker retention policy. Changing the policy from the default value, `"delete"` _is not backwards compatible_. -Older versions of Hadoop will - Versions of Hadoop which are incompatible with other marker retention policies - -| Branch | Incompatible releases | Future Fix Planned? | -|---------|-----------------------|--------------------| -| Hadoop 2.x | All releases | WONTFIX | -| Hadoop 3.0 | All releases | WONTFIX | -| Hadoop 3.1 | All releases | Yes | -| Hadoop 3.2 | All releases | Yes | -| Hadoop 3.3 | 3.3.0 | Yes | +------------------------------------------------------- +| Branch | Compatible Since | Future Fix Planned? | +|------------|------------------|---------------------| +| Hadoop 2.x | | NO | +| Hadoop 3.0 | | NO | +| Hadoop 3.1 | | Yes | +| Hadoop 3.2 | | Yes | +| Hadoop 3.3 | 3.3.1 | Done | +------------------------------------------------------- External Hadoop-based applications should also be assumed to be incompatible unless otherwise stated/known. @@ -43,9 +42,9 @@ It is only safe change the directory marker policy if the following 1. You know exactly which applications are writing to and reading from (including backing up) an S3 bucket. -2. You know all applications which read data from the bucket are listed as compatible. +2. You know all applications which read data from the bucket are as compatible. -### Applications backing up data MUST ALSO be based on compatible products. +### Applications backing up data. It is not enough to have a version of Apache Hadoop which is compatible, any application which backs up an S3 bucket or copies elsewhere must have an S3 @@ -54,13 +53,19 @@ distcp is used, it _must_ be from a compatible hadoop version. ### How will incompatible applications/versions fail? -Applications using an incompatible version of the S3A connector will mistake directories -containing data for empty directories. This means that +Applications using an incompatible version of the S3A connector will mistake +directories containing data for empty directories. This means that * Listing directories/directory trees may exclude files which exist. * Queries across the data will miss data files. * Renaming a directory to a new location may exclude files underneath. +### If an application has updated a directory tree incompatibly-- what can be done? + +There's a tool on the hadoop command line, [marker tool](#marker-tool) which can audit +a bucket/path for markers, and clean up any which were found. +It can be used to make a bucket compatible with older applications. + Now that this is all clear, let's explain the problem. @@ -91,7 +96,7 @@ file tree" in which files are always stored in "directories" The S3A connector mocks this entire metaphor by grouping all objects which have the same prefix as if they are in the same directory tree. -If there are two objects Cand `a/b/file2` then S3A pretends that there is a +If there are two objects `a/b/file1` and `a/b/file2` then S3A pretends that there is a directory `/a/b` containing two files `file1` and `file2`. The directory itself does not exist. @@ -100,7 +105,7 @@ There's a bit of a complication here. #### What does `mkdirs()` do? -1. In HDFS and other "real" filesystems, when mkdirs() is invoked on a path +1. In HDFS and other "real" filesystems, when `mkdirs()` is invoked on a path whose parents are all directories, then an _empty directory_ is created. 1. This directory can be probed for "it exists" and listed (an empty list is @@ -110,13 +115,12 @@ returned) Lots of code contains a big assumption here: after you create a directory it -exists. They also assume that after files in a directory are delted, the +exists. They also assume that after files in a directory are deleted, the directory still exists. Given filesystem mimics directories just by aggregating objects which share a prefix, how can you have empty directories? - The original Hadoop `s3n://` connector created a Directory Marker -any path ending in `_$folder$` was considered to be a sign that a directory existed. A call to `mkdir(s3n://bucket/a/b)` would create a new marker object `a/b_$folder$` . @@ -130,7 +134,7 @@ recreated. And, historically, When a path is listed, if a marker to that path is found, *it has been interpreted as an empty directory.* -## The Problem +## Scale issues related to directory markers Creating, deleting and the listing directory markers adds overhead and can slow down applications. @@ -138,9 +142,9 @@ down applications. Whenever a file is created we have to delete any marker which could exist in parent directory _or any parent paths_. Rather than do a sequence of probes for parent markers existing, the connector issues a single request to S3 to delete -all parents. For example, if a file `/a/b/file1` is created, a `DELETE` request -containing the keys `/a/` and `/a/b/` is issued. If no marker exists, this is -nominally a no-op. +all parents. For example, if a file `/a/b/file1` is created, a multi-object +`DELETE` request containing the keys `/a/` and `/a/b/` is issued. +If no markers exists, this is harmless. When a file is deleted, a check for the parent directory continuing to exist (i.e. are there sibling files/directories?), and if not a marker is created. @@ -176,7 +180,7 @@ For a symptom of this, see HADOOP-16829, The tombstone markers have follow-on consequences -it makes listings slower. This can have adverse effects on those large directories, again. -## Solutions +## How to avoid marker-related problems. ### Presto: there are no directories @@ -186,13 +190,13 @@ empty listing. That is:; by default, every path is an empty directory. Provided no code probes for a directory existing and fails if it is there, this is very efficient. That's a big requirement however, -one Presto can pull off -because they know how their file uses +because they know how their file uses data in S3. ### Hadoop 3.3.1+: marker deletion is now optional -## Controlling marker retention with `fs.s3a.directory.marker.retention` +## Controlling marker retention with `fs.s3a.directory.marker.retention` There is now an option `fs.s3a.directory.marker.retention` which controls how markers are managed when new files are created @@ -206,7 +210,8 @@ not deleted. This is *not* backwards compatible *New* `authoritative`: directory markers are deleted _except for files created in "authoritative" directories_. This is backwards compatible _outside -authoritative directories_. +authoritative directories_. + Until now, the notion of an "authoritative" directory has only been used as a performance optimization for deployments where it is known that all Applications are using the same S3Guard metastore @@ -214,12 +219,26 @@ when writing and reading data. In such a deployment, if it is also known that all applications are using a compatible version of the s3a connector, then they can switch to the higher-performance mode for those specific directories. - Only the default setting, `fs.s3a.directory.marker.retention = delete` is compatible with existing Hadoop releases. +## Directory Markers and S3Guard + +Applications which interact with S3A in S3A clients with S3Guard enabled still +create and delete markers. There's no attempt to skip operations, such as by having +`mkdirs() `create entries in the DynamoDB table but not the store. +Having the client always update S3 ensures that other applications and clients +do (eventually) see the changes made by the "guarded" application. + +When S3Guard is configured to treat some directories as [Authoritative](s3guard.html#authoritative) +then an S3A connector with a retention policy of `fs.s3a.directory.marker.retention` of +`authoritative` will omit deleting markers in authoritative directories. -## The marker tool: 'hadoop s3guard marker' +*Note* there may be further changes in directory semantics in "authoritative mode"; +only use in managed applications where all clients are using the same version of +hadoop, and configured consistently. + +## The marker tool:`hadoop s3guard markers` The marker tool aims to help migration by scanning/auditing directory trees for surplus markers, and for optionally deleting them. @@ -235,6 +254,7 @@ markers [-verbose] [-expected ] (audit || report || clean) ``` ### `markers report` + Scan the path and simply report on the markers found. ### `markers audit` @@ -243,7 +263,7 @@ Audit the path and fail if any markers were found. ### `markers clean` -The `markers clean` command will clean the directory tree +The `markers clean` command will clean the directory tree of all surplus markers ``` > hadoop s3guard markers clean s3a://ireland-bucket/ @@ -253,3 +273,6 @@ The `markers clean` command will clean the directory tree No surplus directory markers were found under s3a://ireland-bucket/ ``` +The `markers clean` command _does not_ delete markers above empty directories -only those which have +files underneath. If invoked on a path, it will clean up the directory tree into a state +where it is safe for older versions of Hadoop to interact with. \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index 964bda49dd069..7ad0ad1c5e8d6 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -33,6 +33,7 @@ See also: * [Working with IAM Assumed Roles](./assumed_roles.html) * [S3A Delegation Token Support](./delegation_tokens.html) * [S3A Delegation Token Architecture](delegation_token_architecture.html). +* [Altering the S3A Directory Marker Behavior](directory_markers.html). * [Testing](./testing.html) ## Overview 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 5754f0b5dfdd8..b60d54622ed20 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 @@ -113,7 +113,19 @@ Currently the only Metadata Store-independent setting, besides the implementation class above, are the *allow authoritative* and *fail-on-error* flags. -#### Allow Authoritative +#### Authoritative S3Guard + +Authoritative S3Guard is a complicated configuration which delivers performance +at the expense of being unsafe for other applications to use the same directory +tree/bucket unless configured consistently. + +It can also be used to support [directory marker retention](directory_markers.html) +in higher-performance but non-backwards-compatible modes. + +Most deployments do not use this setting -it is ony used in deployments where +specific parts of a bucket (e.g. Apache Hive managed tables) are known to +have exclusive access by a single application (Hive) and other tools/applications +from exactly the same Hadoop release. The _authoritative_ expression in S3Guard is present in two different layers, for two different reasons: @@ -178,7 +190,7 @@ recommended that you leave the default setting here: false ``` -. + Note that a MetadataStore MAY persist this bit in the directory listings. (Not MUST). diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md index 5629dab21ff24..e9730444f3a9a 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md @@ -324,6 +324,49 @@ Once a bucket is converted to being versioned, it cannot be converted back to being unversioned. +## Testing Different Marker Retention Policy + +Hadoop supports [different policies for directory marker retention](directory_markers.html) +-essentially the classic "delete" and the higher-performance "keep" options; "authoritative" +is just "keep" restricted to a part of the bucket. + +Example: test with `markers=delete` + +``` +mvn verify -Dparallel-tests -DtestsThreadCount=4 -Dmarkers=delete +``` + +Example: test with `markers=keep` + +``` +mvn verify -Dparallel-tests -DtestsThreadCount=4 -Dmarkers=keep +``` + +Example: test with `markers=authoritative` + +``` +mvn verify -Dparallel-tests -DtestsThreadCount=4 -Dmarkers=authoritative +``` + +This final option is of limited use unless paths in the bucket have actually been configured to be +of mixed status; unless anything is set up then the outcome should equal that of "delete" + +### Enabling auditing of markers + +To enable an audit of the output directory of every test suite, +enable the option `fs.s3a.directory.marker.audit` + +``` +-Dfs.s3a.directory.marker.audit=true +``` + +When set, if the marker policy is to delete markers under the test output directory, then +the marker tool audit command will be run. This will fail if a marker was found. + +This adds extra overhead to every operation, but helps verify that the connector is +not keeping markers where it needs to be deleting them -and hence backwards compatibility +is maintained. + ## Scale Tests There are a set of tests designed to measure the scalability and performance diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java index 42ca823de4983..afdd2a4e82f13 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java @@ -25,11 +25,8 @@ import org.apache.hadoop.fs.contract.AbstractFSContractTestBase; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.contract.s3a.S3AContract; -import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; -import org.apache.hadoop.fs.s3a.tools.ITestMarkerTool; import org.apache.hadoop.fs.s3a.tools.MarkerTool; import org.apache.hadoop.io.IOUtils; -import org.junit.Before; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,6 +36,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestDynamoTablePrefix; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBool; /** * An extension of the contract test base set up for S3A tests. @@ -67,19 +65,27 @@ public void setup() throws Exception { public void teardown() throws Exception { Thread.currentThread().setName("teardown"); // Perform an audit of the directory tree with the marker tool + // if logging at debug S3AFileSystem fs = getFileSystem(); - if (fs != null && - fs.getDirectoryMarkerPolicy() == DirectoryPolicy.MarkerPolicy.Delete - && fs.isDirectory(methodPath())) { - try { - MarkerTool.ScanResult result = MarkerTool.execMarkerTool(fs, - methodPath(), true, 0); - if (result.getExitCode() != 0) { - LOG.warn("Marker Tool Exit Code indicated failure {}", result); + if (fs != null) { + + boolean audit = getTestPropertyBool(fs.getConf(), + DIRECTORY_MARKER_AUDIT, false); + Path methodPath = methodPath(); + if (audit + && !fs.getDirectoryMarkerPolicy().keepDirectoryMarkers(methodPath) + && fs.isDirectory(methodPath)) { + try { + MarkerTool.ScanResult result = MarkerTool.execMarkerTool(fs, + methodPath, true, 0); + + if (result.getExitCode() != 0) { + fail("Audit of " + methodPath + " failed: " + result); + } + } catch (FileNotFoundException ignored) { + } catch (Exception e) { + LOG.warn("Marker Tool Failure", e); } - } catch (FileNotFoundException ignored) { - } catch (Exception e) { - LOG.warn("Marker Tool Failure", e); } } @@ -88,15 +94,6 @@ public void teardown() throws Exception { IOUtils.closeStream(getFileSystem()); } - @Before - public void nameThread() { - Thread.currentThread().setName("JUnit-" + getMethodName()); - } - - protected String getMethodName() { - return methodName.getMethodName(); - } - @Override protected int getTestTimeoutMillis() { return S3A_TEST_TIMEOUT; 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 1ad4380169cf6..28f71f617de17 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 @@ -43,7 +43,7 @@ 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.fs.s3a.performance.HeadListCosts.*; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.*; import static org.apache.hadoop.test.GenericTestUtils.getTestDir; /** @@ -88,9 +88,10 @@ public void testCostOfLocatedFileStatusOnFile() throws Throwable { Path file = file(methodPath()); S3AFileSystem fs = getFileSystem(); verifyMetrics(() -> fs.listLocatedStatus(file), - rawHeadList(FILESTATUS_FILE_PROBE_H, LIST_LOCATED_STATUS_L), - authHeadList(0, LIST_LOCATED_STATUS_L), - nonauthHeadList(0, LIST_LOCATED_STATUS_L)); + whenRaw(FILE_STATUS_FILE_PROBE + .plus(LIST_LOCATED_STATUS_LIST_OP)), + whenAuthoritative(LIST_LOCATED_STATUS_LIST_OP), + whenNonauth(LIST_LOCATED_STATUS_LIST_OP));; } @Test @@ -100,10 +101,10 @@ public void testCostOfListLocatedStatusOnEmptyDir() throws Throwable { S3AFileSystem fs = getFileSystem(); verifyMetrics(() -> fs.listLocatedStatus(dir), - rawHeadList(FILESTATUS_FILE_PROBE_H, - LIST_LOCATED_STATUS_L + GETFILESTATUS_DIR_L), - authHeadList(0, 0), - nonauthHeadList(0, LIST_LOCATED_STATUS_L)); + whenRaw(LIST_LOCATED_STATUS_LIST_OP + .plus(GET_FILE_STATUS_ON_EMPTY_DIR)), + whenAuthoritative(NO_IO), + whenNonauth(LIST_LOCATED_STATUS_LIST_OP)); } @Test @@ -114,9 +115,9 @@ public void testCostOfListLocatedStatusOnNonEmptyDir() throws Throwable { Path file = file(new Path(dir, "file.txt")); verifyMetrics(() -> fs.listLocatedStatus(dir), - rawHeadList(0, LIST_LOCATED_STATUS_L), - authHeadList(0, 0), - nonauthHeadList(0, LIST_LOCATED_STATUS_L)); + whenRaw(LIST_LOCATED_STATUS_LIST_OP), + whenAuthoritative(NO_IO), + whenNonauth(LIST_LOCATED_STATUS_LIST_OP)); } @Test @@ -127,11 +128,10 @@ public void testCostOfListFilesOnFile() throws Throwable { touch(fs, file); verifyMetrics(() -> fs.listFiles(file, true), - rawHeadList(GETFILESTATUS_SINGLE_FILE_H, - LIST_LOCATED_STATUS_L), - whenAuthoritative(OBJECT_LIST_REQUESTS, 0), - whenNonauth(OBJECT_LIST_REQUESTS, - LIST_FILES_L)); + whenRaw(LIST_LOCATED_STATUS_LIST_OP + .plus(GET_FILE_STATUS_ON_FILE)), + whenAuthoritative(NO_IO), + whenNonauth(LIST_LOCATED_STATUS_LIST_OP)); } @Test @@ -143,10 +143,10 @@ public void testCostOfListFilesOnEmptyDir() throws Throwable { fs.mkdirs(dir); verifyMetrics(() -> fs.listFiles(dir, true), - rawHeadList(GETFILESTATUS_EMPTY_DIR_H, - LIST_FILES_L + GETFILESTATUS_EMPTY_DIR_L), - authHeadList(0, 0), - nonauthHeadList(0, LIST_FILES_L)); + whenRaw(LIST_FILES_LIST_OP + .plus(GET_FILE_STATUS_ON_EMPTY_DIR)), + whenAuthoritative(NO_IO), + whenNonauth(LIST_FILES_LIST_OP)); } @Test @@ -159,9 +159,9 @@ public void testCostOfListFilesOnNonEmptyDir() throws Throwable { touch(fs, file); verifyMetrics(() -> fs.listFiles(dir, true), - rawHeadList(0, LIST_FILES_L), - authHeadList(0, 0), - nonauthHeadList(0, LIST_FILES_L)); + whenRaw(LIST_FILES_LIST_OP), + whenAuthoritative(NO_IO), + whenNonauth(LIST_FILES_LIST_OP)); } @Test @@ -171,10 +171,8 @@ public void testCostOfListFilesOnNonExistingDir() throws Throwable { S3AFileSystem fs = getFileSystem(); verifyMetricsIntercepting(FileNotFoundException.class, "", () -> fs.listFiles(dir, true), - rawHeadList(GETFILESTATUS_FNFE_H, - GETFILESTATUS_FNFE_L + LIST_FILES_L) - - ); + whenRaw(LIST_FILES_LIST_OP + .plus(GET_FILE_STATUS_FNFE))); } @Test @@ -183,33 +181,27 @@ public void testCostOfGetFileStatusOnFile() throws Throwable { Path simpleFile = file(methodPath()); S3AFileStatus status = verifyRawInnerGetFileStatus(simpleFile, true, StatusProbeEnum.ALL, - GETFILESTATUS_SINGLE_FILE_H, - GETFILESTATUS_SINGLE_FILE_L); + GET_FILE_STATUS_ON_FILE); assertTrue("not a file: " + status, status.isFile()); } - @Test public void testCostOfGetFileStatusOnEmptyDir() throws Throwable { describe("performing getFileStatus on an empty directory"); Path dir = dir(methodPath()); S3AFileStatus status = verifyRawInnerGetFileStatus(dir, true, StatusProbeEnum.ALL, - GETFILESTATUS_MARKER_H, - GETFILESTATUS_EMPTY_DIR_L); + GET_FILE_STATUS_ON_DIR_MARKER); assertSame("not empty: " + status, Tristate.TRUE, status.isEmptyDirectory()); // but now only ask for the directories and the file check is skipped. verifyRawInnerGetFileStatus(dir, false, StatusProbeEnum.DIRECTORIES, - FILESTATUS_DIR_PROBE_H, - GETFILESTATUS_EMPTY_DIR_L); + FILE_STATUS_DIR_PROBE); // now look at isFile/isDir against the same entry - isDir(dir, true, FILESTATUS_DIR_PROBE_H, - GETFILESTATUS_EMPTY_DIR_L); - isFile(dir, false, - FILESTATUS_FILE_PROBE_H, FILESTATUS_FILE_PROBE_L); + isDir(dir, true, FILE_STATUS_DIR_PROBE); + isFile(dir, false, FILE_STATUS_FILE_PROBE); } @Test @@ -217,8 +209,7 @@ public void testCostOfGetFileStatusOnMissingFile() throws Throwable { describe("performing getFileStatus on a missing file"); interceptRawGetFileStatusFNFE(methodPath(), false, StatusProbeEnum.ALL, - GETFILESTATUS_FNFE_H, - GETFILESTATUS_FNFE_L); + GET_FILE_STATUS_FNFE); } @Test @@ -227,11 +218,9 @@ public void testIsDirIsFileMissingPath() throws Throwable { Path path = methodPath(); // now look at isFile/isDir against the same entry isDir(path, false, - FILESTATUS_DIR_PROBE_H, - FILESTATUS_DIR_PROBE_L); + FILE_STATUS_DIR_PROBE); isFile(path, false, - FILESTATUS_FILE_PROBE_H, - FILESTATUS_FILE_PROBE_L); + FILE_STATUS_FILE_PROBE); } @Test @@ -240,8 +229,8 @@ public void testCostOfGetFileStatusOnNonEmptyDir() throws Throwable { Path dir = dir(methodPath()); file(new Path(dir, "simple.txt")); S3AFileStatus status = verifyRawInnerGetFileStatus(dir, true, - StatusProbeEnum.ALL, - GETFILESTATUS_DIR_H, GETFILESTATUS_DIR_L); + StatusProbeEnum.ALL, + GET_FILE_STATUS_ON_DIR); assertEmptyDirStatus(status, Tristate.FALSE); } @Test @@ -291,32 +280,31 @@ public void testDirProbes() throws Throwable { // head probe fails interceptRawGetFileStatusFNFE(emptydir, false, StatusProbeEnum.HEAD_ONLY, - FILESTATUS_FILE_PROBE_H, - FILESTATUS_FILE_PROBE_L); + FILE_STATUS_FILE_PROBE); // a LIST will find it and declare as empty S3AFileStatus status = verifyRawInnerGetFileStatus(emptydir, true, - StatusProbeEnum.LIST_ONLY, 0, - GETFILESTATUS_EMPTY_DIR_L); + StatusProbeEnum.LIST_ONLY, + FILE_STATUS_DIR_PROBE); assertEmptyDirStatus(status, Tristate.TRUE); // skip all probes and expect no operations to take place interceptRawGetFileStatusFNFE(emptydir, false, EnumSet.noneOf(StatusProbeEnum.class), - 0, 0); + NO_IO); // now add a trailing slash to the key and use the // deep internal s3GetFileStatus method call. String emptyDirTrailingSlash = fs.pathToKey(emptydir.getParent()) + "/" + emptydir.getName() + "/"; // A HEAD request does not probe for keys with a trailing / - interceptRawHeadList(FileNotFoundException.class, "", - 0, 0, () -> + interceptRaw(FileNotFoundException.class, "", + NO_IO, () -> fs.s3GetFileStatus(emptydir, emptyDirTrailingSlash, StatusProbeEnum.HEAD_ONLY, null, false)); // but ask for a directory marker and you get the entry - status = verifyRawHeadList(0, GETFILESTATUS_EMPTY_DIR_L, () -> + status = verifyRaw(FILE_STATUS_DIR_PROBE, () -> fs.s3GetFileStatus(emptydir, emptyDirTrailingSlash, StatusProbeEnum.LIST_ONLY, @@ -333,9 +321,9 @@ public void testCreateCost() throws Throwable { Path testFile = methodPath(); // when overwrite is false, the path is checked for existence. create(testFile, false, - CREATE_FILE_NO_OVERWRITE_H, CREATE_FILE_NO_OVERWRITE_L); + CREATE_FILE_NO_OVERWRITE); // but when true: only the directory checks take place. - create(testFile, true, CREATE_FILE_OVERWRITE_H, CREATE_FILE_OVERWRITE_L); + create(testFile, true, CREATE_FILE_OVERWRITE); } @Test @@ -346,8 +334,8 @@ public void testCreateCostFileExists() throws Throwable { // now there is a file there, an attempt with overwrite == false will // fail on the first HEAD. - interceptRawHeadList(FileAlreadyExistsException.class, "", - FILESTATUS_FILE_PROBE_H, 0, + interceptRaw(FileAlreadyExistsException.class, "", + FILE_STATUS_FILE_PROBE, () -> file(testFile, false)); } @@ -359,8 +347,8 @@ public void testCreateCostDirExists() throws Throwable { // now there is a file there, an attempt with overwrite == false will // fail on the first HEAD. - interceptRawHeadList(FileAlreadyExistsException.class, "", - GETFILESTATUS_MARKER_H, GETFILESTATUS_EMPTY_DIR_L, + interceptRaw(FileAlreadyExistsException.class, "", + GET_FILE_STATUS_ON_DIR_MARKER, () -> file(testFile, false)); } @@ -377,18 +365,20 @@ public void testCreateBuilder() throws Throwable { // builder defaults to looking for parent existence (non-recursive) buildFile(testFile, false, false, - FILESTATUS_FILE_PROBE_H, // destination file - FILESTATUS_DIR_PROBE_L * 2); // destination file and parent dir + GET_FILE_STATUS_FNFE // destination file + .plus(FILE_STATUS_DIR_PROBE)); // parent dir // recursive = false and overwrite=true: // only make sure the dest path isn't a directory. buildFile(testFile, true, true, - FILESTATUS_DIR_PROBE_H, FILESTATUS_DIR_PROBE_L); + FILE_STATUS_DIR_PROBE + ); // now there is a file there, an attempt with overwrite == false will // fail on the first HEAD. - interceptRawHeadList(FileAlreadyExistsException.class, "", - FILESTATUS_FILE_PROBE_H, 0, () -> - buildFile(testFile, false, true, FILESTATUS_FILE_PROBE_H, 0)); + interceptRaw(FileAlreadyExistsException.class, "", + GET_FILE_STATUS_ON_FILE, () -> + buildFile(testFile, false, true, + GET_FILE_STATUS_ON_FILE)); } @Test @@ -408,8 +398,8 @@ public void testCostOfGlobStatus() throws Throwable { fs.globStatus(basePath.suffix("/*")); // 2 head + 1 list from getFileStatus on path, // plus 1 list to match the glob pattern - verifyRawHeadList(GETFILESTATUS_DIR_H, - GETFILESTATUS_DIR_L + 1, () -> + verifyRaw( + GET_FILE_STATUS_ON_DIR.plus(LIST_OPERATION), () -> fs.globStatus(basePath.suffix("/*"))); } @@ -428,10 +418,8 @@ public void testCostOfGlobStatusNoSymlinkResolution() throws Throwable { // unguarded: 2 head + 1 list from getFileStatus on path, // plus 1 list to match the glob pattern // no additional operations from symlink resolution - verifyRawHeadList( - GETFILESTATUS_DIR_H, - GETFILESTATUS_DIR_L + 1, () -> - fs.globStatus(basePath.suffix("/*"))); + verifyRaw(GET_FILE_STATUS_ON_DIR.plus(LIST_OPERATION), () -> + fs.globStatus(basePath.suffix("/*"))); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java index 118c9ee773a6b..3dc5463376e29 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java @@ -218,4 +218,10 @@ public interface S3ATestConstants { */ String S3GUARD_DDB_TEST_TABLE_NAME_KEY = "fs.s3a.s3guard.ddb.test.table"; + + /** + * Test option to enable audits of the method path after + * every test case. + */ + String DIRECTORY_MARKER_AUDIT = "fs.s3a.directory.marker.audit"; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index 65821c3d5c600..f225800b872f3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -618,6 +618,8 @@ public static Configuration prepareTestConfiguration(final Configuration conf) { // add this so that even on tests where the FS is shared, // the FS is always "magic" conf.setBoolean(MAGIC_COMMITTER_ENABLED, true); + + // directory marker policy String directoryRetention = getTestProperty( conf, DIRECTORY_MARKER_POLICY, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java index d448c09f91d72..b4ee6e47c1a09 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java @@ -40,9 +40,9 @@ import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.*; -import static org.apache.hadoop.fs.s3a.performance.HeadListCosts.*; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.*; +import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.expect; import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.probe; -import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.probes; import static org.apache.hadoop.test.AssertExtensions.dynamicDescription; /** @@ -212,17 +212,15 @@ protected Object getMetricSummary() { * @param path path * @param overwrite overwrite flag * @param recursive true == skip parent existence check - * @param head expected head count - * @param list expected list count + * @param cost expected cost * @return path to new object. */ protected Path buildFile(Path path, boolean overwrite, boolean recursive, - int head, - int list) throws Exception { + OperationCost cost) throws Exception { resetStatistics(); - verifyRawHeadList(head, list, () -> { + verifyRaw(cost, () -> { FSDataOutputStreamBuilder builder = getFileSystem().createFile(path) .overwrite(overwrite); if (recursive) { @@ -272,22 +270,20 @@ protected Path file(Path path, final boolean overwrite) * @return path to new object. */ protected Path create(Path path) throws Exception { - return create(path, true, - CREATE_FILE_OVERWRITE_H, - CREATE_FILE_OVERWRITE_L); + return create(path, true, CREATE_FILE_OVERWRITE); } /** * Create then close the file. * @param path path * @param overwrite overwrite flag - * @param head expected head count - * @param list expected list count + * @param cost expected cost + * @return path to new object. */ protected Path create(Path path, boolean overwrite, - int head, int list) throws Exception { - return verifyRawHeadList(head, list, () -> + OperationCost cost) throws Exception { + return verifyRaw(cost, () -> file(path, overwrite)); } @@ -368,113 +364,116 @@ protected E verifyMetricsIntercepting( * @return the exception caught. * @throws Exception any other exception */ - protected E interceptRawHeadList( + protected E interceptRaw( Class clazz, String text, - int head, - int list, + OperationCost cost, Callable eval) throws Exception { - return verifyMetricsIntercepting(clazz, text, eval, - rawHeadList(head, list)); + return verifyMetricsIntercepting(clazz, text, eval, whenRaw(cost)); } /** - * Create the probes to expect a given set of head and list requests. - * @param enabled is the probe enabled? - * @param head expected HEAD count - * @param list expected LIST count - * @return a probe list + * Declare the expected cost on any FS. + * @param cost costs to expect + * @return a probe. */ - private OperationCostValidator.ExpectedProbe - expectHeadList(boolean enabled, int head, int list) { - return probes(enabled, - probe(OBJECT_METADATA_REQUESTS, head), - probe(OBJECT_LIST_REQUESTS, list)); + protected OperationCostValidator.ExpectedProbe always( + OperationCost cost) { + return expect(true, cost); } /** - * Create the probes to expect a given set of head and list requests. - * @param head expected HEAD count - * @param list expected LIST count - * @return a probe list + * Declare the expected cost on a raw FS. + * @param cost costs to expect + * @return a probe. */ - private OperationCostValidator.ExpectedProbe - alwaysHeadList(int head, int list) { - return expectHeadList(true, head, list); + protected OperationCostValidator.ExpectedProbe whenRaw( + OperationCost cost) { + return expect(isRaw(), cost); } /** - * Declare the expected head and list requests on a raw FS. - * @param head expected HEAD count - * @param list expected LIST count - * @return a probe list + * Declare the expected cost on a guarded FS. + * @param cost costs to expect + * @return a probe. */ - protected OperationCostValidator.ExpectedProbe - rawHeadList(int head, int list) { - return expectHeadList(isRaw(), head, list); + protected OperationCostValidator.ExpectedProbe whenGuarded( + OperationCost cost) { + return expect(isGuarded(), cost); } /** - * Declare the expected head and list requests on an authoritative FS. - * @param head expected HEAD count - * @param list expected LIST count - * @return a probe list + * Declare the expected cost on a guarded auth FS. + * @param cost costs to expect + * @return a probe. */ - protected OperationCostValidator.ExpectedProbe - authHeadList(int head, int list) { - return expectHeadList(isAuthoritative(), head, list); + protected OperationCostValidator.ExpectedProbe whenAuthoritative( + OperationCost cost) { + return expect(isAuthoritative(), cost); } + /** - * Declare the expected head and list requests on a - * non authoritative FS. - * @param head expected HEAD count - * @param list expected LIST count - * @return a probe list + * Declare the expected cost on a guarded nonauth FS. + * @param cost costs to expect + * @return a probe. */ - protected OperationCostValidator.ExpectedProbe - nonauthHeadList(int head, int list) { - return expectHeadList(isNonAuth(), head, list); + protected OperationCostValidator.ExpectedProbe whenNonauth( + OperationCost cost) { + return expect(isNonAuth(), cost); + } + + + /** + * A metric diff which must hold when the fs is keeping markers. + * @param cost expected cost + * @return the diff. + */ + protected OperationCostValidator.ExpectedProbe whenKeeping( + OperationCost cost) { + return expect(isKeepingMarkers(), cost); + } + + /** + * A metric diff which must hold when the fs is keeping markers. + * @param cost expected cost + * @return the diff. + */ + protected OperationCostValidator.ExpectedProbe whenDeleting( + OperationCost cost) { + return expect(isDeleting(), cost); } /** * Execute a closure expecting a specific number of HEAD/LIST calls * on raw S3 stores only. - * @param head expected head request count. - * @param list expected list request count. + * @param cost expected cost * @param eval closure to evaluate * @param return type of closure * @return the result of the evaluation */ - protected T verifyRawHeadList( - int head, - int list, + protected T verifyRaw( + OperationCost cost, Callable eval) throws Exception { - return verifyMetrics(eval, - rawHeadList(head, list)); + return verifyMetrics(eval, whenRaw(cost)); } /** - * Execute {@link S3AFileSystem#innerGetFileStatus(Path, boolean, Set)} + * Execute {@code S3AFileSystem#innerGetFileStatus(Path, boolean, Set)} * for the given probes. - * expect the specific HEAD/LIST count. - *

- * Raw FS only. - *

+ * expect the specific HEAD/LIST count with a raw FS. * @param path path * @param needEmptyDirectoryFlag look for empty directory * @param probes file status probes to perform - * @param head expected head calls - * @param list expected list calls + * @param cost expected cost * @return the status */ public S3AFileStatus verifyRawInnerGetFileStatus( Path path, boolean needEmptyDirectoryFlag, Set probes, - int head, - int list) throws Exception { - return verifyRawHeadList(head, list, () -> + OperationCost cost) throws Exception { + return verifyRaw(cost, () -> innerGetFileStatus(getFileSystem(), path, needEmptyDirectoryFlag, @@ -482,27 +481,22 @@ public S3AFileStatus verifyRawInnerGetFileStatus( } /** - * Execute {@link S3AFileSystem#innerGetFileStatus(Path, boolean, Set)} + * Execute {@code S3AFileSystem#innerGetFileStatus(Path, boolean, Set)} * for the given probes -expect a FileNotFoundException, - * and the specific HEAD/LIST count. - *

- * Raw FS only. - *

+ * and the specific HEAD/LIST count with a raw FS. * @param path path * @param needEmptyDirectoryFlag look for empty directory * @param probes file status probes to perform - * @param head expected head calls - * @param list expected list calls - * @return the status + * @param cost expected cost */ + public void interceptRawGetFileStatusFNFE( Path path, boolean needEmptyDirectoryFlag, Set probes, - int head, - int list) throws Exception { - interceptRawHeadList(FileNotFoundException.class, "", - head, list, () -> + OperationCost cost) throws Exception { + interceptRaw(FileNotFoundException.class, "", + cost, () -> innerGetFileStatus(getFileSystem(), path, needEmptyDirectoryFlag, @@ -514,12 +508,12 @@ public void interceptRawGetFileStatusFNFE( * Metrics are only checked on unguarded stores. * @param path path * @param expected expected outcome - * @param head head count (unguarded) - * @param list listCount (unguarded) + * @param cost expected cost on a Raw FS. */ - protected void isDir(Path path, boolean expected, - int head, int list) throws Exception { - boolean b = verifyRawHeadList(head, list, () -> + protected void isDir(Path path, + boolean expected, + OperationCost cost) throws Exception { + boolean b = verifyRaw(cost, () -> getFileSystem().isDirectory(path)); Assertions.assertThat(b) .describedAs("isDirectory(%s)", path) @@ -531,12 +525,12 @@ protected void isDir(Path path, boolean expected, * Metrics are only checked on unguarded stores. * @param path path * @param expected expected outcome - * @param head head count (unguarded) - * @param list listCount (unguarded) + * @param cost expected cost on a Raw FS. */ - protected void isFile(Path path, boolean expected, - int head, int list) throws Exception { - boolean b = verifyRawHeadList(head, list, () -> + protected void isFile(Path path, + boolean expected, + OperationCost cost) throws Exception { + boolean b = verifyRaw(cost, () -> getFileSystem().isFile(path)); Assertions.assertThat(b) .describedAs("isFile(%s)", path) @@ -560,7 +554,7 @@ protected OperationCostValidator.ExpectedProbe always( * @param expected expected value. * @return the diff. */ - protected OperationCostValidator.ExpectedProbe whenRraw( + protected OperationCostValidator.ExpectedProbe whenRaw( final Statistic stat, final int expected) { return probe(isRaw(), stat, expected); } @@ -627,6 +621,7 @@ protected OperationCostValidator.ExpectedProbe whenDeleting( /** * Assert the empty directory status of a file is as expected. + * The raised assertion message includes a list of the path. * @param status status to probe. * @param expected expected value */ diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/HeadListCosts.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/HeadListCosts.java deleted file mode 100644 index db3d2ae3974ff..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/HeadListCosts.java +++ /dev/null @@ -1,125 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.s3a.performance; - -/** - * Declaration of the costs of head and list calls for various FS IO operations. - */ -public final class HeadListCosts { - - /** Head costs for getFileStatus() directory probe: {@value}. */ - public static final int FILESTATUS_DIR_PROBE_H = 0; - - /** List costs for getFileStatus() directory probe: {@value}. */ - public static final int FILESTATUS_DIR_PROBE_L = 1; - - - /** Head cost getFileStatus() file probe only. */ - public static final int FILESTATUS_FILE_PROBE_H = 1; - - /** Liast cost getFileStatus() file probe only. */ - - public static final int FILESTATUS_FILE_PROBE_L = 0; - - /** Head costs getFileStatus() no file or dir. */ - public static final int GETFILESTATUS_FNFE_H = FILESTATUS_FILE_PROBE_H; - - /** List costs for getFileStatus() on an empty path: {@value}. */ - - public static final int GETFILESTATUS_FNFE_L = FILESTATUS_DIR_PROBE_L; - - /** getFileStatus() directory which is non-empty. */ - public static final int GETFILESTATUS_DIR_H = FILESTATUS_FILE_PROBE_H; - - /** List costs for getFileStatus() on a non-empty directory: {@value}. */ - public static final int GETFILESTATUS_DIR_L = FILESTATUS_DIR_PROBE_L; - - /** List costs for getFileStatus() on an non-empty directory: {@value}. */ - public static final int GETFILESTATUS_EMPTY_DIR_L = FILESTATUS_DIR_PROBE_L; - /** List costs for getFileStatus() on an non-empty directory: {@value}. */ - public static final int GETFILESTATUS_EMPTY_DIR_H = GETFILESTATUS_DIR_H; - - /** getFileStatus() directory marker which exists. */ - public static final int GETFILESTATUS_MARKER_H = FILESTATUS_FILE_PROBE_H; - - /** getFileStatus() on a file which exists. */ - public static final int GETFILESTATUS_SINGLE_FILE_H = FILESTATUS_FILE_PROBE_H; - - - public static final int GETFILESTATUS_SINGLE_FILE_L = FILESTATUS_FILE_PROBE_L; - - public static final int DELETE_OBJECT_REQUEST = 1; - - public static final int DELETE_MARKER_REQUEST = 1; - - /** listLocatedStatus always does a list. */ - public static final int LIST_LOCATED_STATUS_L = 1; - - public static final int LIST_FILES_L = 1; - - /** - * Cost of renaming a file to a different directory. - *

- * LIST on dest not found, look for dest dir, and then, at - * end of rename, whether a parent dir needs to be created. - */ - public static final int RENAME_SINGLE_FILE_RENAME_DIFFERENT_DIR_L = - GETFILESTATUS_FNFE_L + GETFILESTATUS_DIR_L * 2; - - /** - * Cost of renaming a file to a different directory. - *

- * LIST on dest not found, look for dest dir, and then, at - * end of rename, whether a parent dir needs to be created. - */ - public static final int RENAME_SINGLE_FILE_RENAME_SAME_DIR_L = - GETFILESTATUS_FNFE_L; - - /** - * Rename a single file. - *

- * source is found, dest not found, copy adds a - * metadata request. - */ - public static final int RENAME_SINGLE_FILE_RENAME_H = - FILESTATUS_FILE_PROBE_H + GETFILESTATUS_FNFE_H + 1; - - /** - * Create file no overwrite head : {@value}. - */ - public static final int CREATE_FILE_OVERWRITE_H = 0; - - /** - * Create file no overwrite list : {@value}. - */ - public static final int CREATE_FILE_OVERWRITE_L = FILESTATUS_DIR_PROBE_L; - - /** - * Create file no overwrite head : {@value}. - */ - public static final int CREATE_FILE_NO_OVERWRITE_H = FILESTATUS_FILE_PROBE_H; - - /** - * Create file no overwrite list : {@value}. - */ - public static final int CREATE_FILE_NO_OVERWRITE_L = FILESTATUS_DIR_PROBE_L; - - private HeadListCosts() { - } -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java index 13c691e9c00c4..b549c15570e13 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java @@ -29,9 +29,7 @@ import com.amazonaws.AmazonClientException; import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.model.ObjectMetadata; -import org.junit.FixMethodOrder; import org.junit.Test; -import org.junit.runners.MethodSorters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,7 +43,6 @@ 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.impl.DirectoryPolicy; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; @@ -77,7 +74,6 @@ *

* The tests work with unguarded buckets only. */ -@FixMethodOrder(MethodSorters.NAME_ASCENDING) public class ITestDirectoryMarkerListing extends AbstractS3ATestBase { private static final Logger LOG = @@ -109,7 +105,7 @@ public class ITestDirectoryMarkerListing extends AbstractS3ATestBase { *

* The full marker-optimized releases: no. */ - private boolean renameDeletesParentMarkers = false; + private boolean isDeletingMarkers = false; private Path markerDir; @@ -141,6 +137,11 @@ protected Configuration createConfiguration() { return conf; } + /** + * The setup phase includes create the set of test files directories + * under the met + * @throws Exception + */ @Override public void setup() throws Exception { super.setup(); @@ -150,9 +151,11 @@ public void setup() throws Exception { s3client = fs.getAmazonS3ClientForTesting("markers"); bucket = fs.getBucket(); - renameDeletesParentMarkers = fs.getDirectoryMarkerPolicy() - == DirectoryPolicy.MarkerPolicy.Delete; - createTestObjects(new Path(methodPath(), "base")); + Path base = new Path(methodPath(), "base"); + isDeletingMarkers = !fs.getDirectoryMarkerPolicy() + .keepDirectoryMarkers(methodPath()); + + createTestObjects(base); } @Override @@ -427,7 +430,7 @@ public void testRenameUnderMarker() throws Throwable { // renamed into the dest dir assertRenamed(src, dest); assertIsFile(new Path(dest, name)); - if (renameDeletesParentMarkers) { + if (isDeletingMarkers) { head404(markerKeySlash); } else { head(markerKeySlash); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java index b897e5198af19..5c5b2e7360470 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java @@ -35,13 +35,12 @@ import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import static org.apache.hadoop.fs.s3a.Statistic.*; -import static org.apache.hadoop.fs.s3a.performance.HeadListCosts.*; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.*; import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.probe; - /** * Use metrics to assert about the cost of file API calls. - * Parameterized on guarded vs raw. and directory marker keep vs delete + * Parameterized on guarded vs raw. and directory marker keep vs delete. */ @RunWith(Parameterized.class) public class ITestS3ADeleteCost extends AbstractS3ACostTest { @@ -73,10 +72,11 @@ public ITestS3ADeleteCost(final String name, /** * This creates a directory with a child and then deletes it. * The parent dir must be found and declared as empty. + *

When deleting markers, that forces the recreation of a new marker.

*/ @Test - public void testDeleteFile() throws Throwable { - describe("performing getFileStatus on newly emptied directory"); + public void testDeleteSingleFileInDir() throws Throwable { + describe("delete a file"); S3AFileSystem fs = getFileSystem(); // creates the marker Path dir = dir(methodPath()); @@ -91,12 +91,13 @@ public void testDeleteFile() throws Throwable { return "after fs.delete(simpleFile) " + getMetricSummary(); }, // delete file. For keeping: that's it + probe(rawAndKeeping, OBJECT_METADATA_REQUESTS, FILESTATUS_FILE_PROBE_H), // if deleting markers, look for the parent too probe(rawAndDeleting, OBJECT_METADATA_REQUESTS, FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H), - whenRraw(OBJECT_LIST_REQUESTS, + whenRaw(OBJECT_LIST_REQUESTS, FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L), always(DIRECTORIES_DELETED, 0), always(FILES_DELETED, 1), @@ -113,10 +114,54 @@ public void testDeleteFile() throws Throwable { ); // there is an empty dir for a parent S3AFileStatus status = verifyRawInnerGetFileStatus(dir, true, - StatusProbeEnum.ALL, GETFILESTATUS_DIR_H, GETFILESTATUS_DIR_L); + StatusProbeEnum.ALL, GET_FILE_STATUS_ON_DIR); assertEmptyDirStatus(status, Tristate.TRUE); } + /** + * This creates a directory with a two files and then deletes one of the + * files. + */ + @Test + public void testDeleteFileInDir() throws Throwable { + describe("delete a file in a directory with multiple files"); + S3AFileSystem fs = getFileSystem(); + // creates the marker + Path dir = dir(methodPath()); + // file creation may have deleted that marker, but it may + // still be there + Path file1 = file(new Path(dir, "file1.txt")); + Path file2 = file(new Path(dir, "file2.txt")); + + boolean rawAndKeeping = isRaw() && isDeleting(); + boolean rawAndDeleting = isRaw() && isDeleting(); + verifyMetrics(() -> { + fs.delete(file1, false); + return "after fs.delete(file1simpleFile) " + getMetricSummary(); + }, + // delete file. For keeping: that's it + + probe(rawAndKeeping, OBJECT_METADATA_REQUESTS, + FILESTATUS_FILE_PROBE_H), + // if deleting markers, look for the parent too + probe(rawAndDeleting, OBJECT_METADATA_REQUESTS, + FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H), + whenRaw(OBJECT_LIST_REQUESTS, + FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L), + always(DIRECTORIES_DELETED, 0), + always(FILES_DELETED, 1), + + // no need to create a parent + always(DIRECTORIES_CREATED, 0), + + // keeping: create no parent dirs or delete parents + whenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + + // deleting: create a parent and delete any of its parents + whenDeleting(OBJECT_DELETE_REQUESTS, + DELETE_OBJECT_REQUEST)); + } + @Test public void testDirMarkersSubdir() throws Throwable { describe("verify cost of deep subdir creation"); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java index fe5986adb43af..9148f4e4b3dba 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java @@ -34,7 +34,7 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import static org.apache.hadoop.fs.s3a.Statistic.*; -import static org.apache.hadoop.fs.s3a.performance.HeadListCosts.*; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.*; /** * Use metrics to assert about the cost of file API calls. @@ -98,8 +98,7 @@ public void testRenameFileToDifferentDirectory() throws Throwable { // as srcFile2 exists, the parent dir of srcFilePath must not be created. verifyMetrics(() -> execRename(srcFilePath, destFilePath), - rawHeadList(RENAME_SINGLE_FILE_RENAME_H, - RENAME_SINGLE_FILE_RENAME_DIFFERENT_DIR_L), + whenRaw(RENAME_SINGLE_FILE_DIFFERENT_DIR), always(DIRECTORIES_CREATED, 0), always(DIRECTORIES_DELETED, 0), // keeping: only the core delete operation is issued. @@ -136,8 +135,7 @@ public void testRenameSameDirectory() throws Throwable { Path destFile = new Path(parent2, "dest"); verifyMetrics(() -> execRename(sourceFile, destFile), - rawHeadList(RENAME_SINGLE_FILE_RENAME_H, - RENAME_SINGLE_FILE_RENAME_SAME_DIR_L), + whenRaw(RENAME_SINGLE_FILE_SAME_DIR), always(OBJECT_COPY_REQUESTS, 1), always(DIRECTORIES_CREATED, 0), always(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), @@ -145,8 +143,8 @@ public void testRenameSameDirectory() throws Throwable { } @Test - public void testCostOfRootRename() throws Throwable { - describe("assert that a root directory rename doesn't" + public void testCostOfRootFileRename() throws Throwable { + describe("assert that a root file rename doesn't" + " do much in terms of parent dir operations"); S3AFileSystem fs = getFileSystem(); @@ -159,9 +157,9 @@ public void testCostOfRootRename() throws Throwable { fs.rename(src, dest); return "after fs.rename(/src,/dest) " + getMetricSummary(); }, - // TWO HEAD for exists, one for source MD in copy - rawHeadList(RENAME_SINGLE_FILE_RENAME_H, - GETFILESTATUS_FNFE_L), + whenRaw(FILE_STATUS_FILE_PROBE + .plus(GET_FILE_STATUS_FNFE) + .plus(COPY_OP)), // here we expect there to be no fake directories always(DIRECTORIES_CREATED, 0), // one for the renamed file only @@ -172,10 +170,25 @@ public void testCostOfRootRename() throws Throwable { // no fake directories are deleted: This is root always(FAKE_DIRECTORIES_DELETED, 0), always(FILES_DELETED, 1)); + } finally { + fs.delete(src, false); + fs.delete(dest, false); + } + } + @Test + public void testCostOfRootFileDelete() throws Throwable { + describe("assert that a root file delete 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 = file(new Path("/src-" + uuid)); + try { // delete that destination file, assert only the file delete was issued verifyMetrics(() -> { - fs.delete(dest, false); + fs.delete(src, false); return "after fs.delete(/dest) " + getMetricSummary(); }, always(DIRECTORIES_CREATED, 0), @@ -183,14 +196,11 @@ public void testCostOfRootRename() throws Throwable { always(FAKE_DIRECTORIES_DELETED, 0), always(FILES_DELETED, 1), always(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), - rawHeadList(FILESTATUS_FILE_PROBE_H, - 0)); /* no need to look at parent. */ + whenRaw(FILE_STATUS_FILE_PROBE)); /* no need to look at parent. */ } finally { fs.delete(src, false); - fs.delete(dest, false); } } - } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java new file mode 100644 index 0000000000000..56473cc51f347 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java @@ -0,0 +1,202 @@ +/* + * 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.performance; + +/** + * Declaration of the costs of head and list calls for various FS IO + * operations. + *

+ * An instance declares the number of head/list calls expected for + * various operations -with a {@link #plus(OperationCost)} + * method to add operation costs together to produce an + * aggregate cost. These can then be validated in tests + * via {@link OperationCostValidator}. + *

+ * + */ +public final class OperationCost { + + + /** Head costs for getFileStatus() directory probe: {@value}. */ + public static final int FILESTATUS_DIR_PROBE_H = 0; + + /** List costs for getFileStatus() directory probe: {@value}. */ + public static final int FILESTATUS_DIR_PROBE_L = 1; + + /** Head cost getFileStatus() file probe only. */ + public static final int FILESTATUS_FILE_PROBE_H = 1; + + /** Liast cost getFileStatus() file probe only. */ + + public static final int FILESTATUS_FILE_PROBE_L = 0; + + /** + * No IO takes place. + */ + public static final OperationCost NO_IO = + new OperationCost(0,0); + + + /** A HEAD operation. */ + public static final OperationCost HEAD_OPERATION = new OperationCost(1, 0); + + /** A LIST operation. */ + public static final OperationCost LIST_OPERATION = new OperationCost(0, 1); + + /** + * Cost of {@link org.apache.hadoop.fs.s3a.impl.StatusProbeEnum#DIRECTORIES}. + */ + public static final OperationCost FILE_STATUS_DIR_PROBE = LIST_OPERATION; + + /** + * Cost of {@link org.apache.hadoop.fs.s3a.impl.StatusProbeEnum#FILE}. + */ + public static final OperationCost FILE_STATUS_FILE_PROBE = HEAD_OPERATION; + + /** + * Cost of {@link org.apache.hadoop.fs.s3a.impl.StatusProbeEnum#ALL}. + */ + public static final OperationCost FILE_STATUS_ALL_PROBES = + FILE_STATUS_FILE_PROBE.plus(FILE_STATUS_DIR_PROBE); + + /** getFileStatus() on a file which exists. */ + public static final OperationCost GET_FILE_STATUS_ON_FILE = FILE_STATUS_FILE_PROBE; + + /** List costs for getFileStatus() on a non-empty directory: {@value}. */ + public static final OperationCost GET_FILE_STATUS_ON_DIR = + FILE_STATUS_FILE_PROBE.plus(FILE_STATUS_DIR_PROBE); + + /** Costs for getFileStatus() on an empty directory: {@value}. */ + public static final OperationCost GET_FILE_STATUS_ON_EMPTY_DIR = + GET_FILE_STATUS_ON_DIR; + + /** getFileStatus() directory marker which exists. */ + public static final OperationCost GET_FILE_STATUS_ON_DIR_MARKER = + GET_FILE_STATUS_ON_EMPTY_DIR; + + /** getFileStatus() call which fails to find any entry. */ + public static final OperationCost GET_FILE_STATUS_FNFE = + FILE_STATUS_ALL_PROBES; + + /** + * Delete cost when deleting an object. + */ + public static final int DELETE_OBJECT_REQUEST = 1; + + /** + * Delete cost when deleting a marker. + */ + public static final int DELETE_MARKER_REQUEST = DELETE_OBJECT_REQUEST; + + + /** listLocatedStatus always does a list. */ + public static final OperationCost LIST_LOCATED_STATUS_LIST_OP = + new OperationCost(0, 1); + + /** listFiles always does a list. */ + public static final OperationCost LIST_FILES_LIST_OP = + new OperationCost(0, 1); + + + /** + * Cost of renaming a file to a different directory. + *

+ * LIST on dest not found, look for dest dir, and then, at + * end of rename, whether a parent dir needs to be created. + */ + public static final OperationCost RENAME_SINGLE_FILE_DIFFERENT_DIR = + FILE_STATUS_FILE_PROBE // source file + .plus(GET_FILE_STATUS_FNFE) // dest does not exist + .plus(FILE_STATUS_DIR_PROBE) // parent dir of dest + .plus(FILE_STATUS_DIR_PROBE); // recreate source parent dir? + + /** + * Cost of renaming a file to the same directory + *

+ * No need to look for parent directories, so only file + * existence checks. + */ + public static final OperationCost RENAME_SINGLE_FILE_SAME_DIR = + FILE_STATUS_FILE_PROBE // source file + .plus(GET_FILE_STATUS_FNFE); // dest must not exist + + /** + * Metadata cost of a copy operation, as used during rename. + */ + public static final OperationCost COPY_OP = + new OperationCost(1, 0); + + /** + * create(overwrite = true) does not look for the file existing. + */ + public static final OperationCost CREATE_FILE_OVERWRITE = + FILE_STATUS_DIR_PROBE; + + /** + * create(overwrite = false) runs all the checks. + */ + public static final OperationCost CREATE_FILE_NO_OVERWRITE = + FILE_STATUS_ALL_PROBES; + + /** Expected HEAD count. */ + private final int head; + + /** Expected LIST count. */ + private final int list; + + /** + * Constructor. + * @param head head requests. + * @param list list requests. + */ + public OperationCost(final int head, + final int list) { + this.head = head; + this.list = list; + } + + /** Expected HEAD count. */ + int head() { + return head; + } + + /** Expected LIST count. */ + int list() { + return list; + } + + /** + * Add to create a new cost. + * @param that the other entry + * @return cost of the combined operation. + */ + public OperationCost plus(OperationCost that) { + return new OperationCost( + head + that.head, + list + that.list); + } + + @Override + public String toString() { + return "OperationCost{" + + "head=" + head + + ", list=" + list + + '}'; + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java index 91605802b016f..c351d1b185a32 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java @@ -35,6 +35,8 @@ import org.apache.hadoop.fs.s3a.Statistic; import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_LIST_REQUESTS; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_METADATA_REQUESTS; import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** @@ -275,7 +277,8 @@ public static ExpectedProbe always() { } /** - * Create a probe of a statistic which is always enabled. + * Create a probe of a statistic which is enabled whenever the expected + * value is greater than zero. * @param statistic statistic to check. * @param expected expected value. * @return a probe. @@ -283,7 +286,7 @@ public static ExpectedProbe always() { public static ExpectedProbe probe( final Statistic statistic, final int expected) { - return probe(true, statistic, expected); + return probe(expected >= 0, statistic, expected); } /** @@ -316,6 +319,20 @@ public static ExpectedProbe probes( : EMPTY_PROBE; } + /** + * Expect the exact head and list requests of the operation + * cost supplied. + * @param enabled is the probe enabled? + * @param cost expected cost. + * @return a probe. + */ + public static ExpectedProbe expect( + boolean enabled, OperationCost cost) { + return probes(enabled, + probe(OBJECT_METADATA_REQUESTS, cost.head()), + probe(OBJECT_LIST_REQUESTS, cost.list())); + } + /** * An expected probe to verify given criteria to trigger an eval. *

From 6a246ea1ded580d8f3ceef60b139baad46f7bb23 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 4 Aug 2020 15:35:45 +0100 Subject: [PATCH 14/20] HADOOP-13230 getting rename right. Specifically: when you do rename(file, dir) then any marker under dir needs to be deleted. Also: -cost assertions of rename need to include copy_op cost -optional marker audit should ignore FS closed errors -checkstyle -javadoc review/complete for new tests Change-Id: I971950c5d9838133b1e02e4bae9d51a402aaff2a --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 2 +- .../hadoop/fs/s3a/impl/RenameOperation.java | 15 +- .../hadoop/fs/s3a/AbstractS3ATestBase.java | 34 +-- .../fs/s3a/ITestS3AFileOperationCost.java | 2 +- .../ITestDirectoryMarkerListing.java | 226 +++++++++++++++--- .../s3a/performance/ITestS3ADeleteCost.java | 7 +- .../s3a/performance/ITestS3ARenameCost.java | 3 +- .../fs/s3a/performance/OperationCost.java | 29 ++- .../hadoop/fs/s3a/tools/ITestMarkerTool.java | 10 +- 9 files changed, 247 insertions(+), 81 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index c881eb1a2e19b..16f05c269f241 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 @@ -67,6 +67,7 @@ import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.PutObjectRequest; import com.amazonaws.services.s3.model.PutObjectResult; + import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; import com.amazonaws.services.s3.model.SSECustomerKey; import com.amazonaws.services.s3.model.UploadPartRequest; @@ -2668,7 +2669,6 @@ private S3AFileStatus[] innerListStatus(Path f) throws FileNotFoundException, entryPoint(INVOCATION_LIST_STATUS); List result; - entryPoint(INVOCATION_GET_FILE_STATUS); final S3AFileStatus fileStatus = innerGetFileStatus(path, false, StatusProbeEnum.ALL); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java index 37168c5417f2a..c3f594229a97d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java @@ -274,11 +274,15 @@ public Long execute() throws IOException { storeContext, sourcePath, sourceStatus, destPath); + // The path to whichever file or directory is created by the + // rename. When deleting markers all parents of + // this path will need their markers pruned. + Path destCreated = destPath; // Ok! Time to start try { if (sourceStatus.isFile()) { - renameFileToDest(); + destCreated = renameFileToDest(); } else { recursiveDirectoryRename(); } @@ -303,15 +307,17 @@ public Long execute() throws IOException { // Tell the metastore this fact and let it complete its changes renameTracker.completeRename(); - callbacks.finishRename(sourcePath, destPath); + callbacks.finishRename(sourcePath, destCreated); return bytesCopied.get(); } /** - * The source is a file: rename it to the destination. + * The source is a file: rename it to the destination, which + * will be under the current destination path if that is a directory. + * @return the path of the object created. * @throws IOException failure */ - protected void renameFileToDest() throws IOException { + protected Path renameFileToDest() throws IOException { final StoreContext storeContext = getStoreContext(); // the source is a file. Path copyDestinationPath = destPath; @@ -344,6 +350,7 @@ protected void renameFileToDest() throws IOException { callbacks.deleteObjectAtPath(sourcePath, sourceKey, true, null); // and update the tracker renameTracker.sourceObjectsDeleted(Lists.newArrayList(sourcePath)); + return copyDestinationPath; } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java index afdd2a4e82f13..fbfd8b6754185 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java @@ -37,6 +37,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestDynamoTablePrefix; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBool; +import static org.apache.hadoop.fs.s3a.S3AUtils.E_FS_CLOSED; /** * An extension of the contract test base set up for S3A tests. @@ -68,22 +69,23 @@ public void teardown() throws Exception { // if logging at debug S3AFileSystem fs = getFileSystem(); if (fs != null) { - - boolean audit = getTestPropertyBool(fs.getConf(), - DIRECTORY_MARKER_AUDIT, false); - Path methodPath = methodPath(); - if (audit - && !fs.getDirectoryMarkerPolicy().keepDirectoryMarkers(methodPath) - && fs.isDirectory(methodPath)) { - try { - MarkerTool.ScanResult result = MarkerTool.execMarkerTool(fs, - methodPath, true, 0); - - if (result.getExitCode() != 0) { - fail("Audit of " + methodPath + " failed: " + result); - } - } catch (FileNotFoundException ignored) { - } catch (Exception e) { + try { + boolean audit = getTestPropertyBool(fs.getConf(), + DIRECTORY_MARKER_AUDIT, false); + Path methodPath = methodPath(); + if (audit + && !fs.getDirectoryMarkerPolicy().keepDirectoryMarkers(methodPath) + && fs.isDirectory(methodPath)) { + MarkerTool.ScanResult result = MarkerTool.execMarkerTool(fs, + methodPath, true, 0); + if (result.getExitCode() != 0) { + fail("Audit of " + methodPath + " failed: " + result); + } + } + } catch (FileNotFoundException ignored) { + } catch (Exception e) { + // If is this is not due to the FS being closed: log. + if (!e.toString().contains(E_FS_CLOSED)) { LOG.warn("Marker Tool Failure", e); } } 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 28f71f617de17..f5050d783a664 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 @@ -91,7 +91,7 @@ public void testCostOfLocatedFileStatusOnFile() throws Throwable { whenRaw(FILE_STATUS_FILE_PROBE .plus(LIST_LOCATED_STATUS_LIST_OP)), whenAuthoritative(LIST_LOCATED_STATUS_LIST_OP), - whenNonauth(LIST_LOCATED_STATUS_LIST_OP));; + whenNonauth(LIST_LOCATED_STATUS_LIST_OP)); } @Test diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java index b549c15570e13..e522e6d673fac 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.concurrent.Callable; import java.util.stream.Collectors; @@ -30,6 +31,8 @@ import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.model.ObjectMetadata; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,6 +48,11 @@ import org.apache.hadoop.fs.s3a.S3AUtils; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP; +import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE; import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL; import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; @@ -72,8 +80,10 @@ *

* Similarly: JUnit assertions over AssertJ. *

- * The tests work with unguarded buckets only. + * The tests work with unguarded buckets only -the bucket settings are changed + * appropriately. */ +@RunWith(Parameterized.class) public class ITestDirectoryMarkerListing extends AbstractS3ATestBase { private static final Logger LOG = @@ -87,6 +97,17 @@ public class ITestDirectoryMarkerListing extends AbstractS3ATestBase { private static final String MARKER_PEER = "markerpeer"; + /** + * Parameterization. + */ + @Parameterized.Parameters(name = "{0}") + public static Collection params() { + return Arrays.asList(new Object[][]{ + {"keep-markers", true}, + {"delete-markers", false}, + }); + } + /** * Does rename copy markers? * Value: {@value} @@ -98,49 +119,102 @@ public class ITestDirectoryMarkerListing extends AbstractS3ATestBase { private static final boolean RENAME_COPIES_MARKERS = false; /** - * Does rename copy markers? - * Value: {@value} - *

- * Older releases: yes. - *

- * The full marker-optimized releases: no. + * Test configuration name. + */ + private final String name; + + /** + * Does this test configuration keep markers? + */ + private final boolean keepMarkers; + + /** + * Is this FS deleting markers? */ - private boolean isDeletingMarkers = false; + private final boolean isDeletingMarkers; + /** + * Path to a directory which has a marker. + */ private Path markerDir; + /** + * Key to the object representing {@link #markerDir}. + */ private String markerKey; + /** + * Key to the object representing {@link #markerDir} with + * a trailing / added. This references the actual object + * which has been created. + */ private String markerKeySlash; + /** + * bucket of tests. + */ private String bucket; + /** + * S3 Client of the FS. + */ private AmazonS3 s3client; - private String fileKeyUnderMarker; - + /** + * Path to a file under the marker. + */ private Path filePathUnderMarker; + /** + * Key to a file under the marker. + */ + private String fileKeyUnderMarker; + + /** + * base path for the test files; the marker dir goes under this. + */ private Path basePath; + /** + * Path to a file a peer of markerDir. + */ private Path markerPeer; + /** + * Key to a file a peer of markerDir. + */ private String markerPeerKey; + public ITestDirectoryMarkerListing(final String name, + final boolean keepMarkers) { + this.name = name; + this.keepMarkers = keepMarkers; + this.isDeletingMarkers = !keepMarkers; + } + @Override protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); String bucketName = getTestBucketName(conf); + // Turn off S3Guard + removeBaseAndBucketOverrides(bucketName, conf, + S3_METADATA_STORE_IMPL, + METADATASTORE_AUTHORITATIVE, + AUTHORITATIVE_PATH); + + // directory marker options removeBaseAndBucketOverrides(bucketName, conf, - S3_METADATA_STORE_IMPL); + DIRECTORY_MARKER_POLICY); + conf.set(DIRECTORY_MARKER_POLICY, + keepMarkers + ? DIRECTORY_MARKER_POLICY_KEEP + : DIRECTORY_MARKER_POLICY_DELETE); return conf; } /** - * The setup phase includes create the set of test files directories - * under the met - * @throws Exception + * The setup phase includes creating the test objects. */ @Override public void setup() throws Exception { @@ -152,19 +226,21 @@ public void setup() throws Exception { bucket = fs.getBucket(); Path base = new Path(methodPath(), "base"); - isDeletingMarkers = !fs.getDirectoryMarkerPolicy() - .keepDirectoryMarkers(methodPath()); createTestObjects(base); } + /** + * Teardown deletes the objects created before + * the superclass does the directory cleanup. + */ @Override public void teardown() throws Exception { if (s3client != null) { - delete(markerKey); - delete(markerKeySlash); - delete(markerPeerKey); - delete(fileKeyUnderMarker); + deleteObject(markerKey); + deleteObject(markerKeySlash); + deleteObject(markerPeerKey); + deleteObject(fileKeyUnderMarker); } super.teardown(); } @@ -200,15 +276,15 @@ private void createTestObjects(final Path path) throws Exception { @Test public void testMarkerExists() throws Throwable { - describe("Create the test markers for the suite"); + describe("Verify the marker exists"); head(markerKeySlash); assertIsDirectory(markerDir); } @Test public void testObjectUnderMarker() throws Throwable { + describe("verify the file under the marker dir exists"); assertIsFile(filePathUnderMarker); - assertIsDirectory(markerDir); head(fileKeyUnderMarker); } @@ -253,18 +329,20 @@ public void testListStatusBaseDirRecursive() throws Throwable { @Test public void testGlobStatusBaseDirRecursive() throws Throwable { + Path escapedPath = new Path(escape(basePath.toUri().getPath())); List statuses = exec("glob", () -> - toList(getFileSystem().globStatus(new Path(basePath, "*")))); + toList(getFileSystem().globStatus(new Path(escapedPath, "*")))); assertContainsExactlyStatusOfPaths(statuses, markerDir, markerPeer); assertIsFileAtPath(markerPeer, statuses.get(1)); } @Test public void testGlobStatusMarkerDir() throws Throwable { + Path escapedPath = new Path(escape(markerDir.toUri().getPath())); List statuses = exec("glob", () -> - toList(getFileSystem().globStatus(new Path(markerDir, "*")))); + toList(getFileSystem().globStatus(new Path(escapedPath, "*")))); assertContainsFileUnderMarkerOnly(statuses); } @@ -379,7 +457,7 @@ public void testDelete() throws Throwable { */ @Test public void testRenameBase() throws Throwable { - describe("directory rename"); + describe("rename base directory"); Path src = basePath; Path dest = new Path(methodPath(), "dest"); @@ -413,23 +491,73 @@ public void testRenameBase() throws Throwable { } /** - * Rename under a marker. - * The marker must no longer exist. + * Rename a file under a marker by passing in the marker + * directory as the destination; the final path is derived + * from the original filename. + *

+ * After the rename: + *
    + *
  1. The data must be at the derived destination path.
  2. + *
  3. The source file must not exist.
  4. + *
  5. The parent dir of the source file must exist.
  6. + *
  7. The marker above the destination file must not exist.
  8. + *
*/ @Test - public void testRenameUnderMarker() throws Throwable { - describe("directory rename"); + public void testRenameUnderMarkerDir() throws Throwable { + describe("directory rename under an existing marker"); S3AFileSystem fs = getFileSystem(); String name = "sourceFile"; - Path src = new Path(basePath, name); + Path srcDir = new Path(basePath, "srcdir"); + mkdirs(srcDir); + Path src = new Path(srcDir, name); String srcKey = toKey(src); put(srcKey, name); head(srcKey); + // set the destination to be the marker directory. Path dest = markerDir; - // renamed into the dest dir + // rename the source file under the dest dir. assertRenamed(src, dest); assertIsFile(new Path(dest, name)); + assertIsDirectory(srcDir); + if (isDeletingMarkers) { + head404(markerKeySlash); + } else { + head(markerKeySlash); + } + } + + /** + * Rename file under a marker, giving the full path to the destination + * file. + *

+ * After the rename: + *
    + *
  1. The data must be at the explicit destination path.
  2. + *
  3. The source file must not exist.
  4. + *
  5. The parent dir of the source file must exist.
  6. + *
  7. The marker above the destination file must not exist.
  8. + *
+ */ + @Test + public void testRenameUnderMarkerWithPath() throws Throwable { + describe("directory rename under an existing marker"); + S3AFileSystem fs = getFileSystem(); + String name = "sourceFile"; + Path srcDir = new Path(basePath, "srcdir"); + mkdirs(srcDir); + Path src = new Path(srcDir, name); + String srcKey = toKey(src); + put(srcKey, name); + head(srcKey); + + // set the destination to be the final file + Path dest = new Path(markerDir, "destFile"); + // rename the source file to the destination file + assertRenamed(src, dest); + assertIsFile(dest); + assertIsDirectory(srcDir); if (isDeletingMarkers) { head404(markerKeySlash); } else { @@ -492,7 +620,7 @@ private void put(final String key, final String content) throws Exception { * @param key key * @param content string */ - private void delete(final String key) throws Exception { + private void deleteObject(final String key) throws Exception { exec("DELETE " + key, () -> { s3client.deleteObject(bucket, key); return "deleted " + key; @@ -562,9 +690,14 @@ private void assertContainsFileUnderMarkerOnly( private void assertContainsExactlyStatusOfPaths( List statuses, Path... paths) { - String summary = statuses.stream() + String actual = statuses.stream() .map(Object::toString) .collect(Collectors.joining(";")); + String expected = Arrays.stream(paths) + .map(Object::toString) + .collect(Collectors.joining(";")); + String summary = "expected [" + expected + "]" + + " actual = [" + actual + "]"; assertEquals("mismatch in size of listing " + summary, paths.length, statuses.size()); for (int i = 0; i < statuses.size(); i++) { @@ -659,8 +792,31 @@ private void assertRenamed(final Path src, final Path dest) getFileSystem().rename(src, dest)); } - private String toKey(final Path destMarkerDir) { - return getFileSystem().pathToKey(destMarkerDir); + /** + * Convert a path to a key; does not add any trailing / . + * @param path path in + * @return key out + */ + private String toKey(final Path path) { + return getFileSystem().pathToKey(path); + } + + /** + * Escape paths before handing to globStatus; this is needed as + * parameterized runs produce paths with [] in them. + * @param pathstr source path string + * @return an escaped path string + */ + private String escape(String pathstr) { + StringBuilder r = new StringBuilder(); + for (char c : pathstr.toCharArray()) { + String ch = Character.toString(c); + if ("?*[{".contains(ch)) { + r.append("\\"); + } + r.append(ch); + } + return r.toString(); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java index 5c5b2e7360470..594b4667c9301 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java @@ -136,11 +136,10 @@ public void testDeleteFileInDir() throws Throwable { boolean rawAndKeeping = isRaw() && isDeleting(); boolean rawAndDeleting = isRaw() && isDeleting(); verifyMetrics(() -> { - fs.delete(file1, false); - return "after fs.delete(file1simpleFile) " + getMetricSummary(); - }, + fs.delete(file1, false); + return "after fs.delete(file1simpleFile) " + getMetricSummary(); + }, // delete file. For keeping: that's it - probe(rawAndKeeping, OBJECT_METADATA_REQUESTS, FILESTATUS_FILE_PROBE_H), // if deleting markers, look for the parent too diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java index 9148f4e4b3dba..d52da5a2d7837 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java @@ -121,8 +121,7 @@ public void testRenameFileToDifferentDirectory() throws Throwable { */ @Test public void testRenameSameDirectory() throws Throwable { - describe("rename a file to a different directory, " - + "keeping the source dir present"); + describe("rename a file to the same directory"); Path baseDir = dir(methodPath()); final Path sourceFile = file(new Path(baseDir, "source.txt")); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java index 56473cc51f347..136e369f95b14 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java @@ -50,7 +50,7 @@ public final class OperationCost { * No IO takes place. */ public static final OperationCost NO_IO = - new OperationCost(0,0); + new OperationCost(0, 0); /** A HEAD operation. */ @@ -76,7 +76,8 @@ public final class OperationCost { FILE_STATUS_FILE_PROBE.plus(FILE_STATUS_DIR_PROBE); /** getFileStatus() on a file which exists. */ - public static final OperationCost GET_FILE_STATUS_ON_FILE = FILE_STATUS_FILE_PROBE; + public static final OperationCost GET_FILE_STATUS_ON_FILE = + FILE_STATUS_FILE_PROBE; /** List costs for getFileStatus() on a non-empty directory: {@value}. */ public static final OperationCost GET_FILE_STATUS_ON_DIR = @@ -114,6 +115,12 @@ public final class OperationCost { new OperationCost(0, 1); + /** + * Metadata cost of a copy operation, as used during rename. + */ + public static final OperationCost COPY_OP = + new OperationCost(1, 0); + /** * Cost of renaming a file to a different directory. *

@@ -121,26 +128,22 @@ public final class OperationCost { * end of rename, whether a parent dir needs to be created. */ public static final OperationCost RENAME_SINGLE_FILE_DIFFERENT_DIR = - FILE_STATUS_FILE_PROBE // source file + FILE_STATUS_FILE_PROBE // source file probe .plus(GET_FILE_STATUS_FNFE) // dest does not exist .plus(FILE_STATUS_DIR_PROBE) // parent dir of dest - .plus(FILE_STATUS_DIR_PROBE); // recreate source parent dir? + .plus(FILE_STATUS_DIR_PROBE) // recreate source parent dir? + .plus(COPY_OP); // metadata read on copy /** * Cost of renaming a file to the same directory *

* No need to look for parent directories, so only file - * existence checks. + * existence checks and the copy. */ public static final OperationCost RENAME_SINGLE_FILE_SAME_DIR = - FILE_STATUS_FILE_PROBE // source file - .plus(GET_FILE_STATUS_FNFE); // dest must not exist - - /** - * Metadata cost of a copy operation, as used during rename. - */ - public static final OperationCost COPY_OP = - new OperationCost(1, 0); + FILE_STATUS_FILE_PROBE // source file probe + .plus(GET_FILE_STATUS_FNFE) // dest must not exist + .plus(COPY_OP); // metadata read on copy /** * create(overwrite = true) does not look for the file existing. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java index b2861a1da273f..8bf768f2794c4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java @@ -106,8 +106,8 @@ public void setup() throws Exception { @Override public void teardown() throws Exception { - // do this ourselves to avoid the marker checks in the superclass getting - // upset that even when the test FS delete markers, surplus markers are found + // do this ourselves to avoid audits teardown failing + // when surplus markers are found deleteTestDirInTeardown(); super.teardown(); IOUtils.cleanupWithLogger(LOG, getKeepingFS(), @@ -491,16 +491,16 @@ private CreatedPaths createPaths(FileSystem fs, Path base) * @param sourceFS filesystem to use * @param path path to scan * @param doPurge should markers be purged - * @param expectedMarkers number of markers expected + * @param expectedMarkerCount number of markers expected * @return the result */ private MarkerTool.ScanResult markerTool( final FileSystem sourceFS, final Path path, final boolean doPurge, - final int expectedMarkers) + final int expectedMarkerCount) throws IOException { - return markerTool(0, sourceFS, path, doPurge, expectedMarkers); + return markerTool(0, sourceFS, path, doPurge, expectedMarkerCount); } /** From 2ed6e135c69dd642aff5b6954bfe9b30ebf1e001 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 4 Aug 2020 22:52:28 +0100 Subject: [PATCH 15/20] HADOOP-13230. Tests and MarkerTool. -In sync with rebased trunk -Added a progress counter for MarkerTool and the ability to set a limit. Should make a run against landsat viable as a scale test, even though it is markerless (is it? we will find out now) -Test work, including more on the audit phase Change-Id: I4f998bb2637177c66c9c35a976c00e7b003a49ff --- .../org/apache/hadoop/fs/s3a/Listing.java | 8 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 87 ------------------- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 26 +++++- .../hadoop/fs/s3a/tools/MarkerTool.java | 82 +++++++++++------ .../hadoop/fs/s3a/AbstractS3ATestBase.java | 3 +- .../commit/terasort/ITestTerasortOnS3A.java | 9 +- .../ITestDirectoryMarkerListing.java | 3 + .../s3a/performance/ITestS3ADeleteCost.java | 9 ++ .../hadoop/fs/s3a/tools/ITestMarkerTool.java | 45 ++++++++-- 9 files changed, 148 insertions(+), 124 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 2db74fd74423b..34129e0bf1a74 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 @@ -209,8 +209,12 @@ public RemoteIterator getListFilesAssumingDir( String key = maybeAddTrailingSlash(pathToKey(path)); String delimiter = recursive ? null : "/"; - LOG.debug("Requesting all entries under {} with delimiter '{}'", - key, delimiter); + if (recursive) { + LOG.debug("Recursive list of all entries under {}", key); + } else { + LOG.debug("Requesting all entries under {} with delimiter '{}'", + key, delimiter); + } final RemoteIterator cachedFilesIterator; final Set tombstones; boolean allowAuthoritative = listingOperationCallbacks 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 16f05c269f241..4db9b18776091 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 @@ -4410,93 +4410,6 @@ private RemoteIterator innerListFiles( } } - /** - * List files under a path assuming the path to be a directory. - * @param path input path. - * @param recursive recursive listing? - * @param acceptor file status filter - * @param collectTombstones should tombstones be collected from S3Guard? - * @param forceNonAuthoritativeMS forces metadata store to act like non - * authoritative. This is useful when - * listFiles output is used by import tool. - * @return an iterator over listing. - * @throws IOException any exception. - */ - private RemoteIterator getListFilesAssumingDir( - Path path, - boolean recursive, Listing.FileStatusAcceptor acceptor, - boolean collectTombstones, - boolean forceNonAuthoritativeMS) throws IOException { - - String key = maybeAddTrailingSlash(pathToKey(path)); - String delimiter = recursive ? null : "/"; - if (recursive) { - LOG.debug("Recursive list of all entries under {}", key); - } else { - LOG.debug("Requesting all entries under {} with delimiter '{}'", - key, delimiter); - } - final RemoteIterator cachedFilesIterator; - final Set tombstones; - boolean allowAuthoritative = allowAuthoritative(path); - if (recursive) { - final PathMetadata pm = metadataStore.get(path, true); - if (pm != null) { - if (pm.isDeleted()) { - OffsetDateTime deletedAt = OffsetDateTime - .ofInstant(Instant.ofEpochMilli( - pm.getFileStatus().getModificationTime()), - ZoneOffset.UTC); - throw new FileNotFoundException("Path " + path + " is recorded as " + - "deleted by S3Guard at " + deletedAt); - } - } - MetadataStoreListFilesIterator metadataStoreListFilesIterator = - new MetadataStoreListFilesIterator(metadataStore, pm, - allowAuthoritative); - tombstones = metadataStoreListFilesIterator.listTombstones(); - // if all of the below is true - // - authoritative access is allowed for this metadatastore - // for this directory, - // - all the directory listings are authoritative on the client - // - the caller does not force non-authoritative access - // return the listing without any further s3 access - if (!forceNonAuthoritativeMS && - allowAuthoritative && - metadataStoreListFilesIterator.isRecursivelyAuthoritative()) { - S3AFileStatus[] statuses = S3Guard.iteratorToStatuses( - metadataStoreListFilesIterator, tombstones); - cachedFilesIterator = listing.createProvidedFileStatusIterator( - statuses, ACCEPT_ALL, acceptor); - return listing.createLocatedFileStatusIterator(cachedFilesIterator); - } - cachedFilesIterator = metadataStoreListFilesIterator; - } else { - DirListingMetadata meta = - S3Guard.listChildrenWithTtl(metadataStore, path, ttlTimeProvider, - allowAuthoritative); - if (meta != null) { - tombstones = meta.listTombstones(); - } else { - tombstones = null; - } - cachedFilesIterator = listing.createProvidedFileStatusIterator( - S3Guard.dirMetaToStatuses(meta), ACCEPT_ALL, acceptor); - if (allowAuthoritative && meta != null && meta.isAuthoritative()) { - // metadata listing is authoritative, so return it directly - return listing.createLocatedFileStatusIterator(cachedFilesIterator); - } - } - return listing.createTombstoneReconcilingIterator( - listing.createLocatedFileStatusIterator( - listing.createFileStatusListingIterator(path, - createListObjectsRequest(key, delimiter), - ACCEPT_ALL, - acceptor, - cachedFilesIterator)), - collectTombstones ? tombstones : null); - } - /** * Override superclass so as to add statistic collection. * {@inheritDoc} 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 fc1c73937eaaf..a250adf3c0d43 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 @@ -60,6 +60,7 @@ import org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens; import org.apache.hadoop.fs.s3a.commit.CommitConstants; import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants; +import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; import org.apache.hadoop.fs.s3a.select.SelectTool; import org.apache.hadoop.fs.s3a.tools.MarkerTool; import org.apache.hadoop.fs.shell.CommandFormat; @@ -1207,6 +1208,7 @@ public static class BucketInfo extends S3GuardTool { public static final String NONAUTH_FLAG = "nonauth"; public static final String ENCRYPTION_FLAG = "encryption"; public static final String MAGIC_FLAG = "magic"; + public static final String MARKERS_FLAG = "markers"; public static final String PURPOSE = "provide/check S3Guard information" + " about a specific bucket"; @@ -1219,7 +1221,9 @@ public static class BucketInfo extends S3GuardTool { + " -" + NONAUTH_FLAG + " - Require the S3Guard mode to be \"non-authoritative\"\n" + " -" + MAGIC_FLAG + " - Require the S3 filesystem to be support the \"magic\" committer\n" + " -" + ENCRYPTION_FLAG - + " -require {none, sse-s3, sse-kms} - Require encryption policy"; + + " (none, sse-s3, sse-kms) - Require encryption policy\n" + + " -" + MARKERS_FLAG + + " (keep, delete, authoritative) - directory markers policy\n"; /** * Output when the client cannot get the location of a bucket. @@ -1233,6 +1237,7 @@ public BucketInfo(Configuration conf) { super(conf, GUARDED_FLAG, UNGUARDED_FLAG, AUTH_FLAG, NONAUTH_FLAG, MAGIC_FLAG); CommandFormat format = getCommandFormat(); format.addOptionWithValue(ENCRYPTION_FLAG); + format.addOptionWithValue(MARKERS_FLAG); } @Override @@ -1417,6 +1422,25 @@ public int run(String[] args, PrintStream out) fsUri, desiredEncryption, encryption); } + // directory markers + DirectoryPolicy markerPolicy = fs.getDirectoryMarkerPolicy(); + String desc = markerPolicy.describe(); + println(out, "%nDirectory marker policy is %s", desc); + + DirectoryPolicy.MarkerPolicy mp = markerPolicy.getMarkerPolicy(); + String desiredMarker = getCommandFormat() + .getOptValue(MARKERS_FLAG); + desiredMarker = desiredMarker == null + ? "" + : desiredMarker.trim().toLowerCase(Locale.ENGLISH); + if (!desiredMarker.isEmpty() + && !desiredMarker.equals(mp.toString())) { + throw badState("Bucket %s: required marker policy is %s" + + " but actual policy is %s", + fsUri, desiredMarker, mp); + } + + // and finally flush the output and report a success. out.flush(); return SUCCESS; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index 44866351e0bf6..e087cec812179 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -57,20 +57,16 @@ import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE; import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE_DEFAULT; import static org.apache.hadoop.fs.s3a.Invoker.once; +import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_INTERRUPTED; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_SUCCESS; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_USAGE; /** - * Handle directory-related command-line options in the - * s3guard tool. - *
- *   scan: scan for markers
- *   clean: clean up marker entries.
- * 
+ * Audit and S3 bucket for directory markers. + *

* This tool does not go anywhere near S3Guard; its scan bypasses any * metastore as we are explicitly looking for marker objects. - * */ public final class MarkerTool extends S3GuardTool { @@ -112,9 +108,9 @@ public final class MarkerTool extends S3GuardTool { public static final String REPORT = "report"; /** - * Verbose option: {@value}. + * Limit of objects to scan: {@value}. */ - public static final String OPT_VERBOSE = "verbose"; + public static final String OPT_LIMIT = "limit"; /** * Error text when too few arguments are found. @@ -126,7 +122,7 @@ public final class MarkerTool extends S3GuardTool { * Constant to use when there is no limit on the number of * markers expected: {@value}. */ - private static final int UNLIMITED = -1; + public static final int UNLIMITED = -1; /** Will be overridden in run(), but during tests needs to avoid NPEs. */ private PrintStream out = System.out; @@ -162,7 +158,9 @@ public final class MarkerTool extends S3GuardTool { * @param conf */ public MarkerTool(final Configuration conf) { - super(conf, OPT_VERBOSE); + super(conf, VERBOSE); + CommandFormat format = getCommandFormat(); + format.addOptionWithValue(OPT_LIMIT); } @Override @@ -223,14 +221,20 @@ public int run(final String[] args, final PrintStream stream) throw new ExitUtil.ExitException(EXIT_USAGE, "Unknown action: " + action); } - + int limit = UNLIMITED; + String limitOpt = getCommandFormat() + .getOptValue(OPT_LIMIT); + if (limitOpt != null && !limitOpt.isEmpty()) { + limit = Integer.parseInt(limitOpt); + } final String file = parsedArgs.get(1); final Path path = new Path(file); ScanResult result = execute( path.getFileSystem(getConf()), path, purge, - expected); + expected, + limit); if (verbose) { dumpFileSystemStatistics(out); } @@ -243,6 +247,7 @@ public int run(final String[] args, final PrintStream stream) * @param path path to scan. * @param doPurge purge? * @param expectedMarkerCount expected marker count + * @param limit limit of files to scan; -1 for 'unlimited' * @return scan+purge result. * @throws IOException failure */ @@ -251,7 +256,8 @@ ScanResult execute( final FileSystem sourceFS, final Path path, final boolean doPurge, - final int expectedMarkerCount) + final int expectedMarkerCount, + final int limit) throws IOException { S3AFileSystem fs = bindFilesystem(sourceFS); @@ -280,10 +286,9 @@ ScanResult execute( } catch (FileNotFoundException ex) { throw new ExitUtil.ExitException(LauncherExitCodes.EXIT_NOT_FOUND, "Not found: " + path, ex); - } - ScanResult result = scan(path, doPurge, expectedMarkerCount); + ScanResult result = scan(path, doPurge, expectedMarkerCount, limit); return result; } @@ -340,6 +345,7 @@ public MarkerPurgeSummary getPurgeSummary() { * @param path path to scan. * @param doPurge purge? * @param expectedMarkerCount expected marker count + * @param limit limit of files to scan; -1 for 'unlimited' * @return scan+purge result. * @throws IOException IO failure * @throws ExitUtil.ExitException explicitly raised failure @@ -348,16 +354,21 @@ public MarkerPurgeSummary getPurgeSummary() { private ScanResult scan( final Path path, final boolean doPurge, - final int expectedMarkerCount) + final int expectedMarkerCount, + final int limit) throws IOException, ExitUtil.ExitException { ScanResult result = new ScanResult(); + // Mission Accomplished + result.exitCode = EXIT_SUCCESS; + // Now do the work. DirMarkerTracker tracker = new DirMarkerTracker(path); result.tracker = tracker; + boolean completed; try (DurationInfo ignored = new DurationInfo(LOG, "marker scan %s", path)) { - scanDirectoryTree(path, tracker); + completed = scanDirectoryTree(path, tracker, limit); } // scan done. what have we got? @@ -397,7 +408,6 @@ private ScanResult scan( println(out, "Surplus markers were found -failing audit"); result.exitCode = EXIT_NOT_ACCEPTABLE; - return result; } if (doPurge) { @@ -406,8 +416,14 @@ private ScanResult scan( BULK_DELETE_PAGE_SIZE_DEFAULT); result.purgeSummary = purgeMarkers(tracker, deletePageSize); } - result.exitCode = EXIT_SUCCESS; - return result; + + + // now one little check for whether a limit was reached. + if (!completed) { + println(out, "Listing limit reached before completing the scan"); + result.exitCode = EXIT_INTERRUPTED; + } + return result; } /** @@ -423,15 +439,21 @@ private String suffix(final int size) { * Scan a directory tree. * @param path path to scan * @param tracker tracker to update + * @param limit limit of files to scan; -1 for 'unlimited' + * @return true if the scan completedly scanned the entire tree * @throws IOException IO failure */ @Retries.RetryTranslated - private void scanDirectoryTree(final Path path, - final DirMarkerTracker tracker) throws IOException { + private boolean scanDirectoryTree( + final Path path, + final DirMarkerTracker tracker, + final int limit) throws IOException { + int count = 0; RemoteIterator listing = operations .listObjects(path, storeContext.pathToKey(path)); while (listing.hasNext()) { + count++; S3AFileStatus status = listing.next(); Path statusPath = status.getPath(); S3ALocatedFileStatus locatedStatus = new S3ALocatedFileStatus( @@ -450,7 +472,15 @@ private void scanDirectoryTree(final Path path, key, locatedStatus); } + if ((count % 1000) == 0) { + println(out, "Scanned %,d objects", count); + } + if (limit > 0 || count >= limit) { + println(out, "Limit of scanned reached - %,d objects", limit); + return false; + } } + return true; } /** @@ -569,6 +599,7 @@ public void setVerbose(final boolean verbose) { * @param path path to scan * @param doPurge should markers be purged * @param expectedMarkers number of markers expected + * @param limit limit of files to scan; -1 for 'unlimited' * @return the result */ @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") @@ -576,11 +607,12 @@ public static MarkerTool.ScanResult execMarkerTool( final FileSystem sourceFS, final Path path, final boolean doPurge, - final int expectedMarkers) throws IOException { + final int expectedMarkers, + final int limit) throws IOException { MarkerTool tool = new MarkerTool(sourceFS.getConf()); tool.setVerbose(LOG.isDebugEnabled()); return tool.execute(sourceFS, path, doPurge, - expectedMarkers); + expectedMarkers, limit); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java index fbfd8b6754185..9e4a386b8e46c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java @@ -38,6 +38,7 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestDynamoTablePrefix; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBool; import static org.apache.hadoop.fs.s3a.S3AUtils.E_FS_CLOSED; +import static org.apache.hadoop.fs.s3a.tools.MarkerTool.UNLIMITED; /** * An extension of the contract test base set up for S3A tests. @@ -77,7 +78,7 @@ public void teardown() throws Exception { && !fs.getDirectoryMarkerPolicy().keepDirectoryMarkers(methodPath) && fs.isDirectory(methodPath)) { MarkerTool.ScanResult result = MarkerTool.execMarkerTool(fs, - methodPath, true, 0); + methodPath, true, 0, UNLIMITED); if (result.getExitCode() != 0) { fail("Audit of " + methodPath + " failed: " + result); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java index dc6c6d19db9ab..c336d3b2cd83e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java @@ -237,12 +237,17 @@ private void executeStage( } finally { d.close(); } + if (result != 0) { + LOG.warn("Exit code from job was {}", result); + } dumpOutputTree(dest); + // in case the success file arrives even though the exit code is negative + // (logs imply it can happen), dumpt it first. + validateSuccessFile(dest, committerName(), getFileSystem(), stage, + minimumFileCount); assertEquals(stage + "(" + StringUtils.join(", ", args) + ")" + " failed", 0, result); - validateSuccessFile(dest, committerName(), getFileSystem(), stage, - minimumFileCount); completedStage(stage, d); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java index e522e6d673fac..417c25df9bc25 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java @@ -242,6 +242,9 @@ public void teardown() throws Exception { deleteObject(markerPeerKey); deleteObject(fileKeyUnderMarker); } + // do this ourselves to avoid audits teardown failing + // when surplus markers are found + deleteTestDirInTeardown(); super.teardown(); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java index 594b4667c9301..d2d3ae705a053 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java @@ -68,6 +68,15 @@ public ITestS3ADeleteCost(final String name, super(s3guard, keepMarkers, authoritative); } + @Override + public void teardown() throws Exception { + if (isKeepingMarkers()) { + // do this ourselves to avoid audits teardown failing + // when surplus markers are found + deleteTestDirInTeardown(); + } + super.teardown(); + } /** * This creates a directory with a child and then deletes it. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java index 8bf768f2794c4..e4b53bf23f02a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java @@ -44,6 +44,7 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; import static org.apache.hadoop.fs.s3a.tools.MarkerTool.*; +import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_INTERRUPTED; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_FOUND; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_USAGE; @@ -59,7 +60,7 @@ public class ITestMarkerTool extends AbstractS3ATestBase { LoggerFactory.getLogger(ITestMarkerTool.class); /** the -verbose option. */ - private static final String V = "-" + OPT_VERBOSE; + private static final String V = "-" + VERBOSE; /** FS which keeps markers. */ private S3AFileSystem keepingFS; @@ -155,6 +156,17 @@ public void testCleanMarkersLegacyDir() throws Throwable { markerTool(getDeletingFS(), createdPaths.base, true, 0); } + @Test + public void testCleanMarkersFileLimit() throws Throwable { + describe("Clean markers under a keeping FS -with file limit"); + CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); + + // audit will find the expected entries + int expectedMarkerCount = createdPaths.dirs.size(); + markerTool(EXIT_INTERRUPTED, getDeletingFS(), + createdPaths.base, false, 0, 1); + } + @Test public void testCleanMarkersKeepingDir() throws Throwable { describe("Audit then clean markers under a deleting FS " @@ -166,7 +178,7 @@ public void testCleanMarkersKeepingDir() throws Throwable { S3AFileSystem fs = getDeletingFS(); LOG.info("Auditing a directory with retained markers -expect failure"); markerTool(EXIT_NOT_ACCEPTABLE, fs, - createdPaths.base, false, 0); + createdPaths.base, false, 0, UNLIMITED); LOG.info("Auditing a directory expecting retained markers"); markerTool(fs, createdPaths.base, false, @@ -295,6 +307,21 @@ public void testRunVerboseAudit() throws Throwable { createdPaths.base.toString()); } + @Test + public void testRunLimitedAudit() throws Throwable { + describe("Adurit"); + CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); + runToFailure(EXIT_INTERRUPTED, + V, + "-" + OPT_LIMIT, "2", + CLEAN, + createdPaths.base.toString()); + run(NAME, + V, + AUDIT, + createdPaths.base.toString()); + } + private static Path topath(Path base, final String name) { return name.isEmpty() ? base : new Path(base, name); } @@ -500,7 +527,8 @@ private MarkerTool.ScanResult markerTool( final boolean doPurge, final int expectedMarkerCount) throws IOException { - return markerTool(0, sourceFS, path, doPurge, expectedMarkerCount); + return markerTool(0, sourceFS, path, doPurge, expectedMarkerCount, + UNLIMITED); } /** @@ -512,6 +540,7 @@ private MarkerTool.ScanResult markerTool( * @param path path to scan * @param doPurge should markers be purged * @param expectedMarkers number of markers expected +HiMan * @param limit limit of files to scan; -1 for 'unlimited' * @return the result */ public static MarkerTool.ScanResult markerTool( @@ -519,11 +548,15 @@ public static MarkerTool.ScanResult markerTool( final FileSystem sourceFS, final Path path, final boolean doPurge, - final int expectedMarkers) throws IOException { + final int expectedMarkers, + final int limit) throws IOException { - MarkerTool.ScanResult result = MarkerTool.execMarkerTool(sourceFS, path, + MarkerTool.ScanResult result = MarkerTool.execMarkerTool( + sourceFS, + path, doPurge, - expectedMarkers); + expectedMarkers, + limit); Assertions.assertThat(result.getExitCode()) .describedAs("Exit code of marker(%s, %s, %d) -> %s", path, doPurge, expectedMarkers, result) From c9f12eb3c0f740b025ae4c32ddda04f50f14b0b6 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 5 Aug 2020 18:43:14 +0100 Subject: [PATCH 16/20] HADOOP-13230. MarkerTool -nonauth This feature is ideal for test validation of mixed buckets -an audit can be run against the entire bucket, and only markers found outside the auth tree will be considered failures. in a marker clean operation, markers everywhere are purged, so it can be used to make the entire bucket backwards compatible + test tuning, especially of audit option. + better examples in the docs Change-Id: I2e6b7e0c6f3fd1f9eddad9b721afbe09041163b4 --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 5 +- .../hadoop/fs/s3a/impl/DirMarkerTracker.java | 54 ++++- .../fs/s3a/impl/DirectoryPolicyImpl.java | 80 +++--- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 9 +- .../hadoop/fs/s3a/tools/MarkerTool.java | 228 +++++++++++------- .../tools/hadoop-aws/directory_markers.md | 166 +++++++++++-- .../hadoop/fs/s3a/AbstractS3ATestBase.java | 9 +- .../hadoop/fs/s3a/ITestAuthoritativePath.java | 12 +- .../fs/s3a/ITestS3AFileOperationCost.java | 30 +-- .../commit/terasort/ITestTerasortOnS3A.java | 9 +- .../s3a/impl/TestDirectoryMarkerPolicy.java | 10 +- .../s3a/performance/AbstractS3ACostTest.java | 14 +- .../s3a/performance/ITestS3ADeleteCost.java | 55 ++--- .../s3a/performance/ITestS3ARenameCost.java | 42 ++-- .../fs/s3a/performance/OperationCost.java | 32 ++- .../hadoop/fs/s3a/tools/ITestMarkerTool.java | 38 ++- 16 files changed, 541 insertions(+), 252 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 4db9b18776091..1965ac2d74e1b 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 @@ -466,7 +466,7 @@ public void initialize(URI name, Configuration originalConf) S3Guard.logS3GuardDisabled(LOG, warnLevel, bucket); } // directory policy, which may look at authoritative paths - directoryPolicy = new DirectoryPolicyImpl(conf, + directoryPolicy = DirectoryPolicyImpl.getDirectoryPolicy(conf, this::allowAuthoritative); LOG.debug("Directory marker retention policy is {}", directoryPolicy); @@ -2533,7 +2533,8 @@ DeleteObjectsResult removeKeys( final boolean quiet) throws MultiObjectDeleteException, AmazonClientException, IOException { undeletedObjectsOnFailure.clear(); - try (DurationInfo ignored = new DurationInfo(LOG, false, "Deleting")) { + try (DurationInfo ignored = new DurationInfo(LOG, false, + "Deleting %d keys", keysToDelete.size())) { return removeKeysS3(keysToDelete, deleteFakeDir, quiet); } catch (MultiObjectDeleteException ex) { LOG.debug("Partial delete failure"); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java index cf6586822271d..44a3237a32fa4 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java @@ -19,10 +19,14 @@ package org.apache.hadoop.fs.s3a.impl; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.TreeMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; @@ -46,6 +50,9 @@ */ public class DirMarkerTracker { + private static final Logger LOG = + LoggerFactory.getLogger(DirMarkerTracker.class); + /** * all leaf markers. */ @@ -70,10 +77,21 @@ public class DirMarkerTracker { */ private int scanCount; + /** + * How many files were found. + */ private int filesFound; + /** + * How many markers were found. + */ private int markersFound; + /** + * How many objects of any kind were found? + */ + private int objectsFound; + /** * Construct. * Base path is currently only used for information rather than validating @@ -131,9 +149,10 @@ public List fileFound(Path path, * @param source listing source * @return the surplus markers found. */ - public List pathFound(Path path, + private List pathFound(Path path, final String key, final S3ALocatedFileStatus source) { + objectsFound++; List removed = new ArrayList<>(); // all parent entries are superfluous @@ -188,6 +207,15 @@ public Path getLastDirChecked() { return lastDirChecked; } + + /** + * How many objects were found. + * @return count + */ + public int getObjectsFound() { + return objectsFound; + } + public int getScanCount() { return scanCount; } @@ -211,6 +239,30 @@ public String toString() { '}'; } + /** + * Scan the surplus marker list and remove from it all where the directory + * policy says "keep". This is useful when auditing + * @param policy policy to use when auditing markers for + * inclusion/exclusion. + * @return list of markers stripped + */ + public List removeAllowedMarkers(DirectoryPolicy policy) { + List removed = new ArrayList<>(); + Iterator> entries = surplusMarkers.entrySet().iterator(); + while (entries.hasNext()) { + Map.Entry entry = entries.next(); + Path path = entry.getKey(); + if (policy.keepDirectoryMarkers(path)) { + // there's a match + // remove it from the map. + entries.remove(); + LOG.debug("Removing {}", entry.getValue()); + removed.add(path); + } + } + return removed; + } + /** * This is a marker entry stored in the map and * returned as markers are deleted. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java index 072988ba7fefa..e423252833a14 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java @@ -46,6 +46,18 @@ public final class DirectoryPolicyImpl public static final String UNKNOWN_MARKER_POLICY = "Unknown value of " + DIRECTORY_MARKER_POLICY + ": "; + /** + * Keep all markers. + */ + public static final DirectoryPolicy KEEP = new DirectoryPolicyImpl( + MarkerPolicy.Keep, (p) -> false); + + /** + * Delete all markers. + */ + public static final DirectoryPolicy DELETE = new DirectoryPolicyImpl( + MarkerPolicy.Delete, (p) -> false); + private static final Logger LOG = LoggerFactory.getLogger( DirectoryPolicyImpl.class); @@ -60,38 +72,16 @@ public final class DirectoryPolicyImpl */ private final Predicate authoritativeness; + /** * Constructor. - * @param conf config - * @param authoritativeness Callback to evaluate authoritativeness of a - * path. + * @param markerPolicy marker policy + * @param authoritativeness function for authoritativeness */ - public DirectoryPolicyImpl( - final Configuration conf, + public DirectoryPolicyImpl(final MarkerPolicy markerPolicy, final Predicate authoritativeness) { + this.markerPolicy = markerPolicy; this.authoritativeness = authoritativeness; - String option = conf.getTrimmed(DIRECTORY_MARKER_POLICY, - DEFAULT_DIRECTORY_MARKER_POLICY); - MarkerPolicy p; - switch (option.toLowerCase(Locale.ENGLISH)) { - case DIRECTORY_MARKER_POLICY_DELETE: - // backwards compatible. - p = MarkerPolicy.Delete; - LOG.debug("Directory markers will be deleted"); - break; - case DIRECTORY_MARKER_POLICY_KEEP: - p = MarkerPolicy.Keep; - LOG.info("Directory markers will be kept"); - break; - case DIRECTORY_MARKER_POLICY_AUTHORITATIVE: - p = MarkerPolicy.Authoritative; - LOG.info("Directory markers will be kept on authoritative" - + " paths"); - break; - default: - throw new IllegalArgumentException(UNKNOWN_MARKER_POLICY + option); - } - this.markerPolicy = p; } @Override @@ -107,7 +97,6 @@ public boolean keepDirectoryMarkers(final Path path) { } } - @Override public MarkerPolicy getMarkerPolicy() { return markerPolicy; @@ -126,4 +115,39 @@ public String toString() { sb.append('}'); return sb.toString(); } + + /** + * Create/Get the policy for this configuration. + * @param conf config + * @param authoritativeness Callback to evaluate authoritativeness of a + * path. + * @return a policy + */ + public static DirectoryPolicy getDirectoryPolicy( + final Configuration conf, + final Predicate authoritativeness) { + DirectoryPolicy policy; + String option = conf.getTrimmed(DIRECTORY_MARKER_POLICY, + DEFAULT_DIRECTORY_MARKER_POLICY); + switch (option.toLowerCase(Locale.ENGLISH)) { + case DIRECTORY_MARKER_POLICY_DELETE: + // backwards compatible. + LOG.debug("Directory markers will be deleted"); + policy = DELETE; + break; + case DIRECTORY_MARKER_POLICY_KEEP: + LOG.info("Directory markers will be kept"); + policy = KEEP; + break; + case DIRECTORY_MARKER_POLICY_AUTHORITATIVE: + LOG.info("Directory markers will be kept on authoritative" + + " paths"); + policy = new DirectoryPolicyImpl(MarkerPolicy.Authoritative, + authoritativeness); + break; + default: + throw new IllegalArgumentException(UNKNOWN_MARKER_POLICY + option); + } + return policy; + } } 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 a250adf3c0d43..0893959728551 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 @@ -514,6 +514,7 @@ public abstract int run(String[] args, PrintStream out) throws Exception, /** * Dump the filesystem Storage Statistics if the FS is not null. + * Only non-zero statistics are printed. * @param stream output stream */ protected void dumpFileSystemStatistics(PrintStream stream) { @@ -521,14 +522,18 @@ protected void dumpFileSystemStatistics(PrintStream stream) { if (fs == null) { return; } - println(stream, "Storage Statistics for %s", fs.getUri()); + println(stream, "%nStorage Statistics for %s%n", fs.getUri()); StorageStatistics st = fs.getStorageStatistics(); Iterator it = st.getLongStatistics(); while (it.hasNext()) { StorageStatistics.LongStatistic next = it.next(); - println(stream, "%s\t%s", next.getName(), next.getValue()); + long value = next.getValue(); + if (value != 0) { + println(stream, "%s\t%s", next.getName(), value); + } } + println(stream, ""); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index e087cec812179..8c033cdc69f6e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -21,6 +21,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.PrintStream; +import java.net.URI; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -51,7 +52,6 @@ import org.apache.hadoop.service.launcher.LauncherExitCodes; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.ExitUtil; -import org.apache.hadoop.util.OperationDuration; import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH; import static org.apache.hadoop.fs.s3a.Constants.BULK_DELETE_PAGE_SIZE; @@ -83,15 +83,6 @@ public final class MarkerTool extends S3GuardTool { public static final String PURPOSE = "View and manipulate S3 directory markers"; - /** - * Usage string: {@value}. - */ - private static final String USAGE = NAME - + " [-" + VERBOSE + "]" - + " (audit | report | clean)" - + " \n" - + "\t" + PURPOSE + "\n\n"; - /** * Audit sub-command: {@value}. */ @@ -103,15 +94,21 @@ public final class MarkerTool extends S3GuardTool { public static final String CLEAN = "clean"; /** - * Report Sub-command: {@value}. + * Expected number of markers to find: {@value}. */ - public static final String REPORT = "report"; + public static final String OPT_EXPECTED = "expected"; /** * Limit of objects to scan: {@value}. */ public static final String OPT_LIMIT = "limit"; + /** + * Only consider markers found in non-authoritative paths + * as failures: {@value}. + */ + public static final String OPT_NONAUTH = "nonauth"; + /** * Error text when too few arguments are found. */ @@ -120,28 +117,35 @@ public final class MarkerTool extends S3GuardTool { /** * Constant to use when there is no limit on the number of - * markers expected: {@value}. + * objects listed: {@value}. + *

+ * The value is 0 and not -1 because it allows for the limit to be + * set on the command line {@code -limit 0}. + * The command line parser rejects {@code -limit -1} as the -1 + * is interpreted as the (unknown) option "-1". */ - public static final int UNLIMITED = -1; + public static final int UNLIMITED_LISTING = 0; - /** Will be overridden in run(), but during tests needs to avoid NPEs. */ - private PrintStream out = System.out; /** - * Verbosity flag. + * Usage string: {@value}. */ - private boolean verbose; + private static final String USAGE = NAME + + " [-" + OPT_EXPECTED + " ]" + + " [-" + OPT_LIMIT + " ]" + + " [-" + OPT_NONAUTH + "]" + + " [-" + VERBOSE + "]" + + " (audit | clean)" + + " \n" + + "\t" + PURPOSE + "\n\n"; - /** - * Should the scan also purge surplus markers? - */ - private boolean purge; + /** Will be overridden in run(), but during tests needs to avoid NPEs. */ + private PrintStream out = System.out; /** - * How many markers are expected; - * {@link #UNLIMITED} means no limit. + * Verbosity flag. */ - private int expected; + private boolean verbose; /** * Store context. @@ -158,7 +162,7 @@ public final class MarkerTool extends S3GuardTool { * @param conf */ public MarkerTool(final Configuration conf) { - super(conf, VERBOSE); + super(conf, VERBOSE, OPT_NONAUTH); CommandFormat format = getCommandFormat(); format.addOptionWithValue(OPT_LIMIT); } @@ -199,9 +203,19 @@ public int run(final String[] args, final PrintStream stream) CommandFormat commandFormat = getCommandFormat(); verbose = commandFormat.getOpt(VERBOSE); - expected = UNLIMITED; + // How many markers are expected? + int expected = 0; + String value = getCommandFormat() + .getOptValue(OPT_EXPECTED); + if (value != null && !value.isEmpty()) { + expected = Integer.parseInt(value); + } + // Should the scan also purge surplus markers? + boolean purge; + // argument 0 is the action String action = parsedArgs.get(0); + switch (action) { case AUDIT: // audit. no purge; fail if any marker is found @@ -212,29 +226,32 @@ public int run(final String[] args, final PrintStream stream) // clean -purge the markers purge = true; break; - case REPORT: - // report -no purge - purge = false; - break; default: errorln(getUsage()); throw new ExitUtil.ExitException(EXIT_USAGE, "Unknown action: " + action); } - int limit = UNLIMITED; - String limitOpt = getCommandFormat() + int limit = UNLIMITED_LISTING; + value = getCommandFormat() .getOptValue(OPT_LIMIT); - if (limitOpt != null && !limitOpt.isEmpty()) { - limit = Integer.parseInt(limitOpt); + if (value != null && !value.isEmpty()) { + limit = Integer.parseInt(value); } final String file = parsedArgs.get(1); - final Path path = new Path(file); + Path path = new Path(file); + URI uri = path.toUri(); + if (uri.getPath().isEmpty()) { + // fix up empty URI for better CLI experience + path = new Path(path, "/"); + } + FileSystem fs = path.getFileSystem(getConf()); ScanResult result = execute( - path.getFileSystem(getConf()), + fs, path, purge, expected, - limit); + limit, + commandFormat.getOpt(OPT_NONAUTH)); if (verbose) { dumpFileSystemStatistics(out); } @@ -248,6 +265,7 @@ public int run(final String[] args, final PrintStream stream) * @param doPurge purge? * @param expectedMarkerCount expected marker count * @param limit limit of files to scan; -1 for 'unlimited' + * @param nonAuth * @return scan+purge result. * @throws IOException failure */ @@ -257,26 +275,34 @@ ScanResult execute( final Path path, final boolean doPurge, final int expectedMarkerCount, - final int limit) + final int limit, + final boolean nonAuth) throws IOException { S3AFileSystem fs = bindFilesystem(sourceFS); // extract the callbacks needed for the rest of the work storeContext = fs.createStoreContext(); operations = fs.createMarkerToolOperations(); - DirectoryPolicy.MarkerPolicy policy = fs.getDirectoryMarkerPolicy() + // filesystem policy. + // if the -nonauth option is set, this is used to filter + // out surplus markers from the results. + DirectoryPolicy activePolicy = fs.getDirectoryMarkerPolicy(); + DirectoryPolicy.MarkerPolicy policy = activePolicy .getMarkerPolicy(); - println(out, "The store's directory marker policy is \"%s\"", + println(out, "The directory marker policy of %s is \"%s\"", + storeContext.getFsURI(), policy); + String authPath = storeContext.getConfiguration() + .getTrimmed(AUTHORITATIVE_PATH, ""); if (policy == DirectoryPolicy.MarkerPolicy.Authoritative) { // in auth mode, note the auth paths. - String authPath = storeContext.getConfiguration() - .getTrimmed(AUTHORITATIVE_PATH, "unset"); - println(out, "Authoritative path list is %s", authPath); + println(out, "Authoritative path list is \"%s\"", authPath); } + // qualify the path + Path target = path.makeQualified(fs.getUri(),new Path("/")); // initial safety check: does the path exist? try { - getFilesystem().getFileStatus(path); + getFilesystem().getFileStatus(target); } catch (UnknownStoreException ex) { // bucket doesn't exist. // replace the stack trace with an error code. @@ -285,10 +311,15 @@ ScanResult execute( } catch (FileNotFoundException ex) { throw new ExitUtil.ExitException(LauncherExitCodes.EXIT_NOT_FOUND, - "Not found: " + path, ex); + "Not found: " + target, ex); } - ScanResult result = scan(path, doPurge, expectedMarkerCount, limit); + // the default filter policy is that all entries should be deleted + DirectoryPolicy filterPolicy = nonAuth + ? activePolicy + : null; + ScanResult result = scan(target, doPurge, expectedMarkerCount, limit, + filterPolicy); return result; } @@ -343,19 +374,21 @@ public MarkerPurgeSummary getPurgeSummary() { /** * Do the scan/purge. * @param path path to scan. - * @param doPurge purge? + * @param clean purge? * @param expectedMarkerCount expected marker count - * @param limit limit of files to scan; -1 for 'unlimited' - * @return scan+purge result. + * @param limit limit of files to scan; 0 for 'unlimited' + * @param filterPolicy filter policy on a nonauth scan; may be null + * @return result. * @throws IOException IO failure * @throws ExitUtil.ExitException explicitly raised failure */ @Retries.RetryTranslated private ScanResult scan( final Path path, - final boolean doPurge, + final boolean clean, final int expectedMarkerCount, - final int limit) + final int limit, + final DirectoryPolicy filterPolicy) throws IOException, ExitUtil.ExitException { ScanResult result = new ScanResult(); @@ -370,27 +403,30 @@ private ScanResult scan( new DurationInfo(LOG, "marker scan %s", path)) { completed = scanDirectoryTree(path, tracker, limit); } - + int objectsFound = tracker.getObjectsFound(); + println(out, "Listed %d object%s under %s%n", + objectsFound, + suffix(objectsFound), + path); // scan done. what have we got? Map surplusMarkers = tracker.getSurplusMarkers(); Map leafMarkers = tracker.getLeafMarkers(); - int size = surplusMarkers.size(); - if (size == 0) { + int surplus = surplusMarkers.size(); + if (surplus == 0) { println(out, "No surplus directory markers were found under %s", path); } else { println(out, "Found %d surplus directory marker%s under %s", - size, - suffix(size), + surplus, + suffix(surplus), path); for (Path markers : surplusMarkers.keySet()) { println(out, " %s", markers); } - } - if (verbose && !leafMarkers.isEmpty()) { + if (!leafMarkers.isEmpty()) { println(out, "Found %d empty directory 'leaf' marker%s under %s", leafMarkers.size(), suffix(leafMarkers.size()), @@ -400,21 +436,40 @@ private ScanResult scan( } println(out, "These are required to indicate empty directories"); } - if (size > expectedMarkerCount) { - // failure - if (expectedMarkerCount > UNLIMITED) { - println(out, "Expected %d marker%s", expectedMarkerCount, suffix(size)); - } - println(out, "Surplus markers were found -failing audit"); - - result.exitCode = EXIT_NOT_ACCEPTABLE; - } - if (doPurge) { + if (clean) { + // clean: remove the markers, do not worry about their + // presence when reporting success/failiure int deletePageSize = storeContext.getConfiguration() .getInt(BULK_DELETE_PAGE_SIZE, BULK_DELETE_PAGE_SIZE_DEFAULT); result.purgeSummary = purgeMarkers(tracker, deletePageSize); + } else { + // this is an audit, so validate the marker count + + if (filterPolicy != null) { + // if a filter policy is supplied, filter out all markers + // under the auth path + List allowed = tracker.removeAllowedMarkers(filterPolicy); + int allowedMarkers = allowed.size(); + println(out, "%nIgnoring %d marker%s in authoritative paths", + allowedMarkers, suffix(allowedMarkers)); + if (verbose) { + allowed.forEach(p -> println(out, p.toString())); + } + // recalculate the marker size + surplus = surplusMarkers.size(); + } + if (surplus > expectedMarkerCount) { + // failure + if (expectedMarkerCount > 0) { + println(out, "Expected %d marker%s", expectedMarkerCount, + suffix(surplus)); + } + println(out, "Surplus markers were found -failing audit"); + + result.exitCode = EXIT_NOT_ACCEPTABLE; + } } @@ -461,7 +516,7 @@ private boolean scanDirectoryTree( String key = storeContext.pathToKey(statusPath); if (status.isDirectory()) { if (verbose) { - println(out, "Directory Marker %s", key); + println(out, " Directory Marker %s", key); } LOG.debug("{}", key); tracker.markerFound(statusPath, @@ -475,8 +530,8 @@ private boolean scanDirectoryTree( if ((count % 1000) == 0) { println(out, "Scanned %,d objects", count); } - if (limit > 0 || count >= limit) { - println(out, "Limit of scanned reached - %,d objects", limit); + if (limit > 0 && count >= limit) { + println(out, "Limit of scan reached - %,d object%s", limit, suffix(limit)); return false; } } @@ -534,8 +589,9 @@ long getTotalDeleteRequestDuration() { * @throws IOException IO failure */ @Retries.RetryTranslated - private MarkerPurgeSummary purgeMarkers(DirMarkerTracker tracker, - int deletePageSize) + private MarkerPurgeSummary purgeMarkers( + final DirMarkerTracker tracker, + final int deletePageSize) throws MultiObjectDeleteException, AmazonClientException, IOException { MarkerPurgeSummary summary = new MarkerPurgeSummary(); @@ -548,9 +604,10 @@ private MarkerPurgeSummary purgeMarkers(DirMarkerTracker tracker, markers.values().stream() .map(p -> new DeleteObjectsRequest.KeyVersion(p.getKey())) .collect(Collectors.toList()); - // as an array list so .sublist is straightforward - List markerKeys = new ArrayList<>( - collect); + // build an array list for ease of creating the lists of + // keys in each page through the subList() method. + List markerKeys = + new ArrayList<>(collect); // now randomize. Why so? if the list spans multiple S3 partitions, // it should reduce the IO load on each part. @@ -560,9 +617,12 @@ private MarkerPurgeSummary purgeMarkers(DirMarkerTracker tracker, pages += 1; } if (verbose) { - println(out, "%d markers to delete in %d pages of %d keys/page", - size, pages, deletePageSize); + println(out, "%n%d marker%s to delete in %d page%s of %d keys/page", + size, suffix(size), + pages, suffix(pages), + deletePageSize); } + DurationInfo durationInfo = new DurationInfo(LOG, "Deleting markers"); int start = 0; while (start < size) { // end is one past the end of the page @@ -570,16 +630,15 @@ private MarkerPurgeSummary purgeMarkers(DirMarkerTracker tracker, List page = markerKeys.subList(start, end); List undeleted = new ArrayList<>(); - OperationDuration duration = new OperationDuration(); once("Remove S3 Keys", tracker.getBasePath().toString(), () -> operations.removeKeys(page, true, undeleted, null, false)); - duration.finished(); summary.deleteRequests++; - summary.totalDeleteRequestDuration += duration.value(); // and move to the start of the next page start = end; } + durationInfo.close(); + summary.totalDeleteRequestDuration = durationInfo.value(); summary.markersDeleted = size; return summary; } @@ -600,6 +659,7 @@ public void setVerbose(final boolean verbose) { * @param doPurge should markers be purged * @param expectedMarkers number of markers expected * @param limit limit of files to scan; -1 for 'unlimited' + * @param nonAuth only use nonauth path count for failure rules * @return the result */ @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") @@ -608,11 +668,11 @@ public static MarkerTool.ScanResult execMarkerTool( final Path path, final boolean doPurge, final int expectedMarkers, - final int limit) throws IOException { + final int limit, boolean nonAuth) throws IOException { MarkerTool tool = new MarkerTool(sourceFS.getConf()); tool.setVerbose(LOG.isDebugEnabled()); return tool.execute(sourceFS, path, doPurge, - expectedMarkers, limit); + expectedMarkers, limit, nonAuth); } } diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md index d6f42736c7b0f..426888415a6b0 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md @@ -12,7 +12,7 @@ limitations under the License. See accompanying LICENSE file. --> -# Altering the S3A Directory Marker Behavior +# Controlling the S3A Directory Marker Behavior ## Critical: this is not backwards compatible! @@ -248,31 +248,169 @@ will be retained. Syntax ``` -> bin/hadoop s3guard markers -markers [-verbose] [-expected ] (audit || report || clean) - view and manipulate S3 directory markers +> hadoop s3guard markers + markers [-expected ] [-limit ] [-nonauth] [-verbose] (audit | clean) + View and manipulate S3 directory markers ``` -### `markers report` +*Options* -Scan the path and simply report on the markers found. +| option | meaning | +|--------|---------| +| `-expected ]` | Expected number of markers to find (primarily for testing) | +| `-limit ]` | Limit on number of objects to scan | +| `-nonauth` | Only consider markers in non-authoritative paths as errors | +| `-verbose` | Verbose output | -### `markers audit` +*Exit Codes* + +| Code | Meaning | +|-------|---------| +| 0 | Success | +| 2 | Usage | +| 3 | interrupted -the value of `-limit` was reached | +| 46 | Markers were found (see HTTP "406", "unacceptable") | + +All other non-zero status code also indicate errors of some form or other. + +### `markers report` Audit the path and fail if any markers were found. -### `markers clean` -The `markers clean` command will clean the directory tree of all surplus markers +``` +> hadoop s3guard markers -limit 8000 audit s3a://landsat-pds/ + +The directory marker policy of s3a://landsat-pds is "Delete" +2020-08-05 13:42:56,079 [main] INFO tools.MarkerTool (DurationInfo.java:(77)) - Starting: marker scan s3a://landsat-pds/ +Scanned 1,000 objects +Scanned 2,000 objects +Scanned 3,000 objects +Scanned 4,000 objects +Scanned 5,000 objects +Scanned 6,000 objects +Scanned 7,000 objects +Scanned 8,000 objects +Limit of scan reached - 8,000 objects +2020-08-05 13:43:01,184 [main] INFO tools.MarkerTool (DurationInfo.java:close(98)) - marker scan s3a://landsat-pds/: duration 0:05.107s +No surplus directory markers were found under s3a://landsat-pds/ +Listing limit reached before completing the scan +2020-08-05 13:43:01,187 [main] INFO util.ExitUtil (ExitUtil.java:terminate(210)) - Exiting with status 3: +``` + +Here the scan reached its object limit before completing the audit; the exit code of 3, "interrupted" indicates this. + + +Example: a verbose audit of a bucket whose policy if authoritative -it is not an error if markers +are found under the path `/tables`. + +``` +> bin/hadoop s3guard markers audit s3a://london/ + 2020-08-05 18:29:16,473 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143)) - Directory markers will be kept on authoritative paths + The directory marker policy of s3a://london is "Authoritative" + Authoritative path list is "/tables" + 2020-08-05 18:29:19,186 [main] INFO tools.MarkerTool (DurationInfo.java:(77)) - Starting: marker scan s3a://london/ + 2020-08-05 18:29:21,610 [main] INFO tools.MarkerTool (DurationInfo.java:close(98)) - marker scan s3a://london/: duration 0:02.425s + Listed 8 objects under s3a://london/ + + Found 3 surplus directory markers under s3a://london/ + s3a://london/tables + s3a://london/tables/tables-4 + s3a://london/tables/tables-4/tables-5 + Found 5 empty directory 'leaf' markers under s3a://london/ + s3a://london/tables/tables-2 + s3a://london/tables/tables-3 + s3a://london/tables/tables-4/tables-5/06 + s3a://london/tables2 + s3a://london/tables3 + These are required to indicate empty directories + Surplus markers were found -failing audit + 2020-08-05 18:29:21,614 [main] INFO util.ExitUtil (ExitUtil.java:terminate(210)) - Exiting with status 46: +``` + +This fails because surplus markers were found. This S3A bucket would *NOT* be safe for older Hadoop versions +to use. + +The `-nonauth` option does not treat markers under authoritative paths as errors: + +``` +bin/hadoop s3guard markers -nonauth audit s3a://london/ +2020-08-05 18:31:16,255 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143)) - Directory markers will be kept on authoritative paths +The directory marker policy of s3a://london is "Authoritative" +Authoritative path list is "/tables" +2020-08-05 18:31:19,210 [main] INFO tools.MarkerTool (DurationInfo.java:(77)) - Starting: marker scan s3a://london/ +2020-08-05 18:31:22,240 [main] INFO tools.MarkerTool (DurationInfo.java:close(98)) - marker scan s3a://london/: duration 0:03.031s +Listed 8 objects under s3a://london/ + +Found 3 surplus directory markers under s3a://london/ + s3a://london/tables + s3a://london/tables/tables-4 + s3a://london/tables/tables-4/tables-5 +Found 5 empty directory 'leaf' markers under s3a://london/ + s3a://london/tables/tables-2 + s3a://london/tables/tables-3 + s3a://london/tables/tables-4/tables-5/06 + s3a://london/tables2 + s3a://london/tables3 +These are required to indicate empty directories + +Ignoring 3 markers in authoritative paths ``` -> hadoop s3guard markers clean s3a://ireland-bucket/ -2020-07-28 18:58:36,612 [main] INFO tools.MarkerTool (DurationInfo.java:(77)) - Starting: marker scan s3a://ireland-bucket/ -2020-07-28 18:58:37,516 [main] INFO tools.MarkerTool (DurationInfo.java:close(98)) - marker scan s3a://ireland-bucket/: duration 0:00.906s -No surplus directory markers were found under s3a://ireland-bucket/ +All of this S3A bucket _other_ than the authoritative path `/tables` will be safe for +incompatible Hadoop releases to to use. + + +### `markers clean` + +The `markers clean` command will clean the directory tree of all surplus markers. +The `-verbose` option prints more detail on the operation as well as some IO statistics + +``` +> hadoop s3guard markers -verbose clean s3a://london/ + +2020-08-05 18:33:25,303 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143)) - Directory markers will be kept on authoritative paths +The directory marker policy of s3a://london is "Authoritative" +Authoritative path list is "/tables" +2020-08-05 18:33:28,511 [main] INFO tools.MarkerTool (DurationInfo.java:(77)) - Starting: marker scan s3a://london/ + Directory Marker tables + Directory Marker tables/tables-2 + Directory Marker tables/tables-3 + Directory Marker tables/tables-4 + Directory Marker tables/tables-4/tables-5 + Directory Marker tables/tables-4/tables-5/06 + Directory Marker tables2 + Directory Marker tables3 +2020-08-05 18:33:31,685 [main] INFO tools.MarkerTool (DurationInfo.java:close(98)) - marker scan s3a://london/: duration 0:03.175s +Listed 8 objects under s3a://london/ + +Found 3 surplus directory markers under s3a://london/ + s3a://london/tables + s3a://london/tables/tables-4 + s3a://london/tables/tables-4/tables-5 +Found 5 empty directory 'leaf' markers under s3a://london/ + s3a://london/tables/tables-2 + s3a://london/tables/tables-3 + s3a://london/tables/tables-4/tables-5/06 + s3a://london/tables2 + s3a://london/tables3 +These are required to indicate empty directories + +3 markers to delete in 1 page of 250 keys/page +2020-08-05 18:33:31,688 [main] INFO tools.MarkerTool (DurationInfo.java:(77)) - Starting: Deleting markers +2020-08-05 18:33:31,812 [main] INFO tools.MarkerTool (DurationInfo.java:close(98)) - Deleting markers: duration 0:00.124s + +Storage Statistics for s3a://london + +op_get_file_status 1 +object_delete_requests 1 +object_list_requests 2 ``` The `markers clean` command _does not_ delete markers above empty directories -only those which have files underneath. If invoked on a path, it will clean up the directory tree into a state -where it is safe for older versions of Hadoop to interact with. \ No newline at end of file +where it is safe for older versions of Hadoop to interact with. + +Note that if invoked with a `-limit` value, surplus markers found during the scan will be removed, +even though the scan will be considered a failure due to the limit being reached. \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java index 9e4a386b8e46c..90da37d717161 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java @@ -38,7 +38,7 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestDynamoTablePrefix; import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBool; import static org.apache.hadoop.fs.s3a.S3AUtils.E_FS_CLOSED; -import static org.apache.hadoop.fs.s3a.tools.MarkerTool.UNLIMITED; +import static org.apache.hadoop.fs.s3a.tools.MarkerTool.UNLIMITED_LISTING; /** * An extension of the contract test base set up for S3A tests. @@ -78,10 +78,9 @@ public void teardown() throws Exception { && !fs.getDirectoryMarkerPolicy().keepDirectoryMarkers(methodPath) && fs.isDirectory(methodPath)) { MarkerTool.ScanResult result = MarkerTool.execMarkerTool(fs, - methodPath, true, 0, UNLIMITED); - if (result.getExitCode() != 0) { - fail("Audit of " + methodPath + " failed: " + result); - } + methodPath, true, 0, UNLIMITED_LISTING, false); + assertEquals("Audit of " + methodPath + " failed: " + result, + 0, result.getExitCode()); } } catch (FileNotFoundException ignored) { } catch (Exception e) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestAuthoritativePath.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestAuthoritativePath.java index 0a91102bf5aa6..b1d742a400505 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestAuthoritativePath.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestAuthoritativePath.java @@ -109,7 +109,8 @@ private S3AFileSystem createFullyAuthFS() URI uri = testFS.getUri(); removeBaseAndBucketOverrides(uri.getHost(), config, - METADATASTORE_AUTHORITATIVE); + METADATASTORE_AUTHORITATIVE, + AUTHORITATIVE_PATH); config.setBoolean(METADATASTORE_AUTHORITATIVE, true); final S3AFileSystem newFS = createFS(uri, config); // set back the same metadata store instance @@ -124,7 +125,8 @@ private S3AFileSystem createSinglePathAuthFS(String authPath) URI uri = testFS.getUri(); removeBaseAndBucketOverrides(uri.getHost(), config, - METADATASTORE_AUTHORITATIVE); + METADATASTORE_AUTHORITATIVE, + AUTHORITATIVE_PATH); config.set(AUTHORITATIVE_PATH, authPath.toString()); final S3AFileSystem newFS = createFS(uri, config); // set back the same metadata store instance @@ -139,7 +141,8 @@ private S3AFileSystem createMultiPathAuthFS(String first, String middle, String URI uri = testFS.getUri(); removeBaseAndBucketOverrides(uri.getHost(), config, - METADATASTORE_AUTHORITATIVE); + METADATASTORE_AUTHORITATIVE, + AUTHORITATIVE_PATH); config.set(AUTHORITATIVE_PATH, first + "," + middle + "," + last); final S3AFileSystem newFS = createFS(uri, config); // set back the same metadata store instance @@ -155,7 +158,8 @@ private S3AFileSystem createRawFS() throws Exception { removeBaseAndBucketOverrides(uri.getHost(), config, S3_METADATA_STORE_IMPL); removeBaseAndBucketOverrides(uri.getHost(), config, - METADATASTORE_AUTHORITATIVE); + METADATASTORE_AUTHORITATIVE, + AUTHORITATIVE_PATH); return createFS(uri, config); } 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 f5050d783a664..70413e3c3d58b 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 @@ -258,9 +258,9 @@ public void testCostOfCopyFromLocalFile() throws Throwable { s3a.copyFromLocalFile(false, true, localPath, remotePath); return "copy"; }, - always(INVOCATION_COPY_FROM_LOCAL_FILE, 1), - always(OBJECT_PUT_REQUESTS, 1), - always(OBJECT_PUT_BYTES, len)); + with(INVOCATION_COPY_FROM_LOCAL_FILE, 1), + with(OBJECT_PUT_REQUESTS, 1), + with(OBJECT_PUT_BYTES, len)); verifyFileContents(s3a, remotePath, data); // print final stats LOG.info("Filesystem {}", s3a); @@ -365,20 +365,19 @@ public void testCreateBuilder() throws Throwable { // builder defaults to looking for parent existence (non-recursive) buildFile(testFile, false, false, - GET_FILE_STATUS_FNFE // destination file - .plus(FILE_STATUS_DIR_PROBE)); // parent dir + GET_FILE_STATUS_FNFE // destination file + .plus(FILE_STATUS_DIR_PROBE)); // parent dir // recursive = false and overwrite=true: // only make sure the dest path isn't a directory. buildFile(testFile, true, true, - FILE_STATUS_DIR_PROBE - ); + FILE_STATUS_DIR_PROBE); // now there is a file there, an attempt with overwrite == false will // fail on the first HEAD. interceptRaw(FileAlreadyExistsException.class, "", - GET_FILE_STATUS_ON_FILE, () -> - buildFile(testFile, false, true, - GET_FILE_STATUS_ON_FILE)); + GET_FILE_STATUS_ON_FILE, + () -> buildFile(testFile, false, true, + GET_FILE_STATUS_ON_FILE)); } @Test @@ -398,9 +397,9 @@ public void testCostOfGlobStatus() throws Throwable { fs.globStatus(basePath.suffix("/*")); // 2 head + 1 list from getFileStatus on path, // plus 1 list to match the glob pattern - verifyRaw( - GET_FILE_STATUS_ON_DIR.plus(LIST_OPERATION), () -> - fs.globStatus(basePath.suffix("/*"))); + verifyRaw(GET_FILE_STATUS_ON_DIR + .plus(LIST_OPERATION), + () -> fs.globStatus(basePath.suffix("/*"))); } @Test @@ -418,8 +417,9 @@ public void testCostOfGlobStatusNoSymlinkResolution() throws Throwable { // unguarded: 2 head + 1 list from getFileStatus on path, // plus 1 list to match the glob pattern // no additional operations from symlink resolution - verifyRaw(GET_FILE_STATUS_ON_DIR.plus(LIST_OPERATION), () -> - fs.globStatus(basePath.suffix("/*"))); + verifyRaw(GET_FILE_STATUS_ON_DIR + .plus(LIST_OPERATION), + () -> fs.globStatus(basePath.suffix("/*"))); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java index c336d3b2cd83e..dc6c6d19db9ab 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java @@ -237,17 +237,12 @@ private void executeStage( } finally { d.close(); } - if (result != 0) { - LOG.warn("Exit code from job was {}", result); - } dumpOutputTree(dest); - // in case the success file arrives even though the exit code is negative - // (logs imply it can happen), dumpt it first. - validateSuccessFile(dest, committerName(), getFileSystem(), stage, - minimumFileCount); assertEquals(stage + "(" + StringUtils.join(", ", args) + ")" + " failed", 0, result); + validateSuccessFile(dest, committerName(), getFileSystem(), stage, + minimumFileCount); completedStage(stage, d); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java index f0a673254af80..fbb5cbc4f881e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java @@ -27,12 +27,12 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.test.AbstractHadoopTestBase; -import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY; - +/** + * Unit tests for directory marker policies. + */ @RunWith(Parameterized.class) public class TestDirectoryMarkerPolicy extends AbstractHadoopTestBase { @@ -80,9 +80,7 @@ public TestDirectoryMarkerPolicy( private DirectoryPolicyImpl retention( DirectoryPolicy.MarkerPolicy markerPolicy, Predicate authoritativeness) { - Configuration c = new Configuration(false); - c.set(DIRECTORY_MARKER_POLICY, markerPolicy.name()); - return new DirectoryPolicyImpl(c, authoritativeness); + return new DirectoryPolicyImpl(markerPolicy, authoritativeness); } private static final Predicate AUTH_PATH_ONLY = diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java index b4ee6e47c1a09..db0542ddc94a6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java @@ -543,7 +543,7 @@ protected void isFile(Path path, * @param expected expected value. * @return the diff. */ - protected OperationCostValidator.ExpectedProbe always( + protected OperationCostValidator.ExpectedProbe with( final Statistic stat, final int expected) { return probe(stat, expected); } @@ -554,7 +554,7 @@ protected OperationCostValidator.ExpectedProbe always( * @param expected expected value. * @return the diff. */ - protected OperationCostValidator.ExpectedProbe whenRaw( + protected OperationCostValidator.ExpectedProbe withWhenRaw( final Statistic stat, final int expected) { return probe(isRaw(), stat, expected); } @@ -565,7 +565,7 @@ protected OperationCostValidator.ExpectedProbe whenRaw( * @param expected expected value. * @return the diff. */ - protected OperationCostValidator.ExpectedProbe whenGuarded( + protected OperationCostValidator.ExpectedProbe withWhenGuarded( final Statistic stat, final int expected) { return probe(isGuarded(), stat, expected); @@ -577,7 +577,7 @@ protected OperationCostValidator.ExpectedProbe whenGuarded( * @param expected expected value. * @return the diff. */ - protected OperationCostValidator.ExpectedProbe whenAuthoritative( + protected OperationCostValidator.ExpectedProbe withWhenAuthoritative( final Statistic stat, final int expected) { return probe(isAuthoritative(), stat, expected); @@ -589,7 +589,7 @@ protected OperationCostValidator.ExpectedProbe whenAuthoritative( * @param expected expected value. * @return the diff. */ - protected OperationCostValidator.ExpectedProbe whenNonauth( + protected OperationCostValidator.ExpectedProbe withWhenNonauth( final Statistic stat, final int expected) { return probe(isNonAuth(), stat, expected); @@ -601,7 +601,7 @@ protected OperationCostValidator.ExpectedProbe whenNonauth( * @param expected expected value. * @return the diff. */ - protected OperationCostValidator.ExpectedProbe whenKeeping( + protected OperationCostValidator.ExpectedProbe withWhenKeeping( final Statistic stat, final int expected) { return probe(isKeepingMarkers(), stat, expected); @@ -613,7 +613,7 @@ protected OperationCostValidator.ExpectedProbe whenKeeping( * @param expected expected value. * @return the diff. */ - protected OperationCostValidator.ExpectedProbe whenDeleting( + protected OperationCostValidator.ExpectedProbe withWhenDeleting( final Statistic stat, final int expected) { return probe(isDeleting(), stat, expected); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java index d2d3ae705a053..03fdd89261cfa 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java @@ -40,6 +40,7 @@ /** * Use metrics to assert about the cost of file API calls. + *

* Parameterized on guarded vs raw. and directory marker keep vs delete. */ @RunWith(Parameterized.class) @@ -99,25 +100,23 @@ public void testDeleteSingleFileInDir() throws Throwable { fs.delete(simpleFile, false); return "after fs.delete(simpleFile) " + getMetricSummary(); }, - // delete file. For keeping: that's it - probe(rawAndKeeping, OBJECT_METADATA_REQUESTS, FILESTATUS_FILE_PROBE_H), // if deleting markers, look for the parent too probe(rawAndDeleting, OBJECT_METADATA_REQUESTS, FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H), - whenRaw(OBJECT_LIST_REQUESTS, + withWhenRaw(OBJECT_LIST_REQUESTS, FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L), - always(DIRECTORIES_DELETED, 0), - always(FILES_DELETED, 1), + with(DIRECTORIES_DELETED, 0), + with(FILES_DELETED, 1), // keeping: create no parent dirs or delete parents - whenKeeping(DIRECTORIES_CREATED, 0), - whenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + withWhenKeeping(DIRECTORIES_CREATED, 0), + withWhenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), // deleting: create a parent and delete any of its parents - whenDeleting(DIRECTORIES_CREATED, 1), - whenDeleting(OBJECT_DELETE_REQUESTS, + withWhenDeleting(DIRECTORIES_CREATED, 1), + withWhenDeleting(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST + DELETE_MARKER_REQUEST) ); @@ -154,19 +153,19 @@ public void testDeleteFileInDir() throws Throwable { // if deleting markers, look for the parent too probe(rawAndDeleting, OBJECT_METADATA_REQUESTS, FILESTATUS_FILE_PROBE_H + FILESTATUS_DIR_PROBE_H), - whenRaw(OBJECT_LIST_REQUESTS, + withWhenRaw(OBJECT_LIST_REQUESTS, FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L), - always(DIRECTORIES_DELETED, 0), - always(FILES_DELETED, 1), + with(DIRECTORIES_DELETED, 0), + with(FILES_DELETED, 1), // no need to create a parent - always(DIRECTORIES_CREATED, 0), + with(DIRECTORIES_CREATED, 0), // keeping: create no parent dirs or delete parents - whenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + withWhenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), // deleting: create a parent and delete any of its parents - whenDeleting(OBJECT_DELETE_REQUESTS, + withWhenDeleting(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST)); } @@ -180,13 +179,14 @@ public void testDirMarkersSubdir() throws Throwable { mkdirs(subDir); return "after mkdir(subDir) " + getMetricSummary(); }, - always(DIRECTORIES_CREATED, 1), - always(DIRECTORIES_DELETED, 0), - whenKeeping(OBJECT_DELETE_REQUESTS, 0), - whenKeeping(FAKE_DIRECTORIES_DELETED, 0), - whenDeleting(OBJECT_DELETE_REQUESTS, DELETE_MARKER_REQUEST), + with(DIRECTORIES_CREATED, 1), + with(DIRECTORIES_DELETED, 0), + withWhenKeeping(OBJECT_DELETE_REQUESTS, 0), + withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0), + withWhenDeleting(OBJECT_DELETE_REQUESTS, DELETE_MARKER_REQUEST), // delete all possible fake dirs above the subdirectory - whenDeleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(subDir) - 1)); + withWhenDeleting(FAKE_DIRECTORIES_DELETED, + directoriesInPath(subDir) - 1)); } @Test @@ -204,14 +204,15 @@ public void testDirMarkersFileCreation() throws Throwable { file(new Path(srcDir, "source.txt")); return "after touch(fs, srcFilePath) " + getMetricSummary(); }, - always(DIRECTORIES_CREATED, 0), - always(DIRECTORIES_DELETED, 0), + with(DIRECTORIES_CREATED, 0), + with(DIRECTORIES_DELETED, 0), // keeping: no delete operations. - whenKeeping(OBJECT_DELETE_REQUESTS, 0), - whenKeeping(FAKE_DIRECTORIES_DELETED, 0), + withWhenKeeping(OBJECT_DELETE_REQUESTS, 0), + withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0), // delete all possible fake dirs above the file - whenDeleting(OBJECT_DELETE_REQUESTS, 1), - whenDeleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(srcDir))); + withWhenDeleting(OBJECT_DELETE_REQUESTS, 1), + withWhenDeleting(FAKE_DIRECTORIES_DELETED, + directoriesInPath(srcDir))); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java index d52da5a2d7837..85c70768356e6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java @@ -38,6 +38,7 @@ /** * Use metrics to assert about the cost of file API calls. + *

* Parameterized on guarded vs raw. and directory marker keep vs delete */ @RunWith(Parameterized.class) @@ -99,15 +100,16 @@ public void testRenameFileToDifferentDirectory() throws Throwable { verifyMetrics(() -> execRename(srcFilePath, destFilePath), whenRaw(RENAME_SINGLE_FILE_DIFFERENT_DIR), - always(DIRECTORIES_CREATED, 0), - always(DIRECTORIES_DELETED, 0), + with(DIRECTORIES_CREATED, 0), + with(DIRECTORIES_DELETED, 0), // keeping: only the core delete operation is issued. - whenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), - whenKeeping(FAKE_DIRECTORIES_DELETED, 0), + withWhenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0), // deleting: delete any fake marker above the destination. - whenDeleting(OBJECT_DELETE_REQUESTS, + withWhenDeleting(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST + DELETE_MARKER_REQUEST), - whenDeleting(FAKE_DIRECTORIES_DELETED, directoriesInPath(destDir))); + withWhenDeleting(FAKE_DIRECTORIES_DELETED, + directoriesInPath(destDir))); assertIsFile(destFilePath); assertIsDirectory(srcDir); @@ -135,10 +137,10 @@ public void testRenameSameDirectory() throws Throwable { verifyMetrics(() -> execRename(sourceFile, destFile), whenRaw(RENAME_SINGLE_FILE_SAME_DIR), - always(OBJECT_COPY_REQUESTS, 1), - always(DIRECTORIES_CREATED, 0), - always(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), - always(FAKE_DIRECTORIES_DELETED, 0)); + with(OBJECT_COPY_REQUESTS, 1), + with(DIRECTORIES_CREATED, 0), + with(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + with(FAKE_DIRECTORIES_DELETED, 0)); } @Test @@ -160,15 +162,15 @@ public void testCostOfRootFileRename() throws Throwable { .plus(GET_FILE_STATUS_FNFE) .plus(COPY_OP)), // here we expect there to be no fake directories - always(DIRECTORIES_CREATED, 0), + with(DIRECTORIES_CREATED, 0), // one for the renamed file only - always(OBJECT_DELETE_REQUESTS, + with(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), // no directories are deleted: This is root - always(DIRECTORIES_DELETED, 0), + with(DIRECTORIES_DELETED, 0), // no fake directories are deleted: This is root - always(FAKE_DIRECTORIES_DELETED, 0), - always(FILES_DELETED, 1)); + with(FAKE_DIRECTORIES_DELETED, 0), + with(FILES_DELETED, 1)); } finally { fs.delete(src, false); fs.delete(dest, false); @@ -190,11 +192,11 @@ public void testCostOfRootFileDelete() throws Throwable { fs.delete(src, false); return "after fs.delete(/dest) " + getMetricSummary(); }, - always(DIRECTORIES_CREATED, 0), - always(DIRECTORIES_DELETED, 0), - always(FAKE_DIRECTORIES_DELETED, 0), - always(FILES_DELETED, 1), - always(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + with(DIRECTORIES_CREATED, 0), + with(DIRECTORIES_DELETED, 0), + with(FAKE_DIRECTORIES_DELETED, 0), + with(FILES_DELETED, 1), + with(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), whenRaw(FILE_STATUS_FILE_PROBE)); /* no need to look at parent. */ } finally { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java index 136e369f95b14..46a6b712c49bf 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java @@ -22,17 +22,15 @@ * Declaration of the costs of head and list calls for various FS IO * operations. *

- * An instance declares the number of head/list calls expected for + * An instance declares the number of head and list calls expected for * various operations -with a {@link #plus(OperationCost)} * method to add operation costs together to produce an * aggregate cost. These can then be validated in tests * via {@link OperationCostValidator}. - *

* */ public final class OperationCost { - /** Head costs for getFileStatus() directory probe: {@value}. */ public static final int FILESTATUS_DIR_PROBE_H = 0; @@ -46,13 +44,22 @@ public final class OperationCost { public static final int FILESTATUS_FILE_PROBE_L = 0; + /** + * Delete cost when deleting an object. + */ + public static final int DELETE_OBJECT_REQUEST = 1; + + /** + * Delete cost when deleting a marker. + */ + public static final int DELETE_MARKER_REQUEST = DELETE_OBJECT_REQUEST; + /** * No IO takes place. */ public static final OperationCost NO_IO = new OperationCost(0, 0); - /** A HEAD operation. */ public static final OperationCost HEAD_OPERATION = new OperationCost(1, 0); @@ -95,28 +102,17 @@ public final class OperationCost { public static final OperationCost GET_FILE_STATUS_FNFE = FILE_STATUS_ALL_PROBES; - /** - * Delete cost when deleting an object. - */ - public static final int DELETE_OBJECT_REQUEST = 1; - - /** - * Delete cost when deleting a marker. - */ - public static final int DELETE_MARKER_REQUEST = DELETE_OBJECT_REQUEST; - - - /** listLocatedStatus always does a list. */ + /** listLocatedStatus always does a LIST. */ public static final OperationCost LIST_LOCATED_STATUS_LIST_OP = new OperationCost(0, 1); - /** listFiles always does a list. */ + /** listFiles always does a LIST. */ public static final OperationCost LIST_FILES_LIST_OP = new OperationCost(0, 1); - /** * Metadata cost of a copy operation, as used during rename. + * This happens even if the store is guarded. */ public static final OperationCost COPY_OP = new OperationCost(1, 0); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java index e4b53bf23f02a..b5fe743e24c72 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java @@ -161,10 +161,9 @@ public void testCleanMarkersFileLimit() throws Throwable { describe("Clean markers under a keeping FS -with file limit"); CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); - // audit will find the expected entries - int expectedMarkerCount = createdPaths.dirs.size(); + // audit will be interrupted markerTool(EXIT_INTERRUPTED, getDeletingFS(), - createdPaths.base, false, 0, 1); + createdPaths.base, false, 0, 1, false); } @Test @@ -178,7 +177,7 @@ public void testCleanMarkersKeepingDir() throws Throwable { S3AFileSystem fs = getDeletingFS(); LOG.info("Auditing a directory with retained markers -expect failure"); markerTool(EXIT_NOT_ACCEPTABLE, fs, - createdPaths.base, false, 0, UNLIMITED); + createdPaths.base, false, 0, UNLIMITED_LISTING, false); LOG.info("Auditing a directory expecting retained markers"); markerTool(fs, createdPaths.base, false, @@ -246,6 +245,13 @@ public void testAuthPathIsMixed() throws Throwable { markerTool(mixedFSDir2, topath(source, "dir1"), false, 0); markerTool(mixedFSDir2, source, false, expectedMarkersUnderDir2); + // full scan of source will fail + markerTool(EXIT_NOT_ACCEPTABLE, + mixedFSDir2, source, false, 0, 0, false); + + // but add the -nonauth option and the markers under dir2 are skipped + markerTool(0, mixedFSDir2, source, false, 0, 0, true); + // if we now rename, all will be good LOG.info("Executing rename"); mixedFSDir2.rename(source, dest); @@ -299,7 +305,7 @@ public void testRunVerboseAudit() throws Throwable { CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); run(NAME, V, - REPORT, + AUDIT, createdPaths.base.toString()); run(NAME, V, @@ -312,6 +318,7 @@ public void testRunLimitedAudit() throws Throwable { describe("Adurit"); CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); runToFailure(EXIT_INTERRUPTED, + NAME, V, "-" + OPT_LIMIT, "2", CLEAN, @@ -354,8 +361,8 @@ private S3AFileSystem createFS(String markerPolicy, conf.set(DIRECTORY_MARKER_POLICY, markerPolicy); S3AFileSystem fs2 = new S3AFileSystem(); fs2.initialize(testFSUri, conf); - LOG.info("created new filesystem with policy {} and auth path {}: {}", - markerPolicy, authPath, fs2); + LOG.info("created new filesystem with policy {} and auth path {}", + markerPolicy, authPath); return fs2; } @@ -528,7 +535,7 @@ private MarkerTool.ScanResult markerTool( final int expectedMarkerCount) throws IOException { return markerTool(0, sourceFS, path, doPurge, expectedMarkerCount, - UNLIMITED); + UNLIMITED_LISTING, false); } /** @@ -540,7 +547,8 @@ private MarkerTool.ScanResult markerTool( * @param path path to scan * @param doPurge should markers be purged * @param expectedMarkers number of markers expected -HiMan * @param limit limit of files to scan; -1 for 'unlimited' + * @param limit limit of files to scan; -1 for 'unlimited' + * @param nonAuth only use nonauth path count for failure rules * @return the result */ public static MarkerTool.ScanResult markerTool( @@ -549,14 +557,15 @@ public static MarkerTool.ScanResult markerTool( final Path path, final boolean doPurge, final int expectedMarkers, - final int limit) throws IOException { + final int limit, + final boolean nonAuth) throws IOException { MarkerTool.ScanResult result = MarkerTool.execMarkerTool( sourceFS, path, doPurge, expectedMarkers, - limit); + limit, nonAuth); Assertions.assertThat(result.getExitCode()) .describedAs("Exit code of marker(%s, %s, %d) -> %s", path, doPurge, expectedMarkers, result) @@ -609,7 +618,12 @@ protected void runToFailure(int status, String... args) throws Exception { ExitUtil.ExitException ex = intercept(ExitUtil.ExitException.class, - () -> run(args)); + () -> { + int ec = run(args); + if (ec != 0) { + throw new ExitUtil.ExitException(ec, "exit code " + ec); + } + }); if (ex.status != status) { throw ex; } From 54b85e93ee1af30fc53ca5f88ca3ece7b804d925 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 11 Aug 2020 22:11:35 +0100 Subject: [PATCH 17/20] HADOOP-13230. MarkerTool. docs and pathCapabilities Change-Id: I402b563cb1fc0497dce20a3ec9e1561f4424349f --- .../org/apache/hadoop/fs/s3a/Constants.java | 28 ++- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 10 + .../apache/hadoop/fs/s3a/S3ListRequest.java | 3 + .../apache/hadoop/fs/s3a/S3ListResult.java | 5 +- .../hadoop/fs/s3a/impl/DirMarkerTracker.java | 43 +++- .../fs/s3a/impl/DirectoryPolicyImpl.java | 10 +- .../hadoop/fs/s3a/impl/RenameOperation.java | 36 ++- .../hadoop/fs/s3a/impl/StatusProbeEnum.java | 5 +- .../hadoop/fs/s3a/tools/MarkerTool.java | 123 ++++++---- .../tools/hadoop-aws/directory_markers.md | 218 +++++++++++++----- .../hadoop/fs/s3a/AbstractS3ATestBase.java | 25 +- .../ITestS3GuardDDBRootOperations.java | 12 +- .../hadoop/fs/s3a/tools/ITestMarkerTool.java | 152 ++++++++++-- 13 files changed, 505 insertions(+), 165 deletions(-) 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 d7cd2ebadf98a..ba3cda91d74d3 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 @@ -985,8 +985,34 @@ private Constants() { "authoritative"; /** - * Default retention policy. + * Default retention policy: {@value}. */ public static final String DEFAULT_DIRECTORY_MARKER_POLICY = DIRECTORY_MARKER_POLICY_DELETE; + + + /** + * Probe to indicate that a path keeps directory markers. + * returned in {@code PathCapabilities} + * Value: {@value}. + */ + public static final String STORE_CAPABILITY_DIRECTORY_MARKER_AWARE + = "fs.s3a.capability.directory.marker.aware"; + + /** + * Probe to indicate that a path keeps directory markers. + * returned in {@code PathCapabilities} + * Value: {@value}. + */ + public static final String STORE_CAPABILITY_DIRECTORY_MARKER_KEEP + = "fs.s3a.capability.directory.marker.keep"; + + /** + * Probe to indicate that a path keeps directory markers. + * returned in {@code PathCapabilities} + * Value: {@value}. + */ + public static final String STORE_CAPABILITY_DIRECTORY_MARKER_DELETE + = "fs.s3a.capability.directory.marker.delete"; + } 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 1965ac2d74e1b..c03d5219293d8 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 @@ -4633,6 +4633,16 @@ public boolean hasPathCapability(final Path path, final String capability) case CommonPathCapabilities.FS_MULTIPART_UPLOADER: return true; + /* + * Marker policy is dynamically determined for the given path. + */ + case STORE_CAPABILITY_DIRECTORY_MARKER_AWARE: + return true; + case STORE_CAPABILITY_DIRECTORY_MARKER_KEEP: + return directoryPolicy.keepDirectoryMarkers(p); + case STORE_CAPABILITY_DIRECTORY_MARKER_DELETE: + return !directoryPolicy.keepDirectoryMarkers(p); + default: return super.hasPathCapability(p, capability); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java index abce86d410272..d51211516f251 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java @@ -26,6 +26,9 @@ */ public final class S3ListRequest { + /** + * Format for the toString() method: {@value}. + */ private static final String DESCRIPTION = "List %s:/%s delimiter=%s keys=%d requester pays=%s"; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java index 551a8c4375151..69794c04db53c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java @@ -131,7 +131,8 @@ private List objectSummaryKeys() { } /** - * Does this listing have prefixes or objects? + * Does this listing have prefixes or objects after entries with + * tombstones have been stripped? * @param accessors callback for key to path mapping. * @param tombstones Set of tombstone markers, or null if not applicable. * @return true if the reconciled list is non-empty @@ -189,7 +190,7 @@ public boolean representsEmptyDirectory( } /** - * dump the result at debug level only. + * Dmp the result at debug level. * @param log log to use */ public void logAtDebug(Logger log) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java index 44a3237a32fa4..92abc3363cba1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java @@ -65,8 +65,17 @@ public class DirMarkerTracker { private final Map surplusMarkers = new TreeMap<>(); + /** + * Base path of the tracking operation. + */ private final Path basePath; + /** + * Should surplus markers be recorded in + * the {@link #surplusMarkers} map? + */ + private final boolean recordSurplusMarkers; + /** * last parent directory checked. */ @@ -94,12 +103,16 @@ public class DirMarkerTracker { /** * Construct. - * Base path is currently only used for information rather than validating - * paths supplied in other mathods. + *

+ * The base path is currently only used for information rather than + * validating paths supplied in other methods. * @param basePath base path of track + * @param recordSurplusMarkers save surplus markers to a map? */ - public DirMarkerTracker(final Path basePath) { + public DirMarkerTracker(final Path basePath, + boolean recordSurplusMarkers) { this.basePath = basePath; + this.recordSurplusMarkers = recordSurplusMarkers; } /** @@ -112,6 +125,7 @@ public Path getBasePath() { /** * A marker has been found; this may or may not be a leaf. + *

* Trigger a move of all markers above it into the surplus map. * @param path marker path * @param key object key @@ -142,8 +156,9 @@ public List fileFound(Path path, } /** - * A path has been found. Trigger a move of all - * markers above it into the surplus map. + * A path has been found. + *

+ * Declare all markers above it as surplus * @param path marker path * @param key object key * @param source listing source @@ -167,7 +182,13 @@ private List pathFound(Path path, } /** - * Remove all markers from the path and its parents. + * Remove all markers from the path and its parents from the + * {@link #leafMarkers} map. + *

+ * if {@link #recordSurplusMarkers} is true, the marker is + * moved to the surplus map. Not doing this is simply an + * optimisation designed to reduce risk of excess memory consumption + * when renaming (hypothetically) large directory trees. * @param path path to start at * @param removed list of markers removed; is built up during the * recursive operation. @@ -182,13 +203,15 @@ private void removeParentMarkers(final Path path, final Marker value = leafMarkers.remove(path); if (value != null) { // marker is surplus - surplusMarkers.put(path, value); removed.add(value); + if (recordSurplusMarkers) { + surplusMarkers.put(path, value); + } } } /** - * get the map of leaf markers. + * Get the map of leaf markers. * @return all leaf markers. */ public Map getLeafMarkers() { @@ -196,7 +219,9 @@ public Map getLeafMarkers() { } /** - * get the map of surplus markers. + * Get the map of surplus markers. + *

+ * Empty if they were not being recorded. * @return all surplus markers. */ public Map getSurplusMarkers() { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java index e423252833a14..5e0ca91f413d8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java @@ -40,10 +40,14 @@ public final class DirectoryPolicyImpl implements DirectoryPolicy { + private static final Logger LOG = LoggerFactory.getLogger( + DirectoryPolicyImpl.class); + /** * Error string when unable to parse the marker policy option. */ - public static final String UNKNOWN_MARKER_POLICY = "Unknown value of " + public static final String UNKNOWN_MARKER_POLICY = + "Unknown policy in " + DIRECTORY_MARKER_POLICY + ": "; /** @@ -58,9 +62,6 @@ public final class DirectoryPolicyImpl public static final DirectoryPolicy DELETE = new DirectoryPolicyImpl( MarkerPolicy.Delete, (p) -> false); - private static final Logger LOG = LoggerFactory.getLogger( - DirectoryPolicyImpl.class); - /** * Chosen marker policy. */ @@ -72,7 +73,6 @@ public final class DirectoryPolicyImpl */ private final Predicate authoritativeness; - /** * Constructor. * @param markerPolicy marker policy diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java index c3f594229a97d..beb19500812b8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java @@ -57,19 +57,31 @@ /** * A parallelized rename operation which updates the metastore in the * process, through whichever {@link RenameTracker} the store provides. + *

* The parallel execution is in groups of size * {@link InternalConstants#RENAME_PARALLEL_LIMIT}; it is only * after one group completes that the next group is initiated. + *

* Once enough files have been copied that they meet the * {@link InternalConstants#MAX_ENTRIES_TO_DELETE} threshold, a delete * is initiated. * If it succeeds, the rename continues with the next group of files. - * + *

* The RenameTracker has the task of keeping the metastore up to date * as the rename proceeds. - * + *

+ * Directory Markers which have child entries are never copied; only those + * which represent empty directories are copied in the rename. + * The {@link DirMarkerTracker} tracks which markers must be copied, and + * which can simply be deleted from the source. + * As a result: rename always purges all non-leaf directory markers from + * the copied tree. This is to ensure that even if a directory tree + * is copied from an authoritative path to a non-authoritative one + * there is never any contamination of the non-auth path with markers. + *

* The rename operation implements the classic HDFS rename policy of * rename(file, dir) renames the file under the directory. + *

* * There is no validation of input and output paths. * Callers are required to themselves verify that destination is not under @@ -189,7 +201,7 @@ private void completeActiveCopies(String reason) throws IOException { * This object will be deleted when the next page of objects to delete * is posted to S3. Therefore, the COPY must have finished * before that deletion operation takes place. - * This is managed by + * This is managed by: *
    *
  1. * The delete operation only being executed once all active @@ -200,7 +212,7 @@ private void completeActiveCopies(String reason) throws IOException { * been submitted and so is in that thread pool. *
  2. *
- * This method must only be called from the primary thread + * This method must only be called from the primary thread. * @param path path to the object * @param key key of the object. * @param version object version. @@ -282,6 +294,10 @@ public Long execute() throws IOException { // Ok! Time to start try { if (sourceStatus.isFile()) { + // rename the file. The destination path will be different + // from that passed in if the destination is a directory; + // the final value is needed to completely delete parent markers + // when they are not being retained. destCreated = renameFileToDest(); } else { recursiveDirectoryRename(); @@ -382,14 +398,17 @@ protected void recursiveDirectoryRename() throws IOException { // marker. LOG.debug("Deleting fake directory marker at destination {}", destStatus.getPath()); - // TODO: dir marker policy doesn't always need to do this. + // Although the dir marker policy doesn't always need to do this, + // it's simplest just to be consistent here. callbacks.deleteObjectAtPath(destStatus.getPath(), dstKey, false, null); } + Path parentPath = storeContext.keyToPath(srcKey); // Track directory markers so that we know which leaf directories need to be // recreated - DirMarkerTracker dirMarkerTracker = new DirMarkerTracker(parentPath); + DirMarkerTracker dirMarkerTracker = new DirMarkerTracker(parentPath, + false); final RemoteIterator iterator = callbacks.listFilesAndEmptyDirectories(parentPath, @@ -460,6 +479,7 @@ protected void recursiveDirectoryRename() throws IOException { /** * Operations to perform at the end of every loop iteration. + *

* This may block the thread waiting for copies to complete * and/or delete a page of data. */ @@ -491,6 +511,9 @@ private void endOfLoopActions() throws IOException { * data it is at risk of destruction at any point. * If there are lots of empty directory rename operations taking place, * the decision to copy the source may need revisiting. + * Be advised though: the costs of the copy not withstanding, + * it is a lot easier to have one single type of scheduled copy operation + * than have copy and touch calls being scheduled. *

* The duration returned is the time to initiate all copy/delete operations, * including any blocking waits for active copies and paged deletes @@ -603,7 +626,6 @@ private Path copySourceAndUpdateTracker( copyResult = callbacks.copyFile(srcKey, destinationKey, srcAttributes, readContext); } - if (objectRepresentsDirectory(srcKey, len)) { renameTracker.directoryMarkerCopied( sourceFile, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StatusProbeEnum.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StatusProbeEnum.java index f9749a14a3d7d..3b69c7efe3741 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StatusProbeEnum.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StatusProbeEnum.java @@ -21,9 +21,12 @@ import java.util.EnumSet; import java.util.Set; +import org.apache.hadoop.classification.InterfaceAudience; + /** * Enum of probes which can be made of S3. */ +@InterfaceAudience.Private public enum StatusProbeEnum { /** The actual path. */ @@ -52,6 +55,4 @@ public enum StatusProbeEnum { /** Skip the HEAD and only look for directories. */ public static final Set DIRECTORIES = LIST_ONLY; - - } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index 8c033cdc69f6e..f5d8190832887 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -19,8 +19,10 @@ package org.apache.hadoop.fs.s3a.tools; import java.io.FileNotFoundException; +import java.io.FileWriter; import java.io.IOException; import java.io.PrintStream; +import java.io.Writer; import java.net.URI; import java.util.ArrayList; import java.util.Collections; @@ -35,6 +37,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -59,6 +62,7 @@ import static org.apache.hadoop.fs.s3a.Invoker.once; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_INTERRUPTED; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE; +import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_FOUND; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_SUCCESS; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_USAGE; @@ -86,18 +90,33 @@ public final class MarkerTool extends S3GuardTool { /** * Audit sub-command: {@value}. */ - public static final String AUDIT = "audit"; + public static final String OPT_AUDIT = "audit"; /** * Clean Sub-command: {@value}. */ - public static final String CLEAN = "clean"; + public static final String OPT_CLEAN = "clean"; + + /** + * Audit sub-command: {@value}. + */ + public static final String AUDIT = "-" + OPT_AUDIT; + + /** + * Clean Sub-command: {@value}. + */ + public static final String CLEAN = "-" + OPT_CLEAN; /** * Expected number of markers to find: {@value}. */ public static final String OPT_EXPECTED = "expected"; + /** + * Name of a file to save the list of markers to: {@value}. + */ + public static final String OPT_OUT = "out"; + /** * Limit of objects to scan: {@value}. */ @@ -113,7 +132,7 @@ public final class MarkerTool extends S3GuardTool { * Error text when too few arguments are found. */ @VisibleForTesting - static final String TOO_FEW_ARGUMENTS = "Too few arguments"; + static final String E_ARGUMENTS = "Wrong number of arguments: %d"; /** * Constant to use when there is no limit on the number of @@ -131,11 +150,14 @@ public final class MarkerTool extends S3GuardTool { * Usage string: {@value}. */ private static final String USAGE = NAME + + " (-" + OPT_AUDIT + + " | -" + OPT_CLEAN + ")" + " [-" + OPT_EXPECTED + " ]" + + " [-" + OPT_OUT + " ]" + " [-" + OPT_LIMIT + " ]" + " [-" + OPT_NONAUTH + "]" + " [-" + VERBOSE + "]" - + " (audit | clean)" + + " \n" + "\t" + PURPOSE + "\n\n"; @@ -159,12 +181,18 @@ public final class MarkerTool extends S3GuardTool { /** * Constructor. - * @param conf + * @param conf configuration */ public MarkerTool(final Configuration conf) { - super(conf, VERBOSE, OPT_NONAUTH); + super(conf, + OPT_AUDIT, + OPT_CLEAN, + VERBOSE, + OPT_NONAUTH); CommandFormat format = getCommandFormat(); + format.addOptionWithValue(OPT_EXPECTED); format.addOptionWithValue(OPT_LIMIT); + format.addOptionWithValue(OPT_OUT); } @Override @@ -195,50 +223,43 @@ public int run(final String[] args, final PrintStream stream) errorln(getUsage()); throw new ExitUtil.ExitException(EXIT_USAGE, e.getMessage(), e); } - if (parsedArgs.size() < 2) { + if (parsedArgs.size() != 1) { errorln(getUsage()); - throw new ExitUtil.ExitException(EXIT_USAGE, TOO_FEW_ARGUMENTS); + println(out, "Supplied arguments: [" + + parsedArgs.stream() + .collect(Collectors.joining(", ")) + + "]"); + throw new ExitUtil.ExitException(EXIT_USAGE, + String.format(E_ARGUMENTS, parsedArgs.size())); } // read arguments - CommandFormat commandFormat = getCommandFormat(); - verbose = commandFormat.getOpt(VERBOSE); + CommandFormat command = getCommandFormat(); + verbose = command.getOpt(VERBOSE); // How many markers are expected? int expected = 0; - String value = getCommandFormat() - .getOptValue(OPT_EXPECTED); + String value = command.getOptValue(OPT_EXPECTED); if (value != null && !value.isEmpty()) { expected = Integer.parseInt(value); } - // Should the scan also purge surplus markers? - boolean purge; - - // argument 0 is the action - String action = parsedArgs.get(0); - - switch (action) { - case AUDIT: - // audit. no purge; fail if any marker is found - purge = false; - expected = 0; - break; - case CLEAN: - // clean -purge the markers - purge = true; - break; - default: + + // determine the action + boolean audit = command.getOpt(OPT_AUDIT); + boolean clean = command.getOpt(OPT_CLEAN); + if (audit == clean) { + // either both are set or neither are set + // this is equivalent to (not audit xor clean) errorln(getUsage()); throw new ExitUtil.ExitException(EXIT_USAGE, - "Unknown action: " + action); + "Exactly one of " + AUDIT + " and " + CLEAN); } int limit = UNLIMITED_LISTING; - value = getCommandFormat() - .getOptValue(OPT_LIMIT); + value = command.getOptValue(OPT_LIMIT); if (value != null && !value.isEmpty()) { limit = Integer.parseInt(value); } - final String file = parsedArgs.get(1); - Path path = new Path(file); + final String dir = parsedArgs.get(0); + Path path = new Path(dir); URI uri = path.toUri(); if (uri.getPath().isEmpty()) { // fix up empty URI for better CLI experience @@ -248,13 +269,29 @@ public int run(final String[] args, final PrintStream stream) ScanResult result = execute( fs, path, - purge, + clean, expected, limit, - commandFormat.getOpt(OPT_NONAUTH)); + command.getOpt(OPT_NONAUTH)); if (verbose) { dumpFileSystemStatistics(out); } + + // and finally see if the output should be saved to a file + String saveFile = command.getOptValue(OPT_OUT); + if (saveFile != null && !saveFile.isEmpty()) { + println(out, "Saving result to %s", saveFile); + try (Writer writer = new FileWriter(saveFile)) { + final List surplus = result.getTracker() + .getSurplusMarkers() + .keySet() + .stream() + .map(p-> p.toString() + "/") + .sorted() + .collect(Collectors.toList()); + IOUtils.writeLines(surplus, "\n", writer); + } + } return result.exitCode; } @@ -265,7 +302,7 @@ public int run(final String[] args, final PrintStream stream) * @param doPurge purge? * @param expectedMarkerCount expected marker count * @param limit limit of files to scan; -1 for 'unlimited' - * @param nonAuth + * @param nonAuth consider only markers in nonauth paths as errors * @return scan+purge result. * @throws IOException failure */ @@ -306,11 +343,11 @@ ScanResult execute( } catch (UnknownStoreException ex) { // bucket doesn't exist. // replace the stack trace with an error code. - throw new ExitUtil.ExitException(LauncherExitCodes.EXIT_NOT_FOUND, + throw new ExitUtil.ExitException(EXIT_NOT_FOUND, ex.toString(), ex); } catch (FileNotFoundException ex) { - throw new ExitUtil.ExitException(LauncherExitCodes.EXIT_NOT_FOUND, + throw new ExitUtil.ExitException(EXIT_NOT_FOUND, "Not found: " + target, ex); } @@ -396,7 +433,7 @@ private ScanResult scan( // Mission Accomplished result.exitCode = EXIT_SUCCESS; // Now do the work. - DirMarkerTracker tracker = new DirMarkerTracker(path); + DirMarkerTracker tracker = new DirMarkerTracker(path, true); result.tracker = tracker; boolean completed; try (DurationInfo ignored = @@ -423,7 +460,7 @@ private ScanResult scan( path); for (Path markers : surplusMarkers.keySet()) { - println(out, " %s", markers); + println(out, " %s/", markers); } } if (!leafMarkers.isEmpty()) { @@ -432,7 +469,7 @@ private ScanResult scan( suffix(leafMarkers.size()), path); for (Path markers : leafMarkers.keySet()) { - println(out, " %s", markers); + println(out, " %s/", markers); } println(out, "These are required to indicate empty directories"); } @@ -516,7 +553,7 @@ private boolean scanDirectoryTree( String key = storeContext.pathToKey(statusPath); if (status.isDirectory()) { if (verbose) { - println(out, " Directory Marker %s", key); + println(out, " Directory Marker %s/", key); } LOG.debug("{}", key); tracker.markerFound(statusPath, diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md index 426888415a6b0..c6759a0d194e7 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md @@ -16,8 +16,8 @@ ## Critical: this is not backwards compatible! -This document shows how the performance of S3 IO, especially applications -writing many files (hive) or working with versioned S3 buckets can +This document shows how the performance of S3 I/O, especially applications +creating many files (for example Apache Hive) or working with versioned S3 buckets can increase performance by changing the S3A directory marker retention policy. Changing the policy from the default value, `"delete"` _is not backwards compatible_. @@ -90,10 +90,11 @@ file tree" in which files are always stored in "directories" * All files MUST be in a directory "the parent directory" * All directories other than the root directory must be in another directory. * If a directory contains no files or directories, it is "empty" -* When a directory is _listed_, all files and directories in it are enumerated and returned to the caller +* When a directory is _listed_, all files and directories in it are enumerated + and returned to the caller -The S3A connector mocks this entire metaphor by grouping all objects which have +The S3A connector emulates this metaphor by grouping all objects which have the same prefix as if they are in the same directory tree. If there are two objects `a/b/file1` and `a/b/file2` then S3A pretends that there is a @@ -125,8 +126,9 @@ The original Hadoop `s3n://` connector created a Directory Marker -any path endi in `_$folder$` was considered to be a sign that a directory existed. A call to `mkdir(s3n://bucket/a/b)` would create a new marker object `a/b_$folder$` . -The S3A also has directory markers, but it just appends a / to the directory -name, so `mkdir(s3a://bucket/a/b)` would create a new marker object `a/b/` . +The S3A also has directory markers, but it just appends a "/" to the directory +name, so `mkdir(s3a://bucket/a/b)` will create a new marker object `a/b/` . + When a file is created under a path, the directory marker is deleted. And when a file is deleted, if it was the last file in the directory, the marker is recreated. @@ -134,6 +136,8 @@ recreated. And, historically, When a path is listed, if a marker to that path is found, *it has been interpreted as an empty directory.* +It is that little detail which is the cause of the incompatibility issues. + ## Scale issues related to directory markers Creating, deleting and the listing directory markers adds overhead and can slow @@ -182,10 +186,10 @@ This can have adverse effects on those large directories, again. ## How to avoid marker-related problems. -### Presto: there are no directories +### Presto: every path is a directory -In the Presto S3 connectors: `mkdirs()` is a no-op. Instead, whenever you list -any path which isn't an object or a prefix of one more more objects, you get an +In the Presto S3 connectors: `mkdirs()` is a no-op. Instead, whenever it list +any path which isn't an object or a prefix of one more more objects, it gets an empty listing. That is:; by default, every path is an empty directory. Provided no code probes for a directory existing and fails if it is there, this @@ -195,6 +199,10 @@ because they know how their file uses data in S3. ### Hadoop 3.3.1+: marker deletion is now optional +From Hadoop 3.3.1 onwards, th S3A client can be configured to skip deleting +directory markers when creating files under paths. This removes all scalability +problems caused by deleting these markers -however, it is achieved at the expense +of backwards compatibility. ## Controlling marker retention with `fs.s3a.directory.marker.retention` @@ -209,8 +217,7 @@ not deleted. This is *not* backwards compatible *New* `authoritative`: directory markers are deleted _except for files created in "authoritative" directories_. -This is backwards compatible _outside -authoritative directories_. +This is backwards compatible _outside authoritative directories_. Until now, the notion of an "authoritative" directory has only been used as a performance optimization for deployments @@ -248,18 +255,22 @@ will be retained. Syntax ``` -> hadoop s3guard markers - markers [-expected ] [-limit ] [-nonauth] [-verbose] (audit | clean) - View and manipulate S3 directory markers +> hadoop s3guard markers -verbose -nonauth +markers (-audit | -clean) [-expected ] [-out ] [-limit ] [-nonauth] [-verbose] + View and manipulate S3 directory markers + ``` *Options* -| option | meaning | -|--------|---------| -| `-expected ]` | Expected number of markers to find (primarily for testing) | -| `-limit ]` | Limit on number of objects to scan | +| Option | Meaning | +|-------------------------|-------------------------| +| `-audit` | Audit the path for surplus markers | +| `-clean` | Clean all surplus markers under a path | +| `-expected ]` | Expected number of markers to find (primarily for testing) | +| `-limit ]` | Limit the number of objects to scan | | `-nonauth` | Only consider markers in non-authoritative paths as errors | +| `-out ` | Save a list of all markers found to the nominated file | | `-verbose` | Verbose output | *Exit Codes* @@ -267,19 +278,19 @@ Syntax | Code | Meaning | |-------|---------| | 0 | Success | -| 2 | Usage | | 3 | interrupted -the value of `-limit` was reached | +| 42 | Usage | | 46 | Markers were found (see HTTP "406", "unacceptable") | All other non-zero status code also indicate errors of some form or other. -### `markers report` +### `markers -audit` -Audit the path and fail if any markers were found. +Audit the path and fail if any markers were found. ``` -> hadoop s3guard markers -limit 8000 audit s3a://landsat-pds/ +> hadoop s3guard markers -limit 8000 -audit s3a://landsat-pds/ The directory marker policy of s3a://landsat-pds is "Delete" 2020-08-05 13:42:56,079 [main] INFO tools.MarkerTool (DurationInfo.java:(77)) - Starting: marker scan s3a://landsat-pds/ @@ -305,7 +316,8 @@ Example: a verbose audit of a bucket whose policy if authoritative -it is not an are found under the path `/tables`. ``` -> bin/hadoop s3guard markers audit s3a://london/ +> bin/hadoop s3guard markers -audit s3a://london/ + 2020-08-05 18:29:16,473 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143)) - Directory markers will be kept on authoritative paths The directory marker policy of s3a://london is "Authoritative" Authoritative path list is "/tables" @@ -313,16 +325,16 @@ are found under the path `/tables`. 2020-08-05 18:29:21,610 [main] INFO tools.MarkerTool (DurationInfo.java:close(98)) - marker scan s3a://london/: duration 0:02.425s Listed 8 objects under s3a://london/ - Found 3 surplus directory markers under s3a://london/ - s3a://london/tables - s3a://london/tables/tables-4 - s3a://london/tables/tables-4/tables-5 - Found 5 empty directory 'leaf' markers under s3a://london/ - s3a://london/tables/tables-2 - s3a://london/tables/tables-3 - s3a://london/tables/tables-4/tables-5/06 - s3a://london/tables2 - s3a://london/tables3 +Found 3 surplus directory markers under s3a://london/ + s3a://london/tables + s3a://london/tables/tables-4 + s3a://london/tables/tables-4/tables-5 +Found 5 empty directory 'leaf' markers under s3a://london/ + s3a://london/tables/tables-2 + s3a://london/tables/tables-3 + s3a://london/tables/tables-4/tables-5/06 + s3a://london/tables2 + s3a://london/tables3 These are required to indicate empty directories Surplus markers were found -failing audit 2020-08-05 18:29:21,614 [main] INFO util.ExitUtil (ExitUtil.java:terminate(210)) - Exiting with status 46: @@ -335,7 +347,8 @@ The `-nonauth` option does not treat markers under authoritative paths as errors ``` -bin/hadoop s3guard markers -nonauth audit s3a://london/ +bin/hadoop s3guard markers -nonauth -audit s3a://london/ + 2020-08-05 18:31:16,255 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143)) - Directory markers will be kept on authoritative paths The directory marker policy of s3a://london is "Authoritative" Authoritative path list is "/tables" @@ -344,15 +357,15 @@ Authoritative path list is "/tables" Listed 8 objects under s3a://london/ Found 3 surplus directory markers under s3a://london/ - s3a://london/tables - s3a://london/tables/tables-4 - s3a://london/tables/tables-4/tables-5 + s3a://london/tables/ + s3a://london/tables/tables-4/ + s3a://london/tables/tables-4/tables-5/ Found 5 empty directory 'leaf' markers under s3a://london/ - s3a://london/tables/tables-2 - s3a://london/tables/tables-3 - s3a://london/tables/tables-4/tables-5/06 - s3a://london/tables2 - s3a://london/tables3 + s3a://london/tables/tables-2/ + s3a://london/tables/tables-3/ + s3a://london/tables/tables-4/tables-5/06/ + s3a://london/tables2/ + s3a://london/tables3/ These are required to indicate empty directories Ignoring 3 markers in authoritative paths @@ -368,33 +381,33 @@ The `markers clean` command will clean the directory tree of all surplus markers The `-verbose` option prints more detail on the operation as well as some IO statistics ``` -> hadoop s3guard markers -verbose clean s3a://london/ +> hadoop s3guard markers -verbose -clean s3a://london/ 2020-08-05 18:33:25,303 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143)) - Directory markers will be kept on authoritative paths The directory marker policy of s3a://london is "Authoritative" Authoritative path list is "/tables" 2020-08-05 18:33:28,511 [main] INFO tools.MarkerTool (DurationInfo.java:(77)) - Starting: marker scan s3a://london/ Directory Marker tables - Directory Marker tables/tables-2 - Directory Marker tables/tables-3 - Directory Marker tables/tables-4 - Directory Marker tables/tables-4/tables-5 - Directory Marker tables/tables-4/tables-5/06 - Directory Marker tables2 - Directory Marker tables3 + Directory Marker tables/tables-2/ + Directory Marker tables/tables-3/ + Directory Marker tables/tables-4/ + Directory Marker tables/tables-4/tables-5/ + Directory Marker tables/tables-4/tables-5/06/ + Directory Marker tables2/ + Directory Marker tables3/ 2020-08-05 18:33:31,685 [main] INFO tools.MarkerTool (DurationInfo.java:close(98)) - marker scan s3a://london/: duration 0:03.175s Listed 8 objects under s3a://london/ Found 3 surplus directory markers under s3a://london/ - s3a://london/tables - s3a://london/tables/tables-4 - s3a://london/tables/tables-4/tables-5 + s3a://london/tables/ + s3a://london/tables/tables-4/ + s3a://london/tables/tables-4/tables-5/ Found 5 empty directory 'leaf' markers under s3a://london/ - s3a://london/tables/tables-2 - s3a://london/tables/tables-3 - s3a://london/tables/tables-4/tables-5/06 - s3a://london/tables2 - s3a://london/tables3 + s3a://london/tables/tables-2/ + s3a://london/tables/tables-3/ + s3a://london/tables/tables-4/tables-5/06/ + s3a://london/tables2/ + s3a://london/tables3/ These are required to indicate empty directories 3 markers to delete in 1 page of 250 keys/page @@ -408,9 +421,94 @@ object_delete_requests 1 object_list_requests 2 ``` -The `markers clean` command _does not_ delete markers above empty directories -only those which have +The `markers -clean` command _does not_ delete markers above empty directories -only those which have files underneath. If invoked on a path, it will clean up the directory tree into a state where it is safe for older versions of Hadoop to interact with. Note that if invoked with a `-limit` value, surplus markers found during the scan will be removed, -even though the scan will be considered a failure due to the limit being reached. \ No newline at end of file +even though the scan will be considered a failure due to the limit being reached. + +## Advanced Topics + + +### Probing for retention via `PathCapabilities` and `StreamCapabilities` + +An instance of the filesystem can be probed for its directory marker retention ability/ +policy can be probed for through the `org.apache.hadoop.fs.PathCapabilities` interface, +which all FileSystem instances have implemented since Hadoop 3.2. + + +| Probe | Meaning | +|-------------------------|-------------------------| +| `fs.s3a.capability.directory.marker.aware` | Does the filesystem support surplus directory markers? | +| `fs.s3a.capability.directory.marker.keep` | Does the path retain directory markers? | +| `fs.s3a.capability.directory.marker.delete` | Does the path delete directory markers? | + + +The probe `fs.s3a.capability.directory.marker.aware` allows for a filesystem to be +probed to determine if its file listing policy is "aware" of directory marker retention +-that is: it can safely work with S3 buckets where markers have not been deleted. + +The other two probes dynamically query the marker retention behavior of a specific path. + +The S3AFileSystem also implements the `org.apache.hadoop.fs.StreamCapabilities` interface, which +can be used to probe for marker awareness via the `fs.s3a.capability.directory.marker.aware` capability. + +Again, this will be true if-and-only-if the S3A connector is safe to work with S3A buckets/paths where +directories are retained. + +*If an S3A instance, probed by `PathCapabilities` or `StreamCapabilities` for the capability +`fs.s3a.capability.directory.marker.aware` and it returns false, *it is not safe to be used with +S3A paths where markers have been retained*. + +This is programmatic probe -however it can be accessed on the command line via the +external [`cloudstore`](https://github.com/steveloughran/cloudstore) tool: + +``` +> hadoop jar cloudstore-1.0.jar pathcapability fs.s3a.capability.directory.marker.aware s3a://london/ +Probing s3a://london/ for capability fs.s3a.capability.directory.marker.aware + +Using filesystem s3a://london +Path s3a://london/ has capability fs.s3a.capability.directory.marker.aware +``` + +If the exit code of the command is 0, then the hadoop-aws release is safe to work with buckets +where markers have not been deleted. + +The same tool can be used to dynamically probe for the policy. + +Take a bucket with a retention policy of "authoritative" -only paths under `/tables` will have markers retained. + +```xml + + fs.s3a.bucket.london.directory.marker.retention + authoritative + + + fs.s3a.bucket.london.authoritative.path + /tables + ``` +``` + +With this policy the path capability `fs.s3a.capability.directory.marker.keep` will hold under +the path `s3a://london/tables` + +``` +bin/hadoop jar cloudstore-1.0.jar pathcapability fs.s3a.capability.directory.marker.keep s3a://london/tables +Probing s3a://london/tables for capability fs.s3a.capability.directory.marker.keep +2020-08-11 22:03:31,658 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143)) - Directory markers will be kept on authoritative paths +Using filesystem s3a://london +Path s3a://london/tables has capability fs.s3a.capability.directory.marker.keep +``` + +However it will not hold for other paths, so indicating that older Hadoop versions will be safe +to work with data written there by this S3A client. + +``` +bin/hadoop jar $CLOUDSTORE pathcapability fs.s3a.capability.directory.marker.keep s3a://london/tempdir +Probing s3a://london/tempdir for capability fs.s3a.capability.directory.marker.keep +2020-08-11 22:06:56,300 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143)) - Directory markers will be kept on authoritative paths +Using filesystem s3a://london +Path s3a://london/tempdir lacks capability fs.s3a.capability.directory.marker.keep +2020-08-11 22:06:56,308 [main] INFO util.ExitUtil (ExitUtil.java:terminate(210)) - Exiting with status -1: +``` diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java index 90da37d717161..5754fad20207f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java @@ -66,16 +66,29 @@ public void setup() throws Exception { @Override public void teardown() throws Exception { Thread.currentThread().setName("teardown"); - // Perform an audit of the directory tree with the marker tool - // if logging at debug - S3AFileSystem fs = getFileSystem(); + + maybeAuditTestPath(); + + super.teardown(); + describe("closing file system"); + IOUtils.closeStream(getFileSystem()); + } + + /** + * Audit the FS under {@link #methodPath()} if + * the test option {@link #DIRECTORY_MARKER_AUDIT} is + * true. + */ + public void maybeAuditTestPath() { + final S3AFileSystem fs = getFileSystem(); if (fs != null) { try { boolean audit = getTestPropertyBool(fs.getConf(), DIRECTORY_MARKER_AUDIT, false); Path methodPath = methodPath(); if (audit - && !fs.getDirectoryMarkerPolicy().keepDirectoryMarkers(methodPath) + && !fs.getDirectoryMarkerPolicy() + .keepDirectoryMarkers(methodPath) && fs.isDirectory(methodPath)) { MarkerTool.ScanResult result = MarkerTool.execMarkerTool(fs, methodPath, true, 0, UNLIMITED_LISTING, false); @@ -90,10 +103,6 @@ public void teardown() throws Exception { } } } - - super.teardown(); - describe("closing file system"); - IOUtils.closeStream(getFileSystem()); } @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java index 70557588136a8..afb0fd8c55a7b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java @@ -55,6 +55,8 @@ * integration tests. *

* The tests only run if DynamoDB is the metastore. + *

+ * The marker policy is fixed to "delete" */ @FixMethodOrder(MethodSorters.NAME_ASCENDING) public class ITestS3GuardDDBRootOperations extends AbstractS3ATestBase { @@ -87,15 +89,17 @@ protected Configuration createConfiguration() { String bucketName = getTestBucketName(conf); disableFilesystemCaching(conf); - // set a sleep time of 0 on pruning, for speedier test runs. - removeBucketOverrides(bucketName, conf, ENABLE_MULTI_DELETE, + removeBucketOverrides(bucketName, conf, + S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY, + ENABLE_MULTI_DELETE, DIRECTORY_MARKER_POLICY); + conf.set(DIRECTORY_MARKER_POLICY, + DIRECTORY_MARKER_POLICY_DELETE); + // set a sleep time of 0 on pruning, for speedier test runs. conf.setTimeDuration( S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY, 0, TimeUnit.MILLISECONDS); - conf.set(DIRECTORY_MARKER_POLICY, - DIRECTORY_MARKER_POLICY_DELETE); return conf; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java index b5fe743e24c72..4c5b695960a75 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java @@ -18,10 +18,14 @@ package org.apache.hadoop.fs.s3a.tools; +import java.io.File; +import java.io.FileReader; import java.io.IOException; import java.net.URI; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; +import java.util.stream.Collectors; import org.assertj.core.api.Assertions; import org.junit.Test; @@ -37,6 +41,7 @@ import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.ExitUtil; +import org.apache.hadoop.util.StringUtils; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; @@ -60,7 +65,7 @@ public class ITestMarkerTool extends AbstractS3ATestBase { LoggerFactory.getLogger(ITestMarkerTool.class); /** the -verbose option. */ - private static final String V = "-" + VERBOSE; + private static final String V = m(VERBOSE); /** FS which keeps markers. */ private S3AFileSystem keepingFS; @@ -71,14 +76,31 @@ public class ITestMarkerTool extends AbstractS3ATestBase { /** FS which mixes markers; only created in some tests. */ private S3AFileSystem mixedFS; + /** + * How many files to expect. + */ private int expectedFileCount; + /** + * How many markers to expect under dir1. + */ private int expectedMarkersUnderDir1; + /** + * How many markers to expect under dir2. + */ private int expectedMarkersUnderDir2; + /** + * How many markers to expect across both dirs + */ private int expectedMarkers; + /** + * How many markers to expect including the base directory + */ + private int expectedMarkersWithBaseDir; + @Override protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); @@ -289,44 +311,114 @@ public void testRunNoArgs() throws Throwable { @Test public void testRunWrongBucket() throws Throwable { - runToFailure(EXIT_NOT_FOUND, NAME, AUDIT, + runToFailure(EXIT_NOT_FOUND, NAME, + AUDIT, "s3a://this-bucket-does-not-exist-hopefully"); } + /** + * Run with a path that doesn't exist. + */ @Test - public void testRunWrongPath() throws Throwable { - runToFailure(EXIT_NOT_FOUND, NAME, AUDIT, - methodPath().toString()); + public void testRunUnknownPath() throws Throwable { + runToFailure(EXIT_NOT_FOUND, NAME, + AUDIT, + methodPath()); } + /** + * Having both -audit and -clean on the command line is an error. + */ @Test - public void testRunVerboseAudit() throws Throwable { - describe("Run a verbose audit"); + public void testRunTooManyActions() throws Throwable { + runToFailure(EXIT_USAGE, NAME, + AUDIT, CLEAN, + methodPath()); + } + + @Test + public void testRunAuditWithExpectedMarkers() throws Throwable { + describe("Run a verbose audit expecting some markers"); + // a run under the keeping FS will create paths CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); - run(NAME, - V, + final File audit = tempAuditFile(); + run(NAME, V, AUDIT, - createdPaths.base.toString()); - run(NAME, - V, + m(OPT_LIMIT), 0, + m(OPT_OUT), audit, + m(OPT_EXPECTED), expectedMarkersWithBaseDir, + createdPaths.base); + expectMarkersInOutput(audit, expectedMarkersWithBaseDir); + } + + @Test + public void testRunAuditWithExcessMarkers() throws Throwable { + describe("Run a verbose audit failing as surplus markers were found"); + // a run under the keeping FS will create paths + CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); + final File audit = tempAuditFile(); + runToFailure(EXIT_NOT_ACCEPTABLE, NAME, V, AUDIT, - createdPaths.base.toString()); + m(OPT_OUT), audit, + createdPaths.base); + expectMarkersInOutput(audit, expectedMarkersWithBaseDir); } @Test public void testRunLimitedAudit() throws Throwable { - describe("Adurit"); + describe("Audit with a limited number of files (2)"); CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); runToFailure(EXIT_INTERRUPTED, - NAME, - V, - "-" + OPT_LIMIT, "2", + NAME, V, + m(OPT_LIMIT), 2, CLEAN, - createdPaths.base.toString()); - run(NAME, - V, + createdPaths.base); + run(NAME, V, AUDIT, - createdPaths.base.toString()); + createdPaths.base); + } + + /** + * Get a filename for a temp file. + * The generated file is deleted. + * + * @return a file path for a output file + */ + private File tempAuditFile() throws IOException { + final File audit = File.createTempFile("audit", ".txt"); + audit.delete(); + return audit; + } + + /** + * Read the audit output and verify it has the expected number of lines. + * @param auditFile audit file to read + * @param expected expected line count + */ + private void expectMarkersInOutput(final File auditFile, + final int expected) + throws IOException { + final List lines = readOutput(auditFile); + Assertions.assertThat(lines) + .describedAs("Content of %s", auditFile) + .hasSize(expected); + } + + /** + * Read the output file in. Logs the contents at info. + * @param outputFile audit output file. + * @return the lines + */ + public List readOutput(final File outputFile) + throws IOException { + try (FileReader reader = new FileReader(outputFile)) { + final List lines = + org.apache.commons.io.IOUtils.readLines(reader); + + LOG.info("contents of output file {}\n{}", outputFile, + StringUtils.join("\n", lines)); + return lines; + } } private static Path topath(Path base, final String name) { @@ -517,6 +609,7 @@ private CreatedPaths createPaths(FileSystem fs, Path base) // wrap up the expectations. expectedMarkersUnderDir2 = 2; expectedMarkers = expectedMarkersUnderDir1 + expectedMarkersUnderDir2; + expectedMarkersWithBaseDir = expectedMarkers + 1; return r; } @@ -601,10 +694,13 @@ void verifyRenamed(final Path dest, * @return the return code * @throws Exception any exception */ - protected int run(String... args) throws Exception { + protected int run(Object... args) throws Exception { Configuration conf = new Configuration(getConfiguration()); + final String[] argList = Arrays.stream(args) + .map(Object::toString) + .collect(Collectors.toList()).toArray(new String[0]); disableFilesystemCaching(conf); - return S3GuardTool.run(conf, args); + return S3GuardTool.run(conf, argList); } /** @@ -614,7 +710,7 @@ protected int run(String... args) throws Exception { * @param args argument list * @throws Exception any exception */ - protected void runToFailure(int status, String... args) + protected void runToFailure(int status, Object... args) throws Exception { ExitUtil.ExitException ex = intercept(ExitUtil.ExitException.class, @@ -629,4 +725,12 @@ protected void runToFailure(int status, String... args) } } + /** + * Add a - prefix to a string + * @param s string to prefix + * @return a string for passing into the CLI + */ + private static String m(String s) { + return "-" + s; + } } From 11d9d68e1436f6229439cd5f832eeac904be680a Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 12 Aug 2020 17:36:21 +0100 Subject: [PATCH 18/20] HADOOP-13230. bucket-info is marker aware adding the checks to bucket-info, with tests for this, provides a straightforward way to verify that an s3a client is compatible with kept markers. if the command `hadoop s3guard bucket-info -markers aware s3a://bucket/` succeeds then the client has the modifications to support directory markers above files. If it fails as an unknown option: not compatible Change-Id: I2b58501eda160f9c2598bf492908bc6b3bf34f28 --- hadoop-tools/hadoop-aws/pom.xml | 6 + .../hadoop/fs/s3a/impl/DirMarkerTracker.java | 3 + .../hadoop/fs/s3a/impl/DirectoryPolicy.java | 29 +- .../fs/s3a/impl/DirectoryPolicyImpl.java | 22 +- .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 81 +++- .../hadoop/fs/s3a/tools/MarkerTool.java | 11 +- .../tools/hadoop-aws/directory_markers.md | 221 +++++++++-- .../hadoop/fs/s3a/S3ATestConstants.java | 7 +- .../s3guard/AbstractS3GuardToolTestBase.java | 58 ++- .../fs/s3a/s3guard/S3GuardToolTestHelper.java | 61 ++- .../fs/s3a/tools/AbstractMarkerToolTest.java | 334 ++++++++++++++++ .../hadoop/fs/s3a/tools/ITestMarkerTool.java | 356 ++++-------------- .../tools/ITestMarkerToolRootOperations.java | 70 ++++ 13 files changed, 917 insertions(+), 342 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/AbstractMarkerToolTest.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerToolRootOperations.java diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index dc29d66a6ce2f..af8983e2ebe92 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -197,6 +197,8 @@ **/ITestDynamoDBMetadataStoreScale.java **/ITestTerasort*.java + + **/ITestMarkerToolRootOperations.java **/ITestS3GuardDDBRootOperations.java @@ -241,6 +243,10 @@ **/ITestTerasort*.java + + + **/ITestMarkerToolRootOperations.java **/ITestS3AContractRootDir.java **/ITestS3GuardDDBRootOperations.java diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java index 92abc3363cba1..90a976a633c18 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java @@ -47,6 +47,9 @@ * directory of the last file added is cached. This allows for a * quick bailout when many children of the same directory are * returned in a listing. + *

+ * Consult the directory_markers document for details on this feature, + * including terminology. */ public class DirMarkerTracker { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java index 1038f890a787b..da0ee8b55f637 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java @@ -20,9 +20,14 @@ import org.apache.hadoop.fs.Path; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_AUTHORITATIVE; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP; + /** * Interface for Directory Marker policies to implement. */ + public interface DirectoryPolicy { /** @@ -54,14 +59,14 @@ enum MarkerPolicy { *

* This is the classic S3A policy, */ - Delete, + Delete(DIRECTORY_MARKER_POLICY_DELETE), /** * Keep markers. *

* This is Not backwards compatible. */ - Keep, + Keep(DIRECTORY_MARKER_POLICY_KEEP), /** * Keep markers in authoritative paths only. @@ -69,6 +74,24 @@ enum MarkerPolicy { * This is Not backwards compatible within the * auth paths, but is outside these. */ - Authoritative + Authoritative(DIRECTORY_MARKER_POLICY_AUTHORITATIVE); + + /** + * The name of the option as allowed in configuration files + * and marker-aware tooling. + */ + private final String optionName; + + MarkerPolicy(final String optionName) { + this.optionName = optionName; + } + + /** + * Get the option name. + * @return name of the option + */ + public String getOptionName() { + return optionName; + } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java index 5e0ca91f413d8..64901718c5e96 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java @@ -19,7 +19,9 @@ package org.apache.hadoop.fs.s3a.impl; +import java.util.EnumSet; import java.util.Locale; +import java.util.Set; import java.util.function.Predicate; import org.slf4j.Logger; @@ -50,6 +52,12 @@ public final class DirectoryPolicyImpl "Unknown policy in " + DIRECTORY_MARKER_POLICY + ": "; + /** + * All available policies. + */ + private static final Set AVAILABLE_POLICIES = + EnumSet.allOf(MarkerPolicy.class); + /** * Keep all markers. */ @@ -104,14 +112,14 @@ public MarkerPolicy getMarkerPolicy() { @Override public String describe() { - return markerPolicy.toString(); + return markerPolicy.getOptionName(); } @Override public String toString() { final StringBuilder sb = new StringBuilder( "DirectoryMarkerRetention{"); - sb.append("policy='").append(markerPolicy).append('\''); + sb.append("policy='").append(markerPolicy.getOptionName()).append('\''); sb.append('}'); return sb.toString(); } @@ -150,4 +158,14 @@ public static DirectoryPolicy getDirectoryPolicy( } return policy; } + + + /** + * Enumerate all available policies. + * @return set of the policies. + */ + public static Set availablePolicies() { + return AVAILABLE_POLICIES; + } + } 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 0893959728551..f89777f730376 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 @@ -37,6 +37,7 @@ import java.util.Scanner; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import com.amazonaws.services.s3.model.MultipartUpload; import com.google.common.annotations.VisibleForTesting; @@ -45,6 +46,8 @@ import org.slf4j.LoggerFactory; import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.FileStatus; @@ -61,6 +64,7 @@ import org.apache.hadoop.fs.s3a.commit.CommitConstants; import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; +import org.apache.hadoop.fs.s3a.impl.DirectoryPolicyImpl; import org.apache.hadoop.fs.s3a.select.SelectTool; import org.apache.hadoop.fs.s3a.tools.MarkerTool; import org.apache.hadoop.fs.shell.CommandFormat; @@ -83,7 +87,11 @@ /** * CLI to manage S3Guard Metadata Store. + *

+ * Some management tools invoke this class directly. */ +@InterfaceAudience.LimitedPrivate("management tools") +@InterfaceStability.Evolving public abstract class S3GuardTool extends Configured implements Tool, Closeable { private static final Logger LOG = LoggerFactory.getLogger(S3GuardTool.class); @@ -107,7 +115,7 @@ public abstract class S3GuardTool extends Configured implements Tool, "\t" + Diff.NAME + " - " + Diff.PURPOSE + "\n" + "\t" + Fsck.NAME + " - " + Fsck.PURPOSE + "\n" + "\t" + Import.NAME + " - " + Import.PURPOSE + "\n" + - "\t" + MarkerTool.NAME + " - " + MarkerTool.PURPOSE + "\n" + + "\t" + MarkerTool.MARKERS + " - " + MarkerTool.PURPOSE + "\n" + "\t" + Prune.NAME + " - " + Prune.PURPOSE + "\n" + "\t" + SetCapacity.NAME + " - " + SetCapacity.PURPOSE + "\n" + "\t" + SelectTool.NAME + " - " + SelectTool.PURPOSE + "\n" + @@ -1206,7 +1214,8 @@ public int run(String[] args, PrintStream out) throws * Get info about a bucket and its S3Guard integration status. */ public static class BucketInfo extends S3GuardTool { - public static final String NAME = "bucket-info"; + public static final String BUCKET_INFO = "bucket-info"; + public static final String NAME = BUCKET_INFO; public static final String GUARDED_FLAG = "guarded"; public static final String UNGUARDED_FLAG = "unguarded"; public static final String AUTH_FLAG = "auth"; @@ -1214,9 +1223,11 @@ public static class BucketInfo extends S3GuardTool { public static final String ENCRYPTION_FLAG = "encryption"; public static final String MAGIC_FLAG = "magic"; public static final String MARKERS_FLAG = "markers"; + public static final String MARKERS_AWARE = "aware"; public static final String PURPOSE = "provide/check S3Guard information" + " about a specific bucket"; + private static final String USAGE = NAME + " [OPTIONS] s3a://BUCKET\n" + "\t" + PURPOSE + "\n\n" + "Common options:\n" @@ -1228,7 +1239,7 @@ public static class BucketInfo extends S3GuardTool { + " -" + ENCRYPTION_FLAG + " (none, sse-s3, sse-kms) - Require encryption policy\n" + " -" + MARKERS_FLAG - + " (keep, delete, authoritative) - directory markers policy\n"; + + " (aware, keep, delete, authoritative) - directory markers policy\n"; /** * Output when the client cannot get the location of a bucket. @@ -1238,6 +1249,12 @@ public static class BucketInfo extends S3GuardTool { "Location unknown -caller lacks " + RolePolicies.S3_GET_BUCKET_LOCATION + " permission"; + + @VisibleForTesting + public static final String IS_MARKER_AWARE = + "The S3A connector is compatible with buckets where" + + " directory markers are not deleted"; + public BucketInfo(Configuration conf) { super(conf, GUARDED_FLAG, UNGUARDED_FLAG, AUTH_FLAG, NONAUTH_FLAG, MAGIC_FLAG); CommandFormat format = getCommandFormat(); @@ -1428,26 +1445,54 @@ public int run(String[] args, PrintStream out) } // directory markers + processMarkerOption(out, fs, + getCommandFormat().getOptValue(MARKERS_FLAG)); + + // and finally flush the output and report a success. + out.flush(); + return SUCCESS; + } + + /** + * Validate the marker options. + * @param out output stream + * @param fs filesystem + * @param path test path + * @param marker desired marker option -may be null. + */ + private void processMarkerOption(final PrintStream out, + final S3AFileSystem fs, + final String marker) { DirectoryPolicy markerPolicy = fs.getDirectoryMarkerPolicy(); String desc = markerPolicy.describe(); - println(out, "%nDirectory marker policy is %s", desc); + println(out, "%nThe directory marker policy is \"%s\"%n", desc); DirectoryPolicy.MarkerPolicy mp = markerPolicy.getMarkerPolicy(); - String desiredMarker = getCommandFormat() - .getOptValue(MARKERS_FLAG); - desiredMarker = desiredMarker == null + + String desiredMarker = marker == null ? "" - : desiredMarker.trim().toLowerCase(Locale.ENGLISH); - if (!desiredMarker.isEmpty() - && !desiredMarker.equals(mp.toString())) { - throw badState("Bucket %s: required marker policy is %s" - + " but actual policy is %s", - fsUri, desiredMarker, mp); + : marker.trim(); + final String optionName = mp.getOptionName(); + if (!desiredMarker.isEmpty()) { + if (MARKERS_AWARE.equalsIgnoreCase(desiredMarker)) { + // simple awareness test -provides a way to validate compatibility + // on the command line + println(out, IS_MARKER_AWARE); + String pols = DirectoryPolicyImpl.availablePolicies() + .stream() + .map(DirectoryPolicy.MarkerPolicy::getOptionName) + .collect(Collectors.joining(", ")); + println(out, "Available Policies: %s", pols); + + } else { + // compare with current policy + if (!optionName.equalsIgnoreCase(desiredMarker)) { + throw badState("Bucket %s: required marker policy is \"%s\"" + + " but actual policy is \"%s\"", + fs.getUri(), desiredMarker, optionName); + } + } } - - // and finally flush the output and report a success. - out.flush(); - return SUCCESS; } private String printOption(PrintStream out, @@ -2053,7 +2098,7 @@ public static int run(Configuration conf, String...args) throws case Diff.NAME: command = new Diff(conf); break; - case MarkerTool.NAME: + case MarkerTool.MARKERS: command = new MarkerTool(conf); break; case Prune.NAME: diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index f5d8190832887..3cffe5ed83667 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -38,6 +38,8 @@ import org.slf4j.LoggerFactory; import org.apache.commons.io.IOUtils; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -52,7 +54,6 @@ import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; import org.apache.hadoop.fs.shell.CommandFormat; -import org.apache.hadoop.service.launcher.LauncherExitCodes; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.ExitUtil; @@ -72,6 +73,8 @@ * This tool does not go anywhere near S3Guard; its scan bypasses any * metastore as we are explicitly looking for marker objects. */ +@InterfaceAudience.LimitedPrivate("management tools") +@InterfaceStability.Unstable public final class MarkerTool extends S3GuardTool { private static final Logger LOG = LoggerFactory.getLogger(MarkerTool.class); @@ -79,7 +82,7 @@ public final class MarkerTool extends S3GuardTool { /** * Name of this tool: {@value}. */ - public static final String NAME = "markers"; + public static final String MARKERS = "markers"; /** * Purpose of this tool: {@value}. @@ -149,7 +152,7 @@ public final class MarkerTool extends S3GuardTool { /** * Usage string: {@value}. */ - private static final String USAGE = NAME + private static final String USAGE = MARKERS + " (-" + OPT_AUDIT + " | -" + OPT_CLEAN + ")" + " [-" + OPT_EXPECTED + " ]" @@ -202,7 +205,7 @@ public String getUsage() { @Override public String getName() { - return NAME; + return MARKERS; } @Override diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md index c6759a0d194e7..4946f381b545b 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md @@ -22,18 +22,23 @@ increase performance by changing the S3A directory marker retention policy. Changing the policy from the default value, `"delete"` _is not backwards compatible_. -Versions of Hadoop which are incompatible with other marker retention policies +Versions of Hadoop which are incompatible with other marker retention policies, +as of August 2020. ------------------------------------------------------- | Branch | Compatible Since | Future Fix Planned? | |------------|------------------|---------------------| | Hadoop 2.x | | NO | | Hadoop 3.0 | | NO | -| Hadoop 3.1 | | Yes | -| Hadoop 3.2 | | Yes | +| Hadoop 3.1 | check | Yes | +| Hadoop 3.2 | check | Yes | | Hadoop 3.3 | 3.3.1 | Done | ------------------------------------------------------- +The `s3guard bucket-info` tool [can be used to verify support](#bucket-info). +This allows for a command line check of compatibility, including +in scripts. + External Hadoop-based applications should also be assumed to be incompatible unless otherwise stated/known. @@ -42,7 +47,7 @@ It is only safe change the directory marker policy if the following 1. You know exactly which applications are writing to and reading from (including backing up) an S3 bucket. -2. You know all applications which read data from the bucket are as compatible. +2. You know all applications which read data from the bucket are compatible. ### Applications backing up data. @@ -54,12 +59,16 @@ distcp is used, it _must_ be from a compatible hadoop version. ### How will incompatible applications/versions fail? Applications using an incompatible version of the S3A connector will mistake -directories containing data for empty directories. This means that +directories containing data for empty directories. This means that: * Listing directories/directory trees may exclude files which exist. * Queries across the data will miss data files. * Renaming a directory to a new location may exclude files underneath. +The failures are silent: there is no error message, stack trace or +other warning that files may have been missed. They simply aren't +found. + ### If an application has updated a directory tree incompatibly-- what can be done? There's a tool on the hadoop command line, [marker tool](#marker-tool) which can audit @@ -161,35 +170,34 @@ call made whenever a file is created. 1. The number of the objects listed in each request is that of the number of parent directories: deeper trees create longer requests. -1. Every single object listed in the delete request is considered to be a write +2. Every single object listed in the delete request is considered to be a write operation. -1. In versioned S3 buckets, tombstone markers are added to the S3 indices even +3. In versioned S3 buckets, tombstone markers are added to the S3 indices even if no object was deleted. -1. There's also the overhead of actually issuing the request and awaiting the +4. There's also the overhead of actually issuing the request and awaiting the response. Issue #2 has turned out to cause significant problems on some interactions with large hive tables: Because each object listed in a DELETE call is treated as one operation, and -there is (as of summer 2020) a limit of -[3500 write requests/second in a directory tree](https://docs.aws.amazon.com/AmazonS3/latest/dev/optimizing-performance.html), -when writing many files to a deep directory tree, it is the delete calls which +there is (as of summer 2020) a limit of 3500 write requests/second in a directory +tree. +When writing many files to a deep directory tree, it is the delete calls which create throttling problems. -For a symptom of this, see HADOOP-16829, -[Large DeleteObject requests are their own Thundering Herd](https://issues.apache.org/jira/browse/HADOOP-16823). -The tombstone markers have follow-on consequences -it makes listings slower. +The tombstone markers have follow-on consequences -it makes listings against +S3 versioned buckets slower. This can have adverse effects on those large directories, again. ## How to avoid marker-related problems. ### Presto: every path is a directory -In the Presto S3 connectors: `mkdirs()` is a no-op. Instead, whenever it list -any path which isn't an object or a prefix of one more more objects, it gets an +In the Presto S3 connectors: `mkdirs()` is a no-op. Instead, whenever it lists +any path which isn't an object or a prefix of one more more objects, it returns an empty listing. That is:; by default, every path is an empty directory. Provided no code probes for a directory existing and fails if it is there, this @@ -229,7 +237,7 @@ can switch to the higher-performance mode for those specific directories. Only the default setting, `fs.s3a.directory.marker.retention = delete` is compatible with existing Hadoop releases. -## Directory Markers and S3Guard +## Directory Markers and S3Guard Applications which interact with S3A in S3A clients with S3Guard enabled still create and delete markers. There's no attempt to skip operations, such as by having @@ -245,6 +253,105 @@ then an S3A connector with a retention policy of `fs.s3a.directory.marker.retent only use in managed applications where all clients are using the same version of hadoop, and configured consistently. +## Verifying marker policy with `s3guard bucket-info` + +The `bucket-info` command has been enhanced to support verification from the command +line of bucket policies via the `-marker` option + + +| option | verifies | +|--------|--------| +| `-markers aware` | the hadoop release is "aware" of directory markers | +| `-markers delete` | directory markers are deleted | +| `-markers keep` | directory markers are kept (not backwards compatible) | +| `-markers authoritative` | directory markers are kept in authoritative paths| + +All releases of Hadoop which have been updated to be marker aware will support the `-markers aware` option. + + +1. Updated releases which do not support switching marker retention policy will also support the +`-markers delete` option. + + +Example: `s3guard bucket-info -markers aware` on a compatible release. + +``` +> hadoop s3guard bucket-info -markers aware s3a://landsat-pds/ + Filesystem s3a://landsat-pds + Location: us-west-2 + Filesystem s3a://landsat-pds is not using S3Guard + +... + + Security + Delegation token support is disabled + + The directory marker policy is "delete" + + The S3A connector is compatible with buckets where directory markers are not deleted + Available Policies: delete, keep, authoritative +``` + +The same command will fail on older releases, because the `-markers` option +is unknown + +``` +> hadoop s3guard bucket-info -markers aware s3a://landsat-pds/ +Illegal option -markers +Usage: hadoop bucket-info [OPTIONS] s3a://BUCKET + provide/check S3Guard information about a specific bucket + +Common options: + -guarded - Require S3Guard + -unguarded - Force S3Guard to be disabled + -auth - Require the S3Guard mode to be "authoritative" + -nonauth - Require the S3Guard mode to be "non-authoritative" + -magic - Require the S3 filesystem to be support the "magic" committer + -encryption -require {none, sse-s3, sse-kms} - Require encryption policy + +When possible and not overridden by more specific options, metadata +repository information will be inferred from the S3A URL (if provided) + +Generic options supported are: + -conf - specify an application configuration file + -D - define a value for a given property + +2020-08-12 16:47:16,579 [main] INFO util.ExitUtil (ExitUtil.java:terminate(210)) - Exiting with status 42: Illegal option -markers +```` + +A specific policy check verifies that the connector is configured as desired + +``` +> hadoop s3guard bucket-info -markers delete s3a://landsat-pds/ +Filesystem s3a://landsat-pds +Location: us-west-2 +Filesystem s3a://landsat-pds is not using S3Guard + +... + +The directory marker policy is "delete" +``` + +When probing for a specific policy, the error code "46" is returned if the active policy +does not match that requested: + +``` +> hadoop s3guard bucket-info -markers keep s3a://landsat-pds/ +Filesystem s3a://landsat-pds +Location: us-west-2 +Filesystem s3a://landsat-pds is not using S3Guard + +... + +Security + Delegation token support is disabled + +The directory marker policy is "delete" + +2020-08-12 17:14:30,563 [main] INFO util.ExitUtil (ExitUtil.java:terminate(210)) - Exiting with status 46: 46: Bucket s3a://landsat-pds: required marker policy is "keep" but actual policy is "delete" +``` + + ## The marker tool:`hadoop s3guard markers` The marker tool aims to help migration by scanning/auditing directory trees @@ -428,14 +535,14 @@ where it is safe for older versions of Hadoop to interact with. Note that if invoked with a `-limit` value, surplus markers found during the scan will be removed, even though the scan will be considered a failure due to the limit being reached. -## Advanced Topics +## Advanced Topics -### Probing for retention via `PathCapabilities` and `StreamCapabilities` +### Probing for retention via `PathCapabilities` and `StreamCapabilities` An instance of the filesystem can be probed for its directory marker retention ability/ policy can be probed for through the `org.apache.hadoop.fs.PathCapabilities` interface, -which all FileSystem instances have implemented since Hadoop 3.2. +which all FileSystem classes have supported since Hadoop 3.3. | Probe | Meaning | @@ -451,7 +558,7 @@ probed to determine if its file listing policy is "aware" of directory marker re The other two probes dynamically query the marker retention behavior of a specific path. -The S3AFileSystem also implements the `org.apache.hadoop.fs.StreamCapabilities` interface, which +The `S3AFileSystem` class also implements the `org.apache.hadoop.fs.StreamCapabilities` interface, which can be used to probe for marker awareness via the `fs.s3a.capability.directory.marker.aware` capability. Again, this will be true if-and-only-if the S3A connector is safe to work with S3A buckets/paths where @@ -466,13 +573,14 @@ external [`cloudstore`](https://github.com/steveloughran/cloudstore) tool: ``` > hadoop jar cloudstore-1.0.jar pathcapability fs.s3a.capability.directory.marker.aware s3a://london/ + Probing s3a://london/ for capability fs.s3a.capability.directory.marker.aware Using filesystem s3a://london Path s3a://london/ has capability fs.s3a.capability.directory.marker.aware ``` -If the exit code of the command is 0, then the hadoop-aws release is safe to work with buckets +If the exit code of the command is `0`, then the S3A is safe to work with buckets where markers have not been deleted. The same tool can be used to dynamically probe for the policy. @@ -496,7 +604,8 @@ the path `s3a://london/tables` ``` bin/hadoop jar cloudstore-1.0.jar pathcapability fs.s3a.capability.directory.marker.keep s3a://london/tables Probing s3a://london/tables for capability fs.s3a.capability.directory.marker.keep -2020-08-11 22:03:31,658 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143)) - Directory markers will be kept on authoritative paths +2020-08-11 22:03:31,658 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143)) + - Directory markers will be kept on authoritative paths Using filesystem s3a://london Path s3a://london/tables has capability fs.s3a.capability.directory.marker.keep ``` @@ -505,10 +614,72 @@ However it will not hold for other paths, so indicating that older Hadoop versio to work with data written there by this S3A client. ``` -bin/hadoop jar $CLOUDSTORE pathcapability fs.s3a.capability.directory.marker.keep s3a://london/tempdir +bin/hadoop jar cloudstore-1.0.jar pathcapability fs.s3a.capability.directory.marker.keep s3a://london/tempdir Probing s3a://london/tempdir for capability fs.s3a.capability.directory.marker.keep -2020-08-11 22:06:56,300 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143)) - Directory markers will be kept on authoritative paths +2020-08-11 22:06:56,300 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143)) + - Directory markers will be kept on authoritative paths Using filesystem s3a://london Path s3a://london/tempdir lacks capability fs.s3a.capability.directory.marker.keep 2020-08-11 22:06:56,308 [main] INFO util.ExitUtil (ExitUtil.java:terminate(210)) - Exiting with status -1: ``` + + +## Glossary + +#### Directory Marker + +An object in an S3 bucket with a trailing "/", used to indicate that there is a directory at that location. +These are necessary to maintain expectations about directories in an object store: + +1. After `mkdirs(path)`, `exists(path)` holds. +1. After `rm(path/*)`, `exists(path)` holds. + +In previous releases of Hadoop, the marker created by a `mkdirs()` operation was deleted after a file was created. +Rather than make a slow HEAD probe + optional marker DELETE of every parent path element, HADOOP-13164 switched +to enumerating all parent paths and issuing a single bulk DELETE request. +This is faster under light load, but +as each row in the delete consumes one write operation on the allocated IOPs of that bucket partition, creates +load issues when many worker threads/processes are writing to files. +This problem is bad on Apache Hive as: +* The hive partition structure places all files within the same S3 partition. +* As they are deep structures, there are many parent entries to include in the bulk delete calls. +* It's creating a lot temporary files, and still uses rename to commit output. + +Apache Spark has less of an issue when an S3A committer is used -although the partition structure +is the same, the delayed manifestation of output files reduces load. + +#### Leaf Marker + +A directory marker which has not files or directory marker objects underneath. +It genuinely represents an empty directory. + +#### Surplus Marker + +A directory marker which is above one or more files, and so is superfluous. +These are the markers which were traditionally deleted; now it is optional. + +Older versions of Hadoop mistake such surplus markers as Leaf Markers. + +#### Versioned Bucket + +An S3 Bucket which has Object Versioning enabled. + +This provides a backup and recovery mechanism for data within the same +bucket: older objects can be listed and restored through the AWS S3 console +and some applications. + +## References + + + +* [HADOOP-13164](https://issues.apache.org/jira/browse/HADOOP-13164). _Optimize S3AFileSystem::deleteUnnecessaryFakeDirectories._ + +* [HADOOP-13230](https://issues.apache.org/jira/browse/HADOOP-13230). _S3A to optionally retain directory markers_ + +* [HADOOP-16090](https://issues.apache.org/jira/browse/HADOOP-16090).. _S3A Client to add explicit support for versioned stores._ + +* [HADOOP-16823](https://issues.apache.org/jira/browse/HADOOP-16823). _Large DeleteObject requests are their own Thundering Herd_ + +* [Object Versioning](https://docs.aws.amazon.com/AmazonS3/latest/dev/Versioning.html). _Using versioning_ + +* [Optimizing Performance](https://docs.aws.amazon.com/AmazonS3/latest/dev/optimizing-performance.html). _Best Practices Design Patterns: Optimizing Amazon S3 Performance_ diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java index 3dc5463376e29..c5670b09c3db5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java @@ -87,10 +87,15 @@ public interface S3ATestConstants { */ String KEY_CSVTEST_FILE = S3A_SCALE_TEST + "csvfile"; + /** + * The landsat bucket: {@value}. + */ + String LANDSAT_BUCKET = "s3a://landsat-pds/"; + /** * Default path for the multi MB test file: {@value}. */ - String DEFAULT_CSVTEST_FILE = "s3a://landsat-pds/scene_list.gz"; + String DEFAULT_CSVTEST_FILE = LANDSAT_BUCKET + "scene_list.gz"; /** * Name of the property to define the timeout for scale tests: {@value}. 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 3e187a1515630..86b31d8860fa1 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 @@ -60,10 +60,14 @@ 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.S3AUtils.clearBucketOption; +import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.BucketInfo.IS_MARKER_AWARE; import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.E_BAD_STATE; import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.INVALID_ARGUMENT; import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.SUCCESS; import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.exec; +import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.runS3GuardCommand; +import static org.apache.hadoop.fs.s3a.tools.MarkerTool.MARKERS; +import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE; import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** @@ -124,7 +128,7 @@ protected static void expectResult(int expected, public static String expectSuccess( String message, S3GuardTool tool, - String... args) throws Exception { + Object... args) throws Exception { ByteArrayOutputStream buf = new ByteArrayOutputStream(); exec(SUCCESS, message, tool, buf, args); return buf.toString(); @@ -137,9 +141,9 @@ public static String expectSuccess( * @return the return code * @throws Exception any exception */ - protected int run(Configuration conf, String... args) + protected int run(Configuration conf, Object... args) throws Exception { - return S3GuardTool.run(conf, args); + return runS3GuardCommand(conf, args); } /** @@ -149,8 +153,8 @@ protected int run(Configuration conf, String... args) * @return the return code * @throws Exception any exception */ - protected int run(String... args) throws Exception { - return S3GuardTool.run(getConfiguration(), args); + protected int run(Object... args) throws Exception { + return runS3GuardCommand(getConfiguration(), args); } /** @@ -160,11 +164,12 @@ protected int run(String... args) throws Exception { * @param args argument list * @throws Exception any exception */ - protected void runToFailure(int status, String... args) + protected void runToFailure(int status, Object... args) throws Exception { + final Configuration conf = getConfiguration(); ExitUtil.ExitException ex = intercept(ExitUtil.ExitException.class, - () -> run(args)); + () -> runS3GuardCommand(conf, args)); if (ex.status != status) { throw ex; } @@ -445,6 +450,44 @@ public void testBucketInfoUnguarded() throws Exception { info.contains("S3A Client")); } + /** + * Verify that the {@code -markers aware} option works. + * This test case is in this class for ease of backporting. + */ + @Test + public void testBucketInfoMarkerAware() throws Throwable { + final Configuration conf = getConfiguration(); + URI fsUri = getFileSystem().getUri(); + + // run a bucket info command and look for + // confirmation that it got the output from DDB diags + S3GuardTool.BucketInfo infocmd = toClose(new S3GuardTool.BucketInfo(conf)); + String info = exec(infocmd, S3GuardTool.BucketInfo.NAME, + "-" + MARKERS, S3GuardTool.BucketInfo.MARKERS_AWARE, + fsUri.toString()); + + assertTrue("Output should contain information about S3A client " + info, + info.contains(IS_MARKER_AWARE)); + } + + /** + * Verify that the {@code -markers} option fails on unknown options. + * This test case is in this class for ease of backporting. + */ + @Test + public void testBucketInfoMarkerPolicyUnknown() throws Throwable { + final Configuration conf = getConfiguration(); + URI fsUri = getFileSystem().getUri(); + + // run a bucket info command and look for + // confirmation that it got the output from DDB diags + S3GuardTool.BucketInfo infocmd = toClose(new S3GuardTool.BucketInfo(conf)); + intercept(ExitUtil.ExitException.class, ""+ EXIT_NOT_ACCEPTABLE, () -> + exec(infocmd, S3GuardTool.BucketInfo.NAME, + "-" + MARKERS, "unknown", + fsUri.toString())); + } + @Test public void testSetCapacityFailFastIfNotGuarded() throws Exception{ Configuration conf = getConfiguration(); @@ -654,4 +697,5 @@ public void testInitFailsIfNoBucketNameOrDDBTableSet() throws Exception { assertEquals("Mismatched s3 outputs: " + actualOut, filesOnS3, actualOnS3); assertFalse("Diff contained duplicates", duplicates); } + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardToolTestHelper.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardToolTestHelper.java index 4a5e55eb61e3c..89b4051de8776 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardToolTestHelper.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardToolTestHelper.java @@ -20,12 +20,16 @@ import java.io.ByteArrayOutputStream; import java.io.PrintStream; +import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.ExitCodeProvider; +import org.apache.hadoop.util.ExitUtil; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.junit.Assert.assertEquals; /** @@ -48,7 +52,7 @@ private S3GuardToolTestHelper() { * @param args argument list * @throws Exception on any failure */ - public static String exec(S3GuardTool cmd, String... args) throws Exception { + public static String exec(S3GuardTool cmd, Object... args) throws Exception { return expectExecResult(0, cmd, args); } @@ -64,7 +68,7 @@ public static String exec(S3GuardTool cmd, String... args) throws Exception { public static String expectExecResult( final int expectedResult, final S3GuardTool cmd, - final String... args) throws Exception { + final Object... args) throws Exception { ByteArrayOutputStream buf = new ByteArrayOutputStream(); try { exec(expectedResult, "", cmd, buf, args); @@ -77,6 +81,17 @@ public static String expectExecResult( } } + /** + * Given an array of objects, conver to an array of strings. + * @param oargs object args + * @return string equivalent + */ + public static String[] varargsToString(final Object[] oargs) { + return Arrays.stream(oargs) + .map(Object::toString) + .toArray(String[]::new); + } + /** * Execute a command, saving the output into the buffer. * @param expectedResult expected result of the command. @@ -91,8 +106,9 @@ public static void exec(final int expectedResult, final String errorText, final S3GuardTool cmd, final ByteArrayOutputStream buf, - final String... args) + final Object... oargs) throws Exception { + final String[] args = varargsToString(oargs); LOG.info("exec {}", (Object) args); int r; try (PrintStream out = new PrintStream(buf)) { @@ -116,4 +132,43 @@ public static void exec(final int expectedResult, } } + /** + * Run a S3GuardTool command from a varags list. + *

+ * Warning: if the filesystem is retrieved from the cache, + * it will be closed afterwards. + * @param conf configuration + * @param args argument list + * @return the return code + * @throws Exception any exception + */ + public static int runS3GuardCommand(Configuration conf, Object... args) + throws Exception { + return S3GuardTool.run(conf, varargsToString(args)); + } + + /** + * Run a S3GuardTool command from a varags list, catch any raised + * ExitException and verify the status code matches that expected. + * @param conf configuration + * @param status expected status code of the exception + * @param args argument list + * @throws Exception any exception + */ + public static void runS3GuardCommandToFailure(Configuration conf, + int status, + Object... args) throws Exception { + + ExitUtil.ExitException ex = + intercept(ExitUtil.ExitException.class, + () -> { + int ec = runS3GuardCommand(conf, args); + if (ec != 0) { + throw new ExitUtil.ExitException(ec, "exit code " + ec); + } + }); + if (ex.status != status) { + throw ex; + } + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/AbstractMarkerToolTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/AbstractMarkerToolTest.java new file mode 100644 index 0000000000000..c3836e5db9186 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/AbstractMarkerToolTest.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.tools; + +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.net.URI; +import java.util.List; + +import org.assertj.core.api.Assertions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.util.StringUtils; + +import static org.apache.hadoop.fs.s3a.Constants.*; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; +import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.VERBOSE; +import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.runS3GuardCommand; +import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.runS3GuardCommandToFailure; +import static org.apache.hadoop.fs.s3a.tools.MarkerTool.UNLIMITED_LISTING; + +/** + * Class for marker tool tests -sets up keeping/deleting filesystems, + * has methods to invoke + */ +public class AbstractMarkerToolTest extends AbstractS3ATestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(AbstractMarkerToolTest.class); + + /** the -verbose option. */ + protected static final String V = AbstractMarkerToolTest.m(VERBOSE); + + /** FS which keeps markers. */ + private S3AFileSystem keepingFS; + + /** FS which deletes markers. */ + private S3AFileSystem deletingFS; + + /** FS which mixes markers; only created in some tests. */ + private S3AFileSystem mixedFS; + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + String bucketName = getTestBucketName(conf); + removeBaseAndBucketOverrides(bucketName, conf, + S3A_BUCKET_PROBE, + DIRECTORY_MARKER_POLICY, + S3_METADATA_STORE_IMPL, + METADATASTORE_AUTHORITATIVE, + AUTHORITATIVE_PATH); + // base FS is legacy + conf.set(DIRECTORY_MARKER_POLICY, DIRECTORY_MARKER_POLICY_DELETE); + conf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL); + + // turn off bucket probes for a bit of speedup in the connectors we create. + conf.setInt(S3A_BUCKET_PROBE, 0); + return conf; + } + + @Override + public void setup() throws Exception { + super.setup(); + setKeepingFS(createFS(DIRECTORY_MARKER_POLICY_KEEP, null)); + setDeletingFS(createFS(DIRECTORY_MARKER_POLICY_DELETE, null)); + } + + @Override + public void teardown() throws Exception { + // do this ourselves to avoid audits teardown failing + // when surplus markers are found + deleteTestDirInTeardown(); + super.teardown(); + IOUtils.cleanupWithLogger(LOG, getKeepingFS(), + getMixedFS(), getDeletingFS()); + + } + + /** + * FS which deletes markers. + */ + public S3AFileSystem getDeletingFS() { + return deletingFS; + } + + public void setDeletingFS(final S3AFileSystem deletingFS) { + this.deletingFS = deletingFS; + } + + /** + * FS which keeps markers. + */ + protected S3AFileSystem getKeepingFS() { + return keepingFS; + } + + private void setKeepingFS(S3AFileSystem keepingFS) { + this.keepingFS = keepingFS; + } + + /** only created on demand. */ + private S3AFileSystem getMixedFS() { + return mixedFS; + } + + protected void setMixedFS(S3AFileSystem mixedFS) { + this.mixedFS = mixedFS; + } + + /** + * Get a filename for a temp file. + * The generated file is deleted. + * + * @return a file path for a output file + */ + protected File tempAuditFile() throws IOException { + final File audit = File.createTempFile("audit", ".txt"); + audit.delete(); + return audit; + } + + /** + * Read the audit output and verify it has the expected number of lines. + * @param auditFile audit file to read + * @param expected expected line count + */ + protected void expectMarkersInOutput(final File auditFile, + final int expected) + throws IOException { + final List lines = readOutput(auditFile); + Assertions.assertThat(lines) + .describedAs("Content of %s", auditFile) + .hasSize(expected); + } + + /** + * Read the output file in. Logs the contents at info. + * @param outputFile audit output file. + * @return the lines + */ + protected List readOutput(final File outputFile) + throws IOException { + try (FileReader reader = new FileReader(outputFile)) { + final List lines = + org.apache.commons.io.IOUtils.readLines(reader); + + LOG.info("contents of output file {}\n{}", outputFile, + StringUtils.join("\n", lines)); + return lines; + } + } + + /** + * Create a new FS with given marker policy and path. + * This filesystem MUST be closed in test teardown. + * @param markerPolicy markers + * @param authPath authoritative path. If null: no path. + * @return a new FS. + */ + protected S3AFileSystem createFS(String markerPolicy, + String authPath) throws Exception { + S3AFileSystem testFS = getFileSystem(); + Configuration conf = new Configuration(testFS.getConf()); + URI testFSUri = testFS.getUri(); + String bucketName = getTestBucketName(conf); + removeBucketOverrides(bucketName, conf, + DIRECTORY_MARKER_POLICY, + S3_METADATA_STORE_IMPL, + BULK_DELETE_PAGE_SIZE, + AUTHORITATIVE_PATH); + if (authPath != null) { + conf.set(AUTHORITATIVE_PATH, authPath); + } + // Use a very small page size to force the paging + // code to be tested. + conf.setInt(BULK_DELETE_PAGE_SIZE, 2); + conf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL); + conf.set(DIRECTORY_MARKER_POLICY, markerPolicy); + S3AFileSystem fs2 = new S3AFileSystem(); + fs2.initialize(testFSUri, conf); + LOG.info("created new filesystem with policy {} and auth path {}", + markerPolicy, + (authPath == null ? "(null)": authPath)); + return fs2; + } + + /** + * Execute the marker tool, expecting the execution to succeed. + * @param sourceFS filesystem to use + * @param path path to scan + * @param doPurge should markers be purged + * @param expectedMarkerCount number of markers expected + * @return the result + */ + protected MarkerTool.ScanResult markerTool( + final FileSystem sourceFS, + final Path path, + final boolean doPurge, + final int expectedMarkerCount) + throws IOException { + return markerTool(0, sourceFS, path, doPurge, + expectedMarkerCount, + UNLIMITED_LISTING, false); + } + + /** + * Run a S3GuardTool command from a varags list and the + * configuration returned by {@code getConfiguration()}. + * @param args argument list + * @return the return code + * @throws Exception any exception + */ + protected int run(Object... args) throws Exception { + return runS3GuardCommand(uncachedFSConfig(getConfiguration()), args); + } + + /** + * Take a configuration, copy it and disable FS Caching on + * the new one. + * @param conf source config + * @return a new, patched, config + */ + protected Configuration uncachedFSConfig(final Configuration conf) { + Configuration c = new Configuration(conf); + disableFilesystemCaching(c); + return c; + } + + /** + * given an FS instance, create a matching configuration where caching + * is disabled. + * @param fs source + * @return new config. + */ + protected Configuration uncachedFSConfig(final FileSystem fs) { + return uncachedFSConfig(fs.getConf()); + } + + /** + * Run a S3GuardTool command from a varags list, catch any raised + * ExitException and verify the status code matches that expected. + * @param status expected status code of the exception + * @param args argument list + * @throws Exception any exception + */ + protected void runToFailure(int status, Object... args) + throws Exception { + Configuration conf = uncachedFSConfig(getConfiguration()); + runS3GuardCommandToFailure(conf, status, args); + } + + /** + * Given a base and a filename, create a new path. + * @param base base path + * @param name name: may be empty, in which case the base path is returned + * @return a path + */ + protected static Path toPath(final Path base, final String name) { + return name.isEmpty() ? base : new Path(base, name); + } + + /** + * Execute the marker tool, expecting the execution to + * return a specific exit code. + * + * @param sourceFS filesystem to use + * @param exitCode exit code to expect. + * @param path path to scan + * @param doPurge should markers be purged + * @param expectedMarkers number of markers expected + * @param limit limit of files to scan; -1 for 'unlimited' + * @param nonAuth only use nonauth path count for failure rules + * @return the result + */ + public static MarkerTool.ScanResult markerTool( + final int exitCode, + final FileSystem sourceFS, + final Path path, + final boolean doPurge, + final int expectedMarkers, + final int limit, + final boolean nonAuth) throws IOException { + + MarkerTool.ScanResult result = MarkerTool.execMarkerTool( + sourceFS, + path, + doPurge, + expectedMarkers, + limit, nonAuth); + Assertions.assertThat(result.getExitCode()) + .describedAs("Exit code of marker(%s, %s, %d) -> %s", + path, doPurge, expectedMarkers, result) + .isEqualTo(exitCode); + return result; + } + + /** + * Add a - prefix to a string + * @param s string to prefix + * @return a string for passing into the CLI + */ + protected static String m(String s) { + return "-" + s; + } + + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java index 4c5b695960a75..ae4f5a4ef0985 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java @@ -19,13 +19,9 @@ package org.apache.hadoop.fs.s3a.tools; import java.io.File; -import java.io.FileReader; import java.io.IOException; -import java.net.URI; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; -import java.util.stream.Collectors; import org.assertj.core.api.Assertions; import org.junit.Test; @@ -36,46 +32,29 @@ 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.s3guard.S3GuardTool; -import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.util.ExitUtil; -import org.apache.hadoop.util.StringUtils; - -import static org.apache.hadoop.fs.s3a.Constants.*; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides; + +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_AUTHORITATIVE; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE; +import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP; +import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.BucketInfo.BUCKET_INFO; +import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.runS3GuardCommand; +import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.runS3GuardCommandToFailure; import static org.apache.hadoop.fs.s3a.tools.MarkerTool.*; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_INTERRUPTED; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_ACCEPTABLE; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_FOUND; import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_USAGE; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test the marker tool and use it to compare the behavior * of keeping vs legacy S3A FS instances. */ -public class ITestMarkerTool extends AbstractS3ATestBase { +public class ITestMarkerTool extends AbstractMarkerToolTest { - private static final Logger LOG = + protected static final Logger LOG = LoggerFactory.getLogger(ITestMarkerTool.class); - /** the -verbose option. */ - private static final String V = m(VERBOSE); - - /** FS which keeps markers. */ - private S3AFileSystem keepingFS; - - /** FS which deletes markers. */ - private S3AFileSystem deletingFS; - - /** FS which mixes markers; only created in some tests. */ - private S3AFileSystem mixedFS; - /** * How many files to expect. */ @@ -101,74 +80,6 @@ public class ITestMarkerTool extends AbstractS3ATestBase { */ private int expectedMarkersWithBaseDir; - @Override - protected Configuration createConfiguration() { - Configuration conf = super.createConfiguration(); - String bucketName = getTestBucketName(conf); - removeBaseAndBucketOverrides(bucketName, conf, - S3A_BUCKET_PROBE, - DIRECTORY_MARKER_POLICY, - S3_METADATA_STORE_IMPL, - METADATASTORE_AUTHORITATIVE, - AUTHORITATIVE_PATH); - // base FS is legacy - conf.set(DIRECTORY_MARKER_POLICY, DIRECTORY_MARKER_POLICY_DELETE); - conf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL); - - // turn off bucket probes for a bit of speedup in the connectors we create. - conf.setInt(S3A_BUCKET_PROBE, 0); - return conf; - } - - @Override - public void setup() throws Exception { - super.setup(); - setKeepingFS(createFS(DIRECTORY_MARKER_POLICY_KEEP, null)); - setDeletingFS(createFS(DIRECTORY_MARKER_POLICY_DELETE, null)); - } - - @Override - public void teardown() throws Exception { - // do this ourselves to avoid audits teardown failing - // when surplus markers are found - deleteTestDirInTeardown(); - super.teardown(); - IOUtils.cleanupWithLogger(LOG, getKeepingFS(), - getMixedFS(), getDeletingFS()); - - } - - /** - * FS which deletes markers. - */ - public S3AFileSystem getDeletingFS() { - return deletingFS; - } - - public void setDeletingFS(final S3AFileSystem deletingFS) { - this.deletingFS = deletingFS; - } - - /** - * FS which keeps markers. - */ - private S3AFileSystem getKeepingFS() { - return keepingFS; - } - - private void setKeepingFS(S3AFileSystem keepingFS) { - this.keepingFS = keepingFS; - } - - /** only created on demand. */ - private S3AFileSystem getMixedFS() { - return mixedFS; - } - - private void setMixedFS(S3AFileSystem mixedFS) { - this.mixedFS = mixedFS; - } - @Test public void testCleanMarkersLegacyDir() throws Throwable { @@ -264,7 +175,7 @@ public void testAuthPathIsMixed() throws Throwable { CreatedPaths createdPaths = createPaths(mixedFSDir2, source); // markers are only under dir2 - markerTool(mixedFSDir2, topath(source, "dir1"), false, 0); + markerTool(mixedFSDir2, toPath(source, "dir1"), false, 0); markerTool(mixedFSDir2, source, false, expectedMarkersUnderDir2); // full scan of source will fail @@ -304,14 +215,17 @@ private static void assertMarkersDeleted(int expected, .isEqualTo(expected); } + /** + * Marker tool with no args. + */ @Test public void testRunNoArgs() throws Throwable { - runToFailure(EXIT_USAGE, NAME); + runToFailure(EXIT_USAGE, MARKERS); } @Test public void testRunWrongBucket() throws Throwable { - runToFailure(EXIT_NOT_FOUND, NAME, + runToFailure(EXIT_NOT_FOUND, MARKERS, AUDIT, "s3a://this-bucket-does-not-exist-hopefully"); } @@ -321,7 +235,7 @@ public void testRunWrongBucket() throws Throwable { */ @Test public void testRunUnknownPath() throws Throwable { - runToFailure(EXIT_NOT_FOUND, NAME, + runToFailure(EXIT_NOT_FOUND, MARKERS, AUDIT, methodPath()); } @@ -331,7 +245,7 @@ public void testRunUnknownPath() throws Throwable { */ @Test public void testRunTooManyActions() throws Throwable { - runToFailure(EXIT_USAGE, NAME, + runToFailure(EXIT_USAGE, MARKERS, AUDIT, CLEAN, methodPath()); } @@ -342,7 +256,7 @@ public void testRunAuditWithExpectedMarkers() throws Throwable { // a run under the keeping FS will create paths CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); final File audit = tempAuditFile(); - run(NAME, V, + run(MARKERS, V, AUDIT, m(OPT_LIMIT), 0, m(OPT_OUT), audit, @@ -357,7 +271,7 @@ public void testRunAuditWithExcessMarkers() throws Throwable { // a run under the keeping FS will create paths CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); final File audit = tempAuditFile(); - runToFailure(EXIT_NOT_ACCEPTABLE, NAME, V, + runToFailure(EXIT_NOT_ACCEPTABLE, MARKERS, V, AUDIT, m(OPT_OUT), audit, createdPaths.base); @@ -369,93 +283,75 @@ public void testRunLimitedAudit() throws Throwable { describe("Audit with a limited number of files (2)"); CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); runToFailure(EXIT_INTERRUPTED, - NAME, V, + MARKERS, V, m(OPT_LIMIT), 2, CLEAN, createdPaths.base); - run(NAME, V, + run(MARKERS, V, AUDIT, createdPaths.base); } /** - * Get a filename for a temp file. - * The generated file is deleted. - * - * @return a file path for a output file + * Run an audit against the landsat bucket. + *

+ * This tests paging/scale against a larger bucket without + * worrying about setup costs. */ - private File tempAuditFile() throws IOException { - final File audit = File.createTempFile("audit", ".txt"); - audit.delete(); - return audit; - } + @Test + public void testRunLimitedLandsatAudit() throws Throwable { + describe("Audit a few thousand landsat objects"); + final File audit = tempAuditFile(); - /** - * Read the audit output and verify it has the expected number of lines. - * @param auditFile audit file to read - * @param expected expected line count - */ - private void expectMarkersInOutput(final File auditFile, - final int expected) - throws IOException { - final List lines = readOutput(auditFile); - Assertions.assertThat(lines) - .describedAs("Content of %s", auditFile) - .hasSize(expected); + run(MARKERS, + AUDIT, + m(OPT_LIMIT), 3000, + m(OPT_OUT), audit, + LANDSAT_BUCKET); + readOutput(audit); } - /** - * Read the output file in. Logs the contents at info. - * @param outputFile audit output file. - * @return the lines - */ - public List readOutput(final File outputFile) - throws IOException { - try (FileReader reader = new FileReader(outputFile)) { - final List lines = - org.apache.commons.io.IOUtils.readLines(reader); + @Test + public void testBucketInfoKeepingOnDeleting() throws Throwable { + describe("Run bucket info with the keeping config on the deleting fs"); + runS3GuardCommandToFailure(uncachedFSConfig(getDeletingFS()), + EXIT_NOT_ACCEPTABLE, + BUCKET_INFO, + m(MARKERS), DIRECTORY_MARKER_POLICY_KEEP, + methodPath()); + } - LOG.info("contents of output file {}\n{}", outputFile, - StringUtils.join("\n", lines)); - return lines; - } + @Test + public void testBucketInfoKeepingOnKeeping() throws Throwable { + describe("Run bucket info with the keeping config on the keeping fs"); + runS3GuardCommand(uncachedFSConfig(getKeepingFS()), + BUCKET_INFO, + m(MARKERS), DIRECTORY_MARKER_POLICY_KEEP, + methodPath()); } - private static Path topath(Path base, final String name) { - return name.isEmpty() ? base : new Path(base, name); + @Test + public void testBucketInfoDeletingOnDeleting() throws Throwable { + describe("Run bucket info with the deleting config on the deleting fs"); + runS3GuardCommand(uncachedFSConfig(getDeletingFS()), + BUCKET_INFO, + m(MARKERS), DIRECTORY_MARKER_POLICY_DELETE, + methodPath()); } - /** - * Create a new FS with given marker policy and path. - * This filesystem MUST be closed in test teardown. - * @param markerPolicy markers - * @param authPath authoritative path. If null: no path. - * @return a new FS. - */ - private S3AFileSystem createFS(String markerPolicy, - String authPath) throws Exception { - S3AFileSystem testFS = getFileSystem(); - Configuration conf = new Configuration(testFS.getConf()); - URI testFSUri = testFS.getUri(); - String bucketName = getTestBucketName(conf); - removeBucketOverrides(bucketName, conf, - DIRECTORY_MARKER_POLICY, - S3_METADATA_STORE_IMPL, - BULK_DELETE_PAGE_SIZE, - AUTHORITATIVE_PATH); - if (authPath != null) { - conf.set(AUTHORITATIVE_PATH, authPath); - } - // Use a very small page size to force the paging - // code to be tested. - conf.setInt(BULK_DELETE_PAGE_SIZE, 2); - conf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL); - conf.set(DIRECTORY_MARKER_POLICY, markerPolicy); - S3AFileSystem fs2 = new S3AFileSystem(); - fs2.initialize(testFSUri, conf); - LOG.info("created new filesystem with policy {} and auth path {}", - markerPolicy, authPath); - return fs2; + @Test + public void testBucketInfoAuthOnAuth() throws Throwable { + describe("Run bucket info with the auth FS"); + Path base = methodPath(); + + S3AFileSystem authFS = createFS(DIRECTORY_MARKER_POLICY_AUTHORITATIVE, + base.toUri().toString()); + // line up for close in teardown + setMixedFS(authFS); + runS3GuardCommand(uncachedFSConfig(authFS), + BUCKET_INFO, + m(MARKERS), DIRECTORY_MARKER_POLICY_AUTHORITATIVE, + methodPath()); } /** @@ -509,7 +405,7 @@ private int dirs(String... names) throws IOException { * @return the path of the new entry. */ private Path mkdir(String name) throws IOException { - Path dir = topath(base, name); + Path dir = toPath(base, name); fs.mkdirs(dir); dirs.add(dir); dirsUnderBase.add(name); @@ -536,7 +432,7 @@ private int emptydirs(String... names) throws IOException { * @throws IOException failure */ private Path emptydir(String name) throws IOException { - Path dir = topath(base, name); + Path dir = toPath(base, name); fs.mkdirs(dir); emptyDirs.add(dir); emptyDirsUnderBase.add(name); @@ -564,7 +460,7 @@ private int files(String... names) throws IOException { */ private Path mkfile(String name) throws IOException { - Path file = topath(base, name); + Path file = toPath(base, name); ContractTestUtils.touch(fs, file); files.add(file); filesUnderBase.add(name); @@ -613,59 +509,6 @@ private CreatedPaths createPaths(FileSystem fs, Path base) return r; } - /** - * Execute the marker tool, expecting the execution to succeed. - * @param sourceFS filesystem to use - * @param path path to scan - * @param doPurge should markers be purged - * @param expectedMarkerCount number of markers expected - * @return the result - */ - private MarkerTool.ScanResult markerTool( - final FileSystem sourceFS, - final Path path, - final boolean doPurge, - final int expectedMarkerCount) - throws IOException { - return markerTool(0, sourceFS, path, doPurge, expectedMarkerCount, - UNLIMITED_LISTING, false); - } - - /** - * Execute the marker tool, expecting the execution to - * return a specific exit code. - * - * @param sourceFS filesystem to use - * @param exitCode exit code to expect. - * @param path path to scan - * @param doPurge should markers be purged - * @param expectedMarkers number of markers expected - * @param limit limit of files to scan; -1 for 'unlimited' - * @param nonAuth only use nonauth path count for failure rules - * @return the result - */ - public static MarkerTool.ScanResult markerTool( - final int exitCode, - final FileSystem sourceFS, - final Path path, - final boolean doPurge, - final int expectedMarkers, - final int limit, - final boolean nonAuth) throws IOException { - - MarkerTool.ScanResult result = MarkerTool.execMarkerTool( - sourceFS, - path, - doPurge, - expectedMarkers, - limit, nonAuth); - Assertions.assertThat(result.getExitCode()) - .describedAs("Exit code of marker(%s, %s, %d) -> %s", - path, doPurge, expectedMarkers, result) - .isEqualTo(exitCode); - return result; - } - /** * Verify that all the paths renamed from the source exist * under the destination, including all empty directories. @@ -676,61 +519,16 @@ void verifyRenamed(final Path dest, final CreatedPaths createdPaths) throws IOException { // all leaf directories exist for (String p : createdPaths.emptyDirsUnderBase) { - assertIsDirectory(topath(dest, p)); + assertIsDirectory(toPath(dest, p)); } // non-empty dirs for (String p : createdPaths.dirsUnderBase) { - assertIsDirectory(topath(dest, p)); + assertIsDirectory(toPath(dest, p)); } // all files exist for (String p : createdPaths.filesUnderBase) { - assertIsFile(topath(dest, p)); + assertIsFile(toPath(dest, p)); } } - /** - * Run a S3GuardTool command from a varags list and the - * configuration returned by {@code getConfiguration()}. - * @param args argument list - * @return the return code - * @throws Exception any exception - */ - protected int run(Object... args) throws Exception { - Configuration conf = new Configuration(getConfiguration()); - final String[] argList = Arrays.stream(args) - .map(Object::toString) - .collect(Collectors.toList()).toArray(new String[0]); - disableFilesystemCaching(conf); - return S3GuardTool.run(conf, argList); - } - /** - * Run a S3GuardTool command from a varags list, catch any raised - * ExitException and verify the status code matches that expected. - * @param status expected status code of the exception - * @param args argument list - * @throws Exception any exception - */ - protected void runToFailure(int status, Object... args) - throws Exception { - ExitUtil.ExitException ex = - intercept(ExitUtil.ExitException.class, - () -> { - int ec = run(args); - if (ec != 0) { - throw new ExitUtil.ExitException(ec, "exit code " + ec); - } - }); - if (ex.status != status) { - throw ex; - } - } - - /** - * Add a - prefix to a string - * @param s string to prefix - * @return a string for passing into the CLI - */ - private static String m(String s) { - return "-" + s; - } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerToolRootOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerToolRootOperations.java new file mode 100644 index 0000000000000..02fec81513fca --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerToolRootOperations.java @@ -0,0 +1,70 @@ +/* + * 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.tools; + +import java.io.File; + +import org.junit.FixMethodOrder; +import org.junit.Test; +import org.junit.runners.MethodSorters; + +import org.apache.hadoop.fs.Path; + +import static org.apache.hadoop.fs.s3a.tools.MarkerTool.AUDIT; +import static org.apache.hadoop.fs.s3a.tools.MarkerTool.CLEAN; +import static org.apache.hadoop.fs.s3a.tools.MarkerTool.MARKERS; +import static org.apache.hadoop.fs.s3a.tools.MarkerTool.OPT_OUT; + +/** + * Marker tool tests against the root FS; run in the sequential phase. + */ +@FixMethodOrder(MethodSorters.NAME_ASCENDING) +public class ITestMarkerToolRootOperations extends AbstractMarkerToolTest { + + private Path rootPath; + + @Override + public void setup() throws Exception { + super.setup(); + rootPath = getFileSystem().makeQualified(new Path("/")); + } + + @Test + public void test_100_audit_root_noauth() throws Throwable { + describe("Run a verbose audit"); + final File audit = tempAuditFile(); + run(MARKERS, V, + AUDIT, + m(OPT_OUT), audit, + rootPath); + readOutput(audit); + } + + @Test + public void test_200_clean_root() throws Throwable { + describe("Clean the root path"); + final File audit = tempAuditFile(); + run(MARKERS, V, + CLEAN, + m(OPT_OUT), audit, + rootPath); + readOutput(audit); + } + +} From bbeed7ecc54bcf54ce59e233d57d16744318a83f Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 14 Aug 2020 12:06:45 +0100 Subject: [PATCH 19/20] HADOOP-13230. marker policy PathCapabilities enhancements * specific bucket-level "marker.policy.{delete, keep, authoritative}" probes, which * dynamic probes renamed marker.action.{keep, delete} Capability logic all moved into DirectoryMarkerPolicyImpl ...which means it is now testable in unit tests. Tests added. Checkstyles. Change-Id: I27db716097a3bc1e1fe2639d2e90c1e855658675 --- .../org/apache/hadoop/fs/s3a/Constants.java | 46 ++++++++++--- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 29 ++++++--- .../hadoop/fs/s3a/impl/DirMarkerTracker.java | 3 +- .../hadoop/fs/s3a/impl/DirectoryPolicy.java | 13 ++++ .../fs/s3a/impl/DirectoryPolicyImpl.java | 43 +++++++++++- .../hadoop/fs/s3a/tools/MarkerTool.java | 17 +++-- .../tools/hadoop-aws/directory_markers.md | 34 ++++++---- .../hadoop/fs/s3a/AbstractS3ATestBase.java | 4 +- .../fs/s3a/ITestS3AFileOperationCost.java | 9 +++ .../s3a/impl/TestDirectoryMarkerPolicy.java | 65 ++++++++++++++++--- .../ITestDirectoryMarkerListing.java | 19 +++--- .../s3a/performance/ITestS3ADeleteCost.java | 4 +- .../s3guard/AbstractS3GuardToolTestBase.java | 4 +- .../fs/s3a/tools/AbstractMarkerToolTest.java | 4 +- .../hadoop/fs/s3a/tools/ITestMarkerTool.java | 5 +- 15 files changed, 227 insertions(+), 72 deletions(-) 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 ba3cda91d74d3..a1c1d969a8258 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 @@ -992,27 +992,53 @@ private Constants() { /** - * Probe to indicate that a path keeps directory markers. - * returned in {@code PathCapabilities} + * {@code PathCapabilities} probe to verify that an S3A Filesystem + * has the changes needed to safely work with buckets where + * directoy markers have not been deleted. * Value: {@value}. */ public static final String STORE_CAPABILITY_DIRECTORY_MARKER_AWARE = "fs.s3a.capability.directory.marker.aware"; /** - * Probe to indicate that a path keeps directory markers. - * returned in {@code PathCapabilities} + * {@code PathCapabilities} probe to indicate that the filesystem + * keeps directory markers. * Value: {@value}. */ - public static final String STORE_CAPABILITY_DIRECTORY_MARKER_KEEP - = "fs.s3a.capability.directory.marker.keep"; + public static final String STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_KEEP + = "fs.s3a.capability.directory.marker.policy.keep"; /** - * Probe to indicate that a path keeps directory markers. - * returned in {@code PathCapabilities} + * {@code PathCapabilities} probe to indicate that the filesystem + * deletes directory markers. * Value: {@value}. */ - public static final String STORE_CAPABILITY_DIRECTORY_MARKER_DELETE - = "fs.s3a.capability.directory.marker.delete"; + public static final String STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_DELETE + = "fs.s3a.capability.directory.marker.policy.delete"; + + /** + * {@code PathCapabilities} probe to indicate that the filesystem + * keeps directory markers in authoritative paths only. + * Value: {@value}. + */ + public static final String + STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_AUTHORITATIVE = + "fs.s3a.capability.directory.marker.policy.authoritative"; + + /** + * {@code PathCapabilities} probe to indicate that a path + * keeps directory markers. + * Value: {@value}. + */ + public static final String STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP + = "fs.s3a.capability.directory.marker.action.keep"; + + /** + * {@code PathCapabilities} probe to indicate that a path + * deletes directory markers. + * Value: {@value}. + */ + public static final String STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE + = "fs.s3a.capability.directory.marker.action.delete"; } 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 c03d5219293d8..ac9904a867e21 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 @@ -3118,6 +3118,8 @@ S3AFileStatus s3GetFileStatus(final Path path, @Nullable final Set tombstones, final boolean needEmptyDirectoryFlag) throws IOException { LOG.debug("S3GetFileStatus {}", path); + // either you aren't looking for the directory flag, or you are, + // and if you are, the probe list must contain list. Preconditions.checkArgument(!needEmptyDirectoryFlag || probes.contains(StatusProbeEnum.List), "s3GetFileStatus(%s) wants to know if a directory is empty but" @@ -4614,7 +4616,8 @@ public S3AInstrumentation.CommitterStatistics newCommitterStatistics() { public boolean hasPathCapability(final Path path, final String capability) throws IOException { final Path p = makeQualified(path); - switch (validatePathCapabilityArgs(p, capability)) { + String cap = validatePathCapabilityArgs(p, capability); + switch (cap) { case CommitConstants.STORE_CAPABILITY_MAGIC_COMMITTER: case CommitConstants.STORE_CAPABILITY_MAGIC_COMMITTER_OLD: @@ -4633,18 +4636,24 @@ public boolean hasPathCapability(final Path path, final String capability) case CommonPathCapabilities.FS_MULTIPART_UPLOADER: return true; - /* - * Marker policy is dynamically determined for the given path. - */ + // this client is safe to use with buckets + // containing directory markers anywhere in + // the hierarchy case STORE_CAPABILITY_DIRECTORY_MARKER_AWARE: return true; - case STORE_CAPABILITY_DIRECTORY_MARKER_KEEP: - return directoryPolicy.keepDirectoryMarkers(p); - case STORE_CAPABILITY_DIRECTORY_MARKER_DELETE: - return !directoryPolicy.keepDirectoryMarkers(p); + + /* + * Marker policy capabilities are handed off. + */ + case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_KEEP: + case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_DELETE: + case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_AUTHORITATIVE: + case STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP: + case STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE: + return getDirectoryMarkerPolicy().hasPathCapability(path, cap); default: - return super.hasPathCapability(p, capability); + return super.hasPathCapability(p, cap); } } @@ -4659,7 +4668,7 @@ public boolean hasPathCapability(final Path path, final String capability) @Override public boolean hasCapability(String capability) { try { - return hasPathCapability(workingDir, capability); + return hasPathCapability(new Path("/"), capability); } catch (IOException ex) { // should never happen, so log and downgrade. LOG.debug("Ignoring exception on hasCapability({}})", capability, ex); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java index 90a976a633c18..ca04fed65a539 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java @@ -276,7 +276,8 @@ public String toString() { */ public List removeAllowedMarkers(DirectoryPolicy policy) { List removed = new ArrayList<>(); - Iterator> entries = surplusMarkers.entrySet().iterator(); + Iterator> entries = + surplusMarkers.entrySet().iterator(); while (entries.hasNext()) { Map.Entry entry = entries.next(); Path path = entry.getKey(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java index da0ee8b55f637..36dd2e4fd2496 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java @@ -30,6 +30,8 @@ public interface DirectoryPolicy { + + /** * Should a directory marker be retained? * @param path path a file/directory is being created with. @@ -49,6 +51,17 @@ public interface DirectoryPolicy { */ String describe(); + /** + * Does a specific path have the relevant option. + * This is to be forwarded from the S3AFileSystem.hasPathCapability + * But only for those capabilities related to markers* + * @param path path + * @param capability capability + * @return true if the capability is supported, false if not + * @throws IllegalArgumentException if the capability is unknown. + */ + boolean hasPathCapability(Path path, String capability); + /** * Supported retention policies. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java index 64901718c5e96..a1aa2580b655a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java @@ -35,6 +35,12 @@ import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_AUTHORITATIVE; import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE; import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP; +import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_DIRECTORY_MARKER_AWARE; +import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE; +import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP; +import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_AUTHORITATIVE; +import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_DELETE; +import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_KEEP; /** * Implementation of directory policy. @@ -124,6 +130,42 @@ public String toString() { return sb.toString(); } + /** + * Return path policy for store and paths. + * @param path path + * @param capability capability + * @return true if a capability is active + */ + @Override + public boolean hasPathCapability(final Path path, final String capability) { + + switch (capability) { + /* + * Marker policy is dynamically determined for the given path. + */ + case STORE_CAPABILITY_DIRECTORY_MARKER_AWARE: + return true; + + case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_KEEP: + return markerPolicy == MarkerPolicy.Keep; + + case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_DELETE: + return markerPolicy == MarkerPolicy.Delete; + + case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_AUTHORITATIVE: + return markerPolicy == MarkerPolicy.Authoritative; + + case STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP: + return keepDirectoryMarkers(path); + + case STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE: + return !keepDirectoryMarkers(path); + + default: + throw new IllegalArgumentException("Unknown capability " + capability); + } + } + /** * Create/Get the policy for this configuration. * @param conf config @@ -159,7 +201,6 @@ public static DirectoryPolicy getDirectoryPolicy( return policy; } - /** * Enumerate all available policies. * @return set of the policies. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index 3cffe5ed83667..6855c52edbbbb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -19,11 +19,13 @@ package org.apache.hadoop.fs.s3a.tools; import java.io.FileNotFoundException; -import java.io.FileWriter; +import java.io.FileOutputStream; import java.io.IOException; +import java.io.OutputStreamWriter; import java.io.PrintStream; import java.io.Writer; import java.net.URI; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -284,7 +286,10 @@ public int run(final String[] args, final PrintStream stream) String saveFile = command.getOptValue(OPT_OUT); if (saveFile != null && !saveFile.isEmpty()) { println(out, "Saving result to %s", saveFile); - try (Writer writer = new FileWriter(saveFile)) { + try (Writer writer = + new OutputStreamWriter( + new FileOutputStream(saveFile), + StandardCharsets.UTF_8)) { final List surplus = result.getTracker() .getSurplusMarkers() .keySet() @@ -339,7 +344,7 @@ ScanResult execute( println(out, "Authoritative path list is \"%s\"", authPath); } // qualify the path - Path target = path.makeQualified(fs.getUri(),new Path("/")); + Path target = path.makeQualified(fs.getUri(), new Path("/")); // initial safety check: does the path exist? try { getFilesystem().getFileStatus(target); @@ -512,13 +517,12 @@ private ScanResult scan( } } - // now one little check for whether a limit was reached. if (!completed) { println(out, "Listing limit reached before completing the scan"); result.exitCode = EXIT_INTERRUPTED; } - return result; + return result; } /** @@ -571,7 +575,8 @@ private boolean scanDirectoryTree( println(out, "Scanned %,d objects", count); } if (limit > 0 && count >= limit) { - println(out, "Limit of scan reached - %,d object%s", limit, suffix(limit)); + println(out, "Limit of scan reached - %,d object%s", + limit, suffix(limit)); return false; } } diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md index 4946f381b545b..14ee6fd1e6680 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md @@ -72,7 +72,7 @@ found. ### If an application has updated a directory tree incompatibly-- what can be done? There's a tool on the hadoop command line, [marker tool](#marker-tool) which can audit -a bucket/path for markers, and clean up any which were found. +a bucket/path for markers, and clean up any markers which were found. It can be used to make a bucket compatible with older applications. Now that this is all clear, let's explain the problem. @@ -147,7 +147,7 @@ has been interpreted as an empty directory.* It is that little detail which is the cause of the incompatibility issues. -## Scale issues related to directory markers +## The Problem with Directory Markers Creating, deleting and the listing directory markers adds overhead and can slow down applications. @@ -548,15 +548,23 @@ which all FileSystem classes have supported since Hadoop 3.3. | Probe | Meaning | |-------------------------|-------------------------| | `fs.s3a.capability.directory.marker.aware` | Does the filesystem support surplus directory markers? | -| `fs.s3a.capability.directory.marker.keep` | Does the path retain directory markers? | -| `fs.s3a.capability.directory.marker.delete` | Does the path delete directory markers? | +| `fs.s3a.capability.directory.marker.policy.keep` | Is the bucket policy "keep"? | +| `fs.s3a.capability.directory.marker.policy.delete` | Is the bucket policy "delete" | +| `fs.s3a.capability.directory.marker.policy.authoritative` | Is the bucket policy "authoritative" | +| `fs.s3a.capability.directory.marker.action.keep` | Does the path retain directory markers? | +| `fs.s3a.capability.directory.marker.action.delete` | Does the path delete directory markers? | The probe `fs.s3a.capability.directory.marker.aware` allows for a filesystem to be probed to determine if its file listing policy is "aware" of directory marker retention --that is: it can safely work with S3 buckets where markers have not been deleted. +-that is: can this s3a client safely work with S3 buckets where markers have not been deleted. -The other two probes dynamically query the marker retention behavior of a specific path. +The `fs.s3a.capability.directory.marker.policy.` probes return the active policy for the bucket. + +The two `fs.s3a.capability.directory.marker.action.` probes dynamically query the marker +retention behavior of a specific path. +That is: if a file was created at that location, would ancestor directory markers +be kept or deleted? The `S3AFileSystem` class also implements the `org.apache.hadoop.fs.StreamCapabilities` interface, which can be used to probe for marker awareness via the `fs.s3a.capability.directory.marker.aware` capability. @@ -598,28 +606,28 @@ Take a bucket with a retention policy of "authoritative" -only paths under `/tab ``` ``` -With this policy the path capability `fs.s3a.capability.directory.marker.keep` will hold under +With this policy the path capability `fs.s3a.capability.directory.marker.action.keep` will hold under the path `s3a://london/tables` ``` -bin/hadoop jar cloudstore-1.0.jar pathcapability fs.s3a.capability.directory.marker.keep s3a://london/tables -Probing s3a://london/tables for capability fs.s3a.capability.directory.marker.keep +bin/hadoop jar cloudstore-1.0.jar pathcapability fs.s3a.capability.directory.marker.action.keep s3a://london/tables +Probing s3a://london/tables for capability fs.s3a.capability.directory.marker.action.keep 2020-08-11 22:03:31,658 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143)) - Directory markers will be kept on authoritative paths Using filesystem s3a://london -Path s3a://london/tables has capability fs.s3a.capability.directory.marker.keep +Path s3a://london/tables has capability fs.s3a.capability.directory.marker.action.keep ``` However it will not hold for other paths, so indicating that older Hadoop versions will be safe to work with data written there by this S3A client. ``` -bin/hadoop jar cloudstore-1.0.jar pathcapability fs.s3a.capability.directory.marker.keep s3a://london/tempdir -Probing s3a://london/tempdir for capability fs.s3a.capability.directory.marker.keep +bin/hadoop jar cloudstore-1.0.jar pathcapability fs.s3a.capability.directory.marker.action.keep s3a://london/tempdir +Probing s3a://london/tempdir for capability fs.s3a.capability.directory.marker.action.keep 2020-08-11 22:06:56,300 [main] INFO impl.DirectoryPolicyImpl (DirectoryPolicyImpl.java:getDirectoryPolicy(143)) - Directory markers will be kept on authoritative paths Using filesystem s3a://london -Path s3a://london/tempdir lacks capability fs.s3a.capability.directory.marker.keep +Path s3a://london/tempdir lacks capability fs.s3a.capability.directory.marker.action.keep 2020-08-11 22:06:56,308 [main] INFO util.ExitUtil (ExitUtil.java:terminate(210)) - Exiting with status -1: ``` diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java index 5754fad20207f..a2ee9ea5f7b29 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java @@ -90,8 +90,8 @@ public void maybeAuditTestPath() { && !fs.getDirectoryMarkerPolicy() .keepDirectoryMarkers(methodPath) && fs.isDirectory(methodPath)) { - MarkerTool.ScanResult result = MarkerTool.execMarkerTool(fs, - methodPath, true, 0, UNLIMITED_LISTING, false); + MarkerTool.ScanResult result = MarkerTool.execMarkerTool(fs, + methodPath, true, 0, UNLIMITED_LISTING, false); assertEquals("Audit of " + methodPath + " failed: " + result, 0, result.getExitCode()); } 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 70413e3c3d58b..46e6f5fcea74f 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 @@ -45,6 +45,7 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.performance.OperationCost.*; import static org.apache.hadoop.test.GenericTestUtils.getTestDir; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Use metrics to assert about the cost of file API calls. @@ -314,6 +315,14 @@ public void testDirProbes() throws Throwable { assertEmptyDirStatus(status, Tristate.TRUE); } + @Test + public void testNeedEmptyDirectoryProbeRequiresList() throws Throwable { + S3AFileSystem fs = getFileSystem(); + + intercept(IllegalArgumentException.class, "", () -> + fs.s3GetFileStatus(new Path("/something"), "/something", + StatusProbeEnum.HEAD_ONLY, null, true)); + } @Test public void testCreateCost() throws Throwable { describe("Test file creation cost -raw only"); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java index fbb5cbc4f881e..194cd645c0714 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java @@ -30,6 +30,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.test.AbstractHadoopTestBase; +import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE; +import static org.apache.hadoop.fs.s3a.Constants.STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP; + /** * Unit tests for directory marker policies. */ @@ -53,11 +56,14 @@ public static Collection data() { DirectoryPolicy.MarkerPolicy.Authoritative, AUTH_PATH_ONLY, false, true - }}); + } + }); } - private final DirectoryPolicyImpl retention; + private final DirectoryPolicy directoryPolicy; + private final boolean expectNonAuthDelete; + private final boolean expectAuthDelete; public TestDirectoryMarkerPolicy( @@ -65,7 +71,7 @@ public TestDirectoryMarkerPolicy( final Predicate authoritativeness, final boolean expectNonAuthDelete, final boolean expectAuthDelete) { - this.retention = retention(markerPolicy, authoritativeness); + this.directoryPolicy = newPolicy(markerPolicy, authoritativeness); this.expectNonAuthDelete = expectNonAuthDelete; this.expectAuthDelete = expectAuthDelete; } @@ -77,7 +83,7 @@ public TestDirectoryMarkerPolicy( * a path is authoritative. * @return the retention policy. */ - private DirectoryPolicyImpl retention( + private DirectoryPolicy newPolicy( DirectoryPolicy.MarkerPolicy markerPolicy, Predicate authoritativeness) { return new DirectoryPolicyImpl(markerPolicy, authoritativeness); @@ -91,28 +97,67 @@ private DirectoryPolicyImpl retention( }; private final Path nonAuthPath = new Path("s3a://bucket/nonauth/data"); + private final Path authPath = new Path("s3a://bucket/auth/data1"); + private final Path deepAuth = new Path("s3a://bucket/auth/d1/d2/data2"); - private void assertRetention(Path path, boolean retain) { - Assertions.assertThat(retention.keepDirectoryMarkers(path)) - .describedAs("Retention of path %s by %s", path, retention) + /** + * Assert that a path has a retention outcome. + * @param path path + * @param retain should the marker be retained + */ + private void assertMarkerRetention(Path path, boolean retain) { + Assertions.assertThat(directoryPolicy.keepDirectoryMarkers(path)) + .describedAs("Retention of path %s by %s", path, directoryPolicy) .isEqualTo(retain); } + /** + * Assert that a path has a capability. + */ + private void assertPathCapability(Path path, + String capability, + boolean outcome) { + Assertions.assertThat(directoryPolicy) + .describedAs("%s support for capability %s by path %s" + + " expected as %s", + directoryPolicy, capability, path, outcome) + .matches(p -> p.hasPathCapability(path, capability) == outcome, + "pathCapability"); + } + @Test public void testNonAuthPath() throws Throwable { - assertRetention(nonAuthPath, expectNonAuthDelete); + assertMarkerRetention(nonAuthPath, expectNonAuthDelete); + assertPathCapability(nonAuthPath, + STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE, + !expectNonAuthDelete); + assertPathCapability(nonAuthPath, + STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP, + expectNonAuthDelete); } @Test public void testAuthPath() throws Throwable { - assertRetention(authPath, expectAuthDelete); + assertMarkerRetention(authPath, expectAuthDelete); + assertPathCapability(authPath, + STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE, + !expectAuthDelete); + assertPathCapability(authPath, + STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP, + expectAuthDelete); } @Test public void testDeepAuthPath() throws Throwable { - assertRetention(deepAuth, expectAuthDelete); + assertMarkerRetention(deepAuth, expectAuthDelete); + assertPathCapability(deepAuth, + STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE, + !expectAuthDelete); + assertPathCapability(deepAuth, + STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP, + expectAuthDelete); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java index 417c25df9bc25..ed56802ddfec1 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java @@ -509,20 +509,19 @@ public void testRenameBase() throws Throwable { @Test public void testRenameUnderMarkerDir() throws Throwable { describe("directory rename under an existing marker"); - S3AFileSystem fs = getFileSystem(); - String name = "sourceFile"; + String file = "sourceFile"; Path srcDir = new Path(basePath, "srcdir"); mkdirs(srcDir); - Path src = new Path(srcDir, name); + Path src = new Path(srcDir, file); String srcKey = toKey(src); - put(srcKey, name); + put(srcKey, file); head(srcKey); // set the destination to be the marker directory. Path dest = markerDir; // rename the source file under the dest dir. assertRenamed(src, dest); - assertIsFile(new Path(dest, name)); + assertIsFile(new Path(dest, file)); assertIsDirectory(srcDir); if (isDeletingMarkers) { head404(markerKeySlash); @@ -547,12 +546,12 @@ public void testRenameUnderMarkerDir() throws Throwable { public void testRenameUnderMarkerWithPath() throws Throwable { describe("directory rename under an existing marker"); S3AFileSystem fs = getFileSystem(); - String name = "sourceFile"; + String file = "sourceFile"; Path srcDir = new Path(basePath, "srcdir"); mkdirs(srcDir); - Path src = new Path(srcDir, name); + Path src = new Path(srcDir, file); String srcKey = toKey(src); - put(srcKey, name); + put(srcKey, file); head(srcKey); // set the destination to be the final file @@ -576,8 +575,8 @@ public void testRenameUnderMarkerWithPath() throws Throwable { public void testRenameEmptyDirOverMarker() throws Throwable { describe("rename an empty directory over the marker"); S3AFileSystem fs = getFileSystem(); - String name = "sourceDir"; - Path src = new Path(basePath, name); + String dir = "sourceDir"; + Path src = new Path(basePath, dir); fs.mkdirs(src); assertIsDirectory(src); String srcKey = toKey(src) + "/"; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java index 03fdd89261cfa..d3d976e928940 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java @@ -145,8 +145,8 @@ public void testDeleteFileInDir() throws Throwable { boolean rawAndDeleting = isRaw() && isDeleting(); verifyMetrics(() -> { fs.delete(file1, false); - return "after fs.delete(file1simpleFile) " + getMetricSummary(); - }, + return "after fs.delete(file1simpleFile) " + getMetricSummary(); + }, // delete file. For keeping: that's it probe(rawAndKeeping, OBJECT_METADATA_REQUESTS, FILESTATUS_FILE_PROBE_H), 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 86b31d8860fa1..64057d02f8220 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 @@ -168,8 +168,8 @@ protected void runToFailure(int status, Object... args) throws Exception { final Configuration conf = getConfiguration(); ExitUtil.ExitException ex = - intercept(ExitUtil.ExitException.class, - () -> runS3GuardCommand(conf, args)); + intercept(ExitUtil.ExitException.class, () -> + runS3GuardCommand(conf, args)); if (ex.status != status) { throw ex; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/AbstractMarkerToolTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/AbstractMarkerToolTest.java index c3836e5db9186..00e62d9491070 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/AbstractMarkerToolTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/AbstractMarkerToolTest.java @@ -48,7 +48,7 @@ /** * Class for marker tool tests -sets up keeping/deleting filesystems, - * has methods to invoke + * has methods to invoke. */ public class AbstractMarkerToolTest extends AbstractS3ATestBase { @@ -322,7 +322,7 @@ public static MarkerTool.ScanResult markerTool( } /** - * Add a - prefix to a string + * Add a "-" prefix to a string. * @param s string to prefix * @return a string for passing into the CLI */ diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java index ae4f5a4ef0985..4a81b1aba919b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java @@ -28,7 +28,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; @@ -71,12 +70,12 @@ public class ITestMarkerTool extends AbstractMarkerToolTest { private int expectedMarkersUnderDir2; /** - * How many markers to expect across both dirs + * How many markers to expect across both dirs? */ private int expectedMarkers; /** - * How many markers to expect including the base directory + * How many markers to expect including the base directory? */ private int expectedMarkersWithBaseDir; From e24d42b5a8758270a9f017eaf919affeb31e641c Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Sat, 15 Aug 2020 12:44:30 +0100 Subject: [PATCH 20/20] HADOOP-13230. Final doc review Change-Id: If76d9f3c6918d5c3cfd9bb28d4a97e35654139ea --- .../tools/hadoop-aws/directory_markers.md | 47 ++++++++++--------- .../site/markdown/tools/hadoop-aws/index.md | 20 ++++++-- 2 files changed, 39 insertions(+), 28 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md index 14ee6fd1e6680..3773629b8e439 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md @@ -2,9 +2,9 @@ Licensed 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. @@ -93,8 +93,8 @@ In normal Unix-style filesystems, the "filesystem" is really a "directory and file tree" in which files are always stored in "directories" -* A directory may contain 0 or more files. -* A directory may contain 0 or more directories "subdirectories" +* A directory may contain zero or more files. +* A directory may contain zero or more directories "subdirectories" * At the base of a filesystem is the "root directory" * All files MUST be in a directory "the parent directory" * All directories other than the root directory must be in another directory. @@ -128,7 +128,7 @@ Lots of code contains a big assumption here: after you create a directory it exists. They also assume that after files in a directory are deleted, the directory still exists. -Given filesystem mimics directories just by aggregating objects which share a +Given the S3A connector mimics directories just by aggregating objects which share a prefix, how can you have empty directories? The original Hadoop `s3n://` connector created a Directory Marker -any path ending @@ -192,12 +192,13 @@ The tombstone markers have follow-on consequences -it makes listings against S3 versioned buckets slower. This can have adverse effects on those large directories, again. -## How to avoid marker-related problems. +## Strategies to avoid marker-related problems. ### Presto: every path is a directory -In the Presto S3 connectors: `mkdirs()` is a no-op. Instead, whenever it lists -any path which isn't an object or a prefix of one more more objects, it returns an +In the Presto [S3 connector](https://prestodb.io/docs/current/connector/hive.html#amazon-s3-configuration), +`mkdirs()` is a no-op. +Whenever it lists any path which isn't an object or a prefix of one more more objects, it returns an empty listing. That is:; by default, every path is an empty directory. Provided no code probes for a directory existing and fails if it is there, this @@ -207,7 +208,7 @@ because they know how their file uses data in S3. ### Hadoop 3.3.1+: marker deletion is now optional -From Hadoop 3.3.1 onwards, th S3A client can be configured to skip deleting +From Hadoop 3.3.1 onwards, the S3A client can be configured to skip deleting directory markers when creating files under paths. This removes all scalability problems caused by deleting these markers -however, it is achieved at the expense of backwards compatibility. @@ -220,8 +221,9 @@ markers are managed when new files are created *Default* `delete`: a request is issued to delete any parental directory markers whenever a file or directory is created. -*New* `keep`: No delete request is issued. Any directory markers which exist are -not deleted. This is *not* backwards compatible +*New* `keep`: No delete request is issued. +Any directory markers which exist are not deleted. +This is *not* backwards compatible *New* `authoritative`: directory markers are deleted _except for files created in "authoritative" directories_. @@ -230,12 +232,13 @@ This is backwards compatible _outside authoritative directories_. Until now, the notion of an "authoritative" directory has only been used as a performance optimization for deployments where it is known that all Applications are using the same S3Guard metastore -when writing and reading data. In such a deployment, if it is also known that -all applications are using a compatible version of the s3a connector, then they +when writing and reading data. +In such a deployment, if it is also known that all applications are using a +compatible version of the s3a connector, then they can switch to the higher-performance mode for those specific directories. Only the default setting, `fs.s3a.directory.marker.retention = delete` is compatible with -existing Hadoop releases. +every shipping Hadoop releases. ## Directory Markers and S3Guard @@ -264,7 +267,7 @@ line of bucket policies via the `-marker` option | `-markers aware` | the hadoop release is "aware" of directory markers | | `-markers delete` | directory markers are deleted | | `-markers keep` | directory markers are kept (not backwards compatible) | -| `-markers authoritative` | directory markers are kept in authoritative paths| +| `-markers authoritative` | directory markers are kept in authoritative paths | All releases of Hadoop which have been updated to be marker aware will support the `-markers aware` option. @@ -386,7 +389,7 @@ markers (-audit | -clean) [-expected ] [-out ] [-limit ] |-------|---------| | 0 | Success | | 3 | interrupted -the value of `-limit` was reached | -| 42 | Usage | +| 42 | Usage | | 46 | Markers were found (see HTTP "406", "unacceptable") | All other non-zero status code also indicate errors of some form or other. @@ -418,7 +421,6 @@ Listing limit reached before completing the scan Here the scan reached its object limit before completing the audit; the exit code of 3, "interrupted" indicates this. - Example: a verbose audit of a bucket whose policy if authoritative -it is not an error if markers are found under the path `/tables`. @@ -452,7 +454,6 @@ to use. The `-nonauth` option does not treat markers under authoritative paths as errors: - ``` bin/hadoop s3guard markers -nonauth -audit s3a://london/ @@ -548,11 +549,11 @@ which all FileSystem classes have supported since Hadoop 3.3. | Probe | Meaning | |-------------------------|-------------------------| | `fs.s3a.capability.directory.marker.aware` | Does the filesystem support surplus directory markers? | +| `fs.s3a.capability.directory.marker.policy.delete` | Is the bucket policy "delete"? | | `fs.s3a.capability.directory.marker.policy.keep` | Is the bucket policy "keep"? | -| `fs.s3a.capability.directory.marker.policy.delete` | Is the bucket policy "delete" | -| `fs.s3a.capability.directory.marker.policy.authoritative` | Is the bucket policy "authoritative" | -| `fs.s3a.capability.directory.marker.action.keep` | Does the path retain directory markers? | -| `fs.s3a.capability.directory.marker.action.delete` | Does the path delete directory markers? | +| `fs.s3a.capability.directory.marker.policy.authoritative` | Is the bucket policy "authoritative"? | +| `fs.s3a.capability.directory.marker.action.delete` | If a file was created at this path, would directory markers be deleted? | +| `fs.s3a.capability.directory.marker.action.keep` | If a file was created at this path, would directory markers be retained? | The probe `fs.s3a.capability.directory.marker.aware` allows for a filesystem to be @@ -684,7 +685,7 @@ and some applications. * [HADOOP-13230](https://issues.apache.org/jira/browse/HADOOP-13230). _S3A to optionally retain directory markers_ -* [HADOOP-16090](https://issues.apache.org/jira/browse/HADOOP-16090).. _S3A Client to add explicit support for versioned stores._ +* [HADOOP-16090](https://issues.apache.org/jira/browse/HADOOP-16090). _S3A Client to add explicit support for versioned stores._ * [HADOOP-16823](https://issues.apache.org/jira/browse/HADOOP-16823). _Large DeleteObject requests are their own Thundering Herd_ diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index 7ad0ad1c5e8d6..861da4d82ee23 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -16,24 +16,34 @@ -**NOTE: Hadoop's `s3:` and `s3n:` connectors have been removed. -Please use `s3a:` as the connector to data hosted in S3 with Apache Hadoop.** -**Consult the [s3n documentation](./s3n.html) for migration instructions.** +## Compatibility -See also: + +### Directory Marker Compatibility + +1. This release can safely list/index/read S3 buckets where "empty directory" +markers are retained. + +1. This release can be configured to retain these directory makers at the +expense of being backwards incompatible. + +Consult [Controlling the S3A Directory Marker Behavior](directory_markers.html) for +full details. + +## Documents * [Encryption](./encryption.html) * [Performance](./performance.html) * [S3Guard](./s3guard.html) * [Troubleshooting](./troubleshooting_s3a.html) +* [Controlling the S3A Directory Marker Behavior](directory_markers.html). * [Committing work to S3 with the "S3A Committers"](./committers.html) * [S3A Committers Architecture](./committer_architecture.html) * [Working with IAM Assumed Roles](./assumed_roles.html) * [S3A Delegation Token Support](./delegation_tokens.html) * [S3A Delegation Token Architecture](delegation_token_architecture.html). -* [Altering the S3A Directory Marker Behavior](directory_markers.html). * [Testing](./testing.html) ## Overview