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..af8983e2ebe92 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -52,6 +52,10 @@ 200000 + + false + + @@ -123,6 +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} @@ -163,6 +170,7 @@ ${fs.s3a.s3guard.test.enabled} ${fs.s3a.s3guard.test.authoritative} ${fs.s3a.s3guard.test.implementation} + ${fs.s3a.directory.marker.retention} ${test.integration.timeout} @@ -189,6 +197,8 @@ **/ITestDynamoDBMetadataStoreScale.java **/ITestTerasort*.java + + **/ITestMarkerToolRootOperations.java **/ITestS3GuardDDBRootOperations.java @@ -215,6 +225,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} @@ -230,6 +243,10 @@ **/ITestTerasort*.java + + + **/ITestMarkerToolRootOperations.java **/ITestS3AContractRootDir.java **/ITestS3GuardDDBRootOperations.java @@ -269,6 +286,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} @@ -332,6 +352,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..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 @@ -953,4 +953,92 @@ 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.marker.retention"; + + /** + * Delete directory markers. This is the backwards compatible option. + * Value: {@value}. + */ + public static final String DIRECTORY_MARKER_POLICY_DELETE = + "delete"; + + /** + * Retain directory markers. + * Value: {@value}. + */ + public static final String DIRECTORY_MARKER_POLICY_KEEP = + "keep"; + + /** + * Retain directory markers in authoritative directory trees only. + * Value: {@value}. + */ + public static final String DIRECTORY_MARKER_POLICY_AUTHORITATIVE = + "authoritative"; + + /** + * Default retention policy: {@value}. + */ + public static final String DEFAULT_DIRECTORY_MARKER_POLICY = + DIRECTORY_MARKER_POLICY_DELETE; + + + /** + * {@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"; + + /** + * {@code PathCapabilities} probe to indicate that the filesystem + * keeps directory markers. + * Value: {@value}. + */ + public static final String STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_KEEP + = "fs.s3a.capability.directory.marker.policy.keep"; + + /** + * {@code PathCapabilities} probe to indicate that the filesystem + * deletes directory markers. + * Value: {@value}. + */ + 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/InconsistentAmazonS3Client.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/InconsistentAmazonS3Client.java index 34c043be9cb73..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 @@ -330,7 +330,9 @@ 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/Listing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java index fcb492857e617..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 @@ -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 @@ -194,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 2cd23255c4b26..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 @@ -67,7 +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.S3ObjectSummary; + import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; import com.amazonaws.services.s3.model.SSECustomerKey; import com.amazonaws.services.s3.model.UploadPartRequest; @@ -104,6 +104,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; @@ -116,6 +118,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; @@ -295,6 +299,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 +465,10 @@ public void initialize(URI name, Configuration originalConf) DEFAULT_S3GUARD_DISABLED_WARN_LEVEL); S3Guard.logS3GuardDisabled(LOG, warnLevel, bucket); } + // directory policy, which may look at authoritative paths + directoryPolicy = DirectoryPolicyImpl.getDirectoryPolicy(conf, + this::allowAuthoritative); + LOG.debug("Directory marker retention policy is {}", directoryPolicy); initMultipartUploads(conf); @@ -1285,7 +1302,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 +1310,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 +1460,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,7 +1567,7 @@ 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)); } @@ -1585,7 +1617,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(destParent)) { + deleteUnnecessaryFakeDirectories(destParent, null); + } maybeCreateFakeParentDirectory(sourceRenamed); } } @@ -1940,6 +1974,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")) { @@ -2381,6 +2416,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 @@ -2490,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"); @@ -2573,7 +2617,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 +2633,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 +2662,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 +2670,8 @@ public FileStatus[] innerListStatus(Path f) throws FileNotFoundException, entryPoint(INVOCATION_LIST_STATUS); List result; - final FileStatus fileStatus = getFileStatus(path); + final S3AFileStatus fileStatus = innerGetFileStatus(path, false, + StatusProbeEnum.ALL); if (fileStatus.isDirectory()) { if (!key.isEmpty()) { @@ -2658,7 +2703,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 +2814,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 +2837,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 +2855,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 +2916,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 +2928,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 +2942,114 @@ 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); + // 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 of {} is out of date, probing S3", path); + try { + 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) { + // there's new data in S3 + LOG.debug("S3Guard metadata for {} is outdated;" + + " s3modtime={}; msModTime={} updating metastore", + path, s3ModTime, msModTime); + // add to S3Guard + 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); + } + // 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 + // 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 +3104,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 +3115,88 @@ 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); + // 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" + + " 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 +3215,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 +3229,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 +3632,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,16 +3766,21 @@ private Optional generateSSECustomerKey() { /** * Perform post-write actions. - * Calls {@link #deleteUnnecessaryFakeDirectories(Path)} 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)}
  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 @@ -3743,12 +3803,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 +3874,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 +3903,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()) { @@ -3952,6 +4032,14 @@ public long getDefaultBlockSize() { return getConf().getLongBytes(FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE); } + /** + * Get the directory marker policy of this filesystem. + * @return the marker policy. + */ + public DirectoryPolicy getDirectoryMarkerPolicy() { + return directoryPolicy; + } + @Override public String toString() { final StringBuilder sb = new StringBuilder( @@ -3990,6 +4078,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("}"); @@ -4086,25 +4175,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; + } } /** @@ -4511,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: @@ -4530,8 +4636,24 @@ public boolean hasPathCapability(final Path path, final String capability) case CommonPathCapabilities.FS_MULTIPART_UPLOADER: return true; + // this client is safe to use with buckets + // containing directory markers anywhere in + // the hierarchy + case STORE_CAPABILITY_DIRECTORY_MARKER_AWARE: + return true; + + /* + * 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); } } @@ -4546,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); @@ -4800,6 +4922,15 @@ public StoreContext createStoreContext() { .build(); } + /** + * Create a marker tools operations binding for this store. + * @return callbacks for operations. + */ + @InterfaceAudience.Private + public MarkerToolOperations createMarkerToolOperations() { + return new MarkerToolOperationsImpl(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..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 @@ -24,11 +24,18 @@ /** * 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) { + /** + * Format for the toString() method: {@value}. + */ + 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 +77,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..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 @@ -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,110 @@ 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 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 + */ + 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(); + } + + /** + * Dmp the result at debug level. + * @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..ca04fed65a539 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java @@ -0,0 +1,352 @@ +/* + * 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.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; + +/** + * 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. + *

+ * Consult the directory_markers document for details on this feature, + * including terminology. + */ +public class DirMarkerTracker { + + private static final Logger LOG = + LoggerFactory.getLogger(DirMarkerTracker.class); + + /** + * all leaf markers. + */ + private final Map leafMarkers + = new TreeMap<>(); + + /** + * all surplus markers. + */ + 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. + */ + private Path lastDirChecked; + + /** + * Count of scans; used for test assertions. + */ + 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. + *

+ * 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, + boolean recordSurplusMarkers) { + this.basePath = basePath; + this.recordSurplusMarkers = recordSurplusMarkers; + } + + /** + * 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. + * @param path marker path + * @param key object key + * @param source listing source + * @return the surplus markers found. + */ + public List markerFound(Path path, + final String key, + final S3ALocatedFileStatus source) { + markersFound++; + leafMarkers.put(path, new Marker(path, key, source)); + return pathFound(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 surplus markers found. + */ + public List fileFound(Path path, + final String key, + final S3ALocatedFileStatus source) { + filesFound++; + return pathFound(path, key, source); + } + + /** + * A path has been found. + *

+ * Declare all markers above it as surplus + * @param path marker path + * @param key object key + * @param source listing source + * @return the surplus markers found. + */ + private List pathFound(Path path, + final String key, + final S3ALocatedFileStatus source) { + objectsFound++; + List removed = new ArrayList<>(); + + // all parent entries are superfluous + final Path parent = path.getParent(); + if (parent == null || parent.equals(lastDirChecked)) { + // short cut exit + return removed; + } + removeParentMarkers(parent, removed); + lastDirChecked = parent; + return removed; + } + + /** + * 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. + */ + private void removeParentMarkers(final Path path, + List removed) { + if (path == null || path.isRoot()) { + return; + } + scanCount++; + removeParentMarkers(path.getParent(), removed); + final Marker value = leafMarkers.remove(path); + if (value != null) { + // marker is surplus + removed.add(value); + if (recordSurplusMarkers) { + surplusMarkers.put(path, value); + } + } + } + + /** + * Get the map of leaf markers. + * @return all leaf markers. + */ + public Map getLeafMarkers() { + return leafMarkers; + } + + /** + * Get the map of surplus markers. + *

+ * Empty if they were not being recorded. + * @return all surplus markers. + */ + public Map getSurplusMarkers() { + return surplusMarkers; + } + + public Path getLastDirChecked() { + return lastDirChecked; + } + + + /** + * How many objects were found. + * @return count + */ + public int getObjectsFound() { + return objectsFound; + } + + 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 + + '}'; + } + + /** + * 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. + */ + 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; + + private 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; + } + + /** + * 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{" + + "path=" + path + + ", key='" + key + '\'' + + ", status=" + status + + '}'; + } + + } + +} 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..36dd2e4fd2496 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicy.java @@ -0,0 +1,110 @@ +/* + * 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; + +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 { + + + + /** + * 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); + + /** + * Get the marker policy. + * @return policy. + */ + MarkerPolicy getMarkerPolicy(); + + /** + * Describe the policy for marker tools and logs. + * @return description of the current policy. + */ + 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. + */ + enum MarkerPolicy { + + /** + * Delete markers. + *

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

+ * This is Not backwards compatible. + */ + Keep(DIRECTORY_MARKER_POLICY_KEEP), + + /** + * Keep markers in authoritative paths only. + *

+ * This is Not backwards compatible within the + * auth paths, but is outside these. + */ + 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 new file mode 100644 index 0000000000000..a1aa2580b655a --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirectoryPolicyImpl.java @@ -0,0 +1,212 @@ +/* + * 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.EnumSet; +import java.util.Locale; +import java.util.Set; +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; +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. + */ +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 policy in " + + DIRECTORY_MARKER_POLICY + ": "; + + /** + * All available policies. + */ + private static final Set AVAILABLE_POLICIES = + EnumSet.allOf(MarkerPolicy.class); + + /** + * 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); + + /** + * Chosen marker policy. + */ + private final MarkerPolicy markerPolicy; + + /** + * Callback to evaluate authoritativeness of a + * path. + */ + private final Predicate authoritativeness; + + /** + * Constructor. + * @param markerPolicy marker policy + * @param authoritativeness function for authoritativeness + */ + public DirectoryPolicyImpl(final MarkerPolicy markerPolicy, + final Predicate authoritativeness) { + this.markerPolicy = markerPolicy; + this.authoritativeness = authoritativeness; + } + + @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 MarkerPolicy getMarkerPolicy() { + return markerPolicy; + } + + @Override + public String describe() { + return markerPolicy.getOptionName(); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "DirectoryMarkerRetention{"); + sb.append("policy='").append(markerPolicy.getOptionName()).append('\''); + sb.append('}'); + 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 + * @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; + } + + /** + * 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/impl/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java index 750aebf500a4b..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 @@ -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; @@ -43,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; @@ -55,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 @@ -183,12 +197,59 @@ 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 key key of the object. + * @param version object version. */ - private void queueToDelete(Path path, String key) { + private void queueToDelete(Path path, String key, String version) { + LOG.debug("Queueing to delete {}", path); 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. + *

+ * See {@link #queueToDelete(Path, String, String)} for + * details on safe use of this method. + * + * @param markersToDelete markers + */ + private void queueToDelete( + List markersToDelete) { + markersToDelete.forEach(m -> + 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()); } /** @@ -225,11 +286,19 @@ 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(); + // 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(); } @@ -254,15 +323,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; @@ -295,12 +366,14 @@ protected void renameFileToDest() throws IOException { callbacks.deleteObjectAtPath(sourcePath, sourceKey, true, null); // and update the tracker renameTracker.sourceObjectsDeleted(Lists.newArrayList(sourcePath)); + return copyDestinationPath; } /** * 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,10 +398,18 @@ protected void recursiveDirectoryRename() throws IOException { // marker. LOG.debug("Deleting fake directory marker at destination {}", destStatus.getPath()); + // 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, + false); + final RemoteIterator iterator = callbacks.listFilesAndEmptyDirectories(parentPath, sourceStatus, @@ -347,36 +428,45 @@ 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); - - // 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()); + List markersToDelete; - 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) { + // 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 { + // it is a file. + // 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); + // 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); + + // 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); + 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 + // 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 // have been deleted. @@ -387,6 +477,93 @@ 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 (keysToDelete.size() == pageSize) { + // finish ongoing copies then delete all queued keys. + 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. + * 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 + * 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)); + queueToDelete(entry); + // end of loop + endOfLoopActions(); + } + duration.close(); + return duration; + } + /** * Initiate a copy operation in the executor. * @param source status of the source object. @@ -487,6 +664,16 @@ private void removeSourceObjects( List undeletedObjects = new ArrayList<>(); try { // remove the keys + + // 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) { + 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( @@ -498,7 +685,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/impl/StatusProbeEnum.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StatusProbeEnum.java index f843b20ab28b0..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. */ @@ -33,28 +36,23 @@ 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/NullMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java index 666c233575ad6..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("null tracker", 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/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..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 @@ -30,12 +30,14 @@ 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; 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; @@ -44,12 +46,15 @@ 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; 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; @@ -58,7 +63,10 @@ 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.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; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.security.UserGroupInformation; @@ -79,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); @@ -98,15 +110,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.MARKERS + " - " + 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)"; @@ -116,6 +130,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; @@ -472,6 +487,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; } @@ -497,6 +520,30 @@ 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. + * Only non-zero statistics are printed. + * @param stream output stream + */ + protected void dumpFileSystemStatistics(PrintStream stream) { + FileSystem fs = getFilesystem(); + if (fs == null) { + return; + } + 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(); + long value = next.getValue(); + if (value != 0) { + println(stream, "%s\t%s", next.getName(), value); + } + } + println(stream, ""); + } + /** * Create the metadata store. */ @@ -1167,16 +1214,20 @@ 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"; 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 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" @@ -1186,7 +1237,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 + + " (aware, keep, delete, authoritative) - directory markers policy\n"; /** * Output when the client cannot get the location of a bucket. @@ -1196,10 +1249,17 @@ 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(); format.addOptionWithValue(ENCRYPTION_FLAG); + format.addOptionWithValue(MARKERS_FLAG); } @Override @@ -1384,10 +1444,57 @@ public int run(String[] args, PrintStream out) fsUri, desiredEncryption, encryption); } + // 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, "%nThe directory marker policy is \"%s\"%n", desc); + + DirectoryPolicy.MarkerPolicy mp = markerPolicy.getMarkerPolicy(); + + String desiredMarker = marker == null + ? "" + : 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); + } + } + } + } + private String printOption(PrintStream out, String description, String key, String defVal) { String t = getFilesystem().getConf().getTrimmed(key, defVal); @@ -1991,6 +2098,9 @@ public static int run(Configuration conf, String...args) throws case Diff.NAME: command = new Diff(conf); break; + case MarkerTool.MARKERS: + 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..6855c52edbbbb --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -0,0 +1,723 @@ +/* + * 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.FileNotFoundException; +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; +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; + +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; +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; +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.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.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; +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; + +/** + * 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. + */ +@InterfaceAudience.LimitedPrivate("management tools") +@InterfaceStability.Unstable +public final class MarkerTool extends S3GuardTool { + + private static final Logger LOG = LoggerFactory.getLogger(MarkerTool.class); + + /** + * Name of this tool: {@value}. + */ + public static final String MARKERS = "markers"; + + /** + * Purpose of this tool: {@value}. + */ + public static final String PURPOSE = + "View and manipulate S3 directory markers"; + + /** + * Audit sub-command: {@value}. + */ + public static final String OPT_AUDIT = "audit"; + + /** + * Clean Sub-command: {@value}. + */ + 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}. + */ + 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. + */ + @VisibleForTesting + static final String E_ARGUMENTS = "Wrong number of arguments: %d"; + + /** + * Constant to use when there is no limit on the number of + * 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_LISTING = 0; + + + /** + * Usage string: {@value}. + */ + private static final String USAGE = MARKERS + + " (-" + OPT_AUDIT + + " | -" + OPT_CLEAN + ")" + + " [-" + OPT_EXPECTED + " ]" + + " [-" + OPT_OUT + " ]" + + " [-" + OPT_LIMIT + " ]" + + " [-" + OPT_NONAUTH + "]" + + " [-" + VERBOSE + "]" + + + " \n" + + "\t" + PURPOSE + "\n\n"; + + /** Will be overridden in run(), but during tests needs to avoid NPEs. */ + private PrintStream out = System.out; + + /** + * Verbosity flag. + */ + private boolean verbose; + + /** + * Store context. + */ + private StoreContext storeContext; + + /** + * Operations during the scan. + */ + private MarkerToolOperations operations; + + /** + * Constructor. + * @param conf configuration + */ + public MarkerTool(final Configuration conf) { + 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 + public String getUsage() { + return USAGE; + } + + @Override + public String getName() { + return MARKERS; + } + + @Override + public void resetBindings() { + super.resetBindings(); + storeContext = null; + operations = null; + } + + @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() != 1) { + errorln(getUsage()); + println(out, "Supplied arguments: [" + + parsedArgs.stream() + .collect(Collectors.joining(", ")) + + "]"); + throw new ExitUtil.ExitException(EXIT_USAGE, + String.format(E_ARGUMENTS, parsedArgs.size())); + } + // read arguments + CommandFormat command = getCommandFormat(); + verbose = command.getOpt(VERBOSE); + + // How many markers are expected? + int expected = 0; + String value = command.getOptValue(OPT_EXPECTED); + if (value != null && !value.isEmpty()) { + expected = Integer.parseInt(value); + } + + // 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, + "Exactly one of " + AUDIT + " and " + CLEAN); + } + int limit = UNLIMITED_LISTING; + value = command.getOptValue(OPT_LIMIT); + if (value != null && !value.isEmpty()) { + limit = Integer.parseInt(value); + } + 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 + path = new Path(path, "/"); + } + FileSystem fs = path.getFileSystem(getConf()); + ScanResult result = execute( + fs, + path, + clean, + expected, + limit, + 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 OutputStreamWriter( + new FileOutputStream(saveFile), + StandardCharsets.UTF_8)) { + final List surplus = result.getTracker() + .getSurplusMarkers() + .keySet() + .stream() + .map(p-> p.toString() + "/") + .sorted() + .collect(Collectors.toList()); + IOUtils.writeLines(surplus, "\n", writer); + } + } + return result.exitCode; + } + + /** + * 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 + * @param limit limit of files to scan; -1 for 'unlimited' + * @param nonAuth consider only markers in nonauth paths as errors + * @return scan+purge result. + * @throws IOException failure + */ + @VisibleForTesting + ScanResult execute( + final FileSystem sourceFS, + final Path path, + final boolean doPurge, + final int expectedMarkerCount, + 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(); + // 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 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. + 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(target); + } catch (UnknownStoreException ex) { + // bucket doesn't exist. + // replace the stack trace with an error code. + throw new ExitUtil.ExitException(EXIT_NOT_FOUND, + ex.toString(), ex); + + } catch (FileNotFoundException ex) { + throw new ExitUtil.ExitException(EXIT_NOT_FOUND, + "Not found: " + target, ex); + } + + // 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; + } + + /** + * Result of the scan operation. + */ + public 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; + + private ScanResult() { + } + + @Override + public String toString() { + return "ScanResult{" + + "exitCode=" + exitCode + + ", tracker=" + tracker + + ", 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; + } + } + + /** + * Do the scan/purge. + * @param path path to scan. + * @param clean purge? + * @param expectedMarkerCount expected marker count + * @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 clean, + final int expectedMarkerCount, + final int limit, + final DirectoryPolicy filterPolicy) + throws IOException, ExitUtil.ExitException { + + ScanResult result = new ScanResult(); + + // Mission Accomplished + result.exitCode = EXIT_SUCCESS; + // Now do the work. + DirMarkerTracker tracker = new DirMarkerTracker(path, true); + result.tracker = tracker; + boolean completed; + try (DurationInfo ignored = + 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 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", + surplus, + suffix(surplus), + path); + + for (Path markers : surplusMarkers.keySet()) { + println(out, " %s/", markers); + } + } + if (!leafMarkers.isEmpty()) { + println(out, "Found %d empty directory 'leaf' marker%s under %s", + leafMarkers.size(), + suffix(leafMarkers.size()), + path); + for (Path markers : leafMarkers.keySet()) { + println(out, " %s/", markers); + } + println(out, "These are required to indicate empty directories"); + } + + 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; + } + } + + // 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; + } + + /** + * 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"; + } + + /** + * 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 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( + status, null); + String key = storeContext.pathToKey(statusPath); + if (status.isDirectory()) { + if (verbose) { + println(out, " Directory Marker %s/", key); + } + LOG.debug("{}", key); + tracker.markerFound(statusPath, + key + "/", + locatedStatus); + } else { + tracker.fileFound(statusPath, + key, + locatedStatus); + } + if ((count % 1000) == 0) { + println(out, "Scanned %,d objects", count); + } + if (limit > 0 && count >= limit) { + println(out, "Limit of scan reached - %,d object%s", + limit, suffix(limit)); + return false; + } + } + return true; + } + + /** + * 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 + + '}'; + } + + + int getMarkersDeleted() { + return markersDeleted; + } + + int getDeleteRequests() { + return deleteRequests; + } + + long getTotalDeleteRequestDuration() { + return totalDeleteRequestDuration; + } + } + + /** + * Purge the markers. + * @param tracker tracker with the details + * @param deletePageSize page size of deletes + * @return summary + * @throws IOException IO failure + */ + @Retries.RetryTranslated + private MarkerPurgeSummary purgeMarkers( + final DirMarkerTracker tracker, + final 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.getKey())) + .collect(Collectors.toList()); + // 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. + Collections.shuffle(markerKeys); + int pages = size / deletePageSize; + if (size % deletePageSize > 0) { + pages += 1; + } + if (verbose) { + 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 + int end = Math.min(start + deletePageSize, size); + List page = markerKeys.subList(start, + end); + List undeleted = new ArrayList<>(); + once("Remove S3 Keys", + tracker.getBasePath().toString(), () -> + operations.removeKeys(page, true, undeleted, null, false)); + summary.deleteRequests++; + // and move to the start of the next page + start = end; + } + durationInfo.close(); + summary.totalDeleteRequestDuration = durationInfo.value(); + summary.markersDeleted = size; + return summary; + } + + public boolean isVerbose() { + return verbose; + } + + 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 + * @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") + public static MarkerTool.ScanResult execMarkerTool( + final FileSystem sourceFS, + final Path path, + final boolean doPurge, + final int expectedMarkers, + 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, nonAuth); + } +} 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..9ab7636d6c99f --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperations.java @@ -0,0 +1,91 @@ +/* + * 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.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..d14bb6b1d8ebb --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperationsImpl.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.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/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..cb3a3749b658c --- /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. + */ + +/** + * S3A Command line tools independent of S3Guard. + */ +@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/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..3773629b8e439 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/directory_markers.md @@ -0,0 +1,694 @@ + + +# Controlling the S3A Directory Marker Behavior + +## Critical: this is not backwards compatible! + +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_. + +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 | 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. + +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 compatible. + +### 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 +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. + +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 +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. + + +## 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 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. +* 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 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 +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 deleted, the +directory still exists. + +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 +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)` 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. + +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. + +## The Problem with Directory Markers + +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 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. + +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. + +2. Every single object listed in the delete request is considered to be a write +operation. + +3. In versioned S3 buckets, tombstone markers are added to the S3 indices even +if no object was deleted. + +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. +When writing many files to a deep directory tree, it is the delete calls which +create throttling problems. + +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. + +## Strategies to avoid marker-related problems. + +### Presto: every path is a directory + +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 +is very efficient. That's a big requirement however, -one Presto can pull off +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, 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. + +## 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 +every shipping 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. + +*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. + +## 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 +for surplus markers, and for optionally deleting them. +Leaf-node markers for empty directories are not considered surplus and +will be retained. + +Syntax + +``` +> hadoop s3guard markers -verbose -nonauth +markers (-audit | -clean) [-expected ] [-out ] [-limit ] [-nonauth] [-verbose] + View and manipulate S3 directory markers + +``` + +*Options* + +| 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* + +| Code | Meaning | +|-------|---------| +| 0 | Success | +| 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 -audit` + +Audit the path and fail if any markers were found. + + +``` +> 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 +``` + +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. + +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 + + +### 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 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.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 +probed to determine if its file listing policy is "aware" of directory marker retention +-that is: can this s3a client safely work with S3 buckets where markers have not been deleted. + +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. + +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 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. + +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.action.keep` will hold under +the path `s3a://london/tables` + +``` +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.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.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.action.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/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index 964bda49dd069..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,18 +16,29 @@ -**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) 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 d94288dfc307f..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 @@ -25,16 +25,20 @@ 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.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; 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; +import static org.apache.hadoop.fs.s3a.tools.MarkerTool.UNLIMITED_LISTING; /** * An extension of the contract test base set up for S3A tests. @@ -62,18 +66,43 @@ public void setup() throws Exception { @Override public void teardown() throws Exception { Thread.currentThread().setName("teardown"); + + maybeAuditTestPath(); + super.teardown(); describe("closing file system"); IOUtils.closeStream(getFileSystem()); } - @Before - public void nameThread() { - Thread.currentThread().setName("JUnit-" + getMethodName()); - } - - protected String getMethodName() { - return methodName.getMethodName(); + /** + * 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.isDirectory(methodPath)) { + MarkerTool.ScanResult result = MarkerTool.execMarkerTool(fs, + methodPath, true, 0, UNLIMITED_LISTING, false); + assertEquals("Audit of " + methodPath + " failed: " + result, + 0, result.getExitCode()); + } + } 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); + } + } + } } @Override 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/ITestS3ABucketExistence.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABucketExistence.java index 6be9003e4ec38..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 @@ -75,7 +75,11 @@ 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. + 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..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 @@ -18,15 +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.FileStatus; + +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.performance.AbstractS3ACostTest; + -import org.assertj.core.api.Assertions; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -39,26 +38,21 @@ import java.util.Arrays; import java.util.Collection; import java.util.EnumSet; -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.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.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 status queries. - * {@link S3AFileSystem#getFileStatus(Path)}. - * Parameterized on guarded vs raw. + * 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 { - - private MetricDiff metadataRequests; - private MetricDiff listRequests; +public class ITestS3AFileOperationCost extends AbstractS3ACostTest { private static final Logger LOG = LoggerFactory.getLogger(ITestS3AFileOperationCost.class); @@ -69,103 +63,62 @@ public class ITestS3AFileOperationCost extends AbstractS3ATestBase { @Parameterized.Parameters(name = "{0}") public static Collection params() { return Arrays.asList(new Object[][]{ - {"raw", false}, - {"guarded", true} + {"raw-keep-markers", false, true, false}, + {"raw-delete-markers", false, false, false}, + {"nonauth-keep-markers", true, true, false}, + {"auth-delete-markers", true, false, true} }); } - private final String name; - - private final boolean s3guard; - - public ITestS3AFileOperationCost(final String name, final boolean s3guard) { - this.name = name; - this.s3guard = s3guard; - } - - @Override - public Configuration createConfiguration() { - Configuration conf = super.createConfiguration(); - 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); - } - disableFilesystemCaching(conf); - return conf; - } - @Override - public void setup() throws Exception { - super.setup(); - if (s3guard) { - // 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); + public ITestS3AFileOperationCost(final String name, + final boolean s3guard, + final boolean keepMarkers, + final boolean authoritative) { + super(s3guard, keepMarkers, authoritative); } + /** + * 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 = 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), + whenRaw(FILE_STATUS_FILE_PROBE + .plus(LIST_LOCATED_STATUS_LIST_OP)), + whenAuthoritative(LIST_LOCATED_STATUS_LIST_OP), + whenNonauth(LIST_LOCATED_STATUS_LIST_OP)); } @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), + whenRaw(LIST_LOCATED_STATUS_LIST_OP + .plus(GET_FILE_STATUS_ON_EMPTY_DIR)), + whenAuthoritative(NO_IO), + whenNonauth(LIST_LOCATED_STATUS_LIST_OP)); } @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), + whenRaw(LIST_LOCATED_STATUS_LIST_OP), + whenAuthoritative(NO_IO), + whenNonauth(LIST_LOCATED_STATUS_LIST_OP)); } @Test @@ -174,36 +127,27 @@ public void testCostOfListFilesOnFile() throws Throwable { Path file = path(getMethodName() + ".txt"); S3AFileSystem fs = getFileSystem(); touch(fs, file); - resetMetricDiffs(); - fs.listFiles(file, true); - if (!fs.hasMetadataStore()) { - metadataRequests.assertDiffEquals(1); - } else { - if (fs.allowAuthoritative(file)) { - listRequests.assertDiffEquals(0); - } else { - listRequests.assertDiffEquals(1); - } - } + verifyMetrics(() -> + fs.listFiles(file, true), + whenRaw(LIST_LOCATED_STATUS_LIST_OP + .plus(GET_FILE_STATUS_ON_FILE)), + whenAuthoritative(NO_IO), + whenNonauth(LIST_LOCATED_STATUS_LIST_OP)); } @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); - 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), + whenRaw(LIST_FILES_LIST_OP + .plus(GET_FILE_STATUS_ON_EMPTY_DIR)), + whenAuthoritative(NO_IO), + whenNonauth(LIST_FILES_LIST_OP)); } @Test @@ -214,17 +158,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), + whenRaw(LIST_FILES_LIST_OP), + whenAuthoritative(NO_IO), + whenNonauth(LIST_FILES_LIST_OP)); } @Test @@ -232,118 +170,70 @@ 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), + whenRaw(LIST_FILES_LIST_OP + .plus(GET_FILE_STATUS_FNFE))); } @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 = verifyRawInnerGetFileStatus(simpleFile, true, + StatusProbeEnum.ALL, + GET_FILE_STATUS_ON_FILE); 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 = verifyRawInnerGetFileStatus(dir, true, + StatusProbeEnum.ALL, + GET_FILE_STATUS_ON_DIR_MARKER); 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); - } + verifyRawInnerGetFileStatus(dir, false, + StatusProbeEnum.DIRECTORIES, + FILE_STATUS_DIR_PROBE); + + // now look at isFile/isDir against the same entry + isDir(dir, true, FILE_STATUS_DIR_PROBE); + isFile(dir, false, FILE_STATUS_FILE_PROBE); } @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); + interceptRawGetFileStatusFNFE(methodPath(), false, + StatusProbeEnum.ALL, + GET_FILE_STATUS_FNFE); } @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, + FILE_STATUS_DIR_PROBE); + isFile(path, false, + FILE_STATUS_FILE_PROBE); } @Test public void testCostOfGetFileStatusOnNonEmptyDir() throws Throwable { describe("performing getFileStatus on a non-empty 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); - } + Path dir = dir(methodPath()); + file(new Path(dir, "simple.txt")); + S3AFileStatus status = verifyRawInnerGetFileStatus(dir, true, + StatusProbeEnum.ALL, + GET_FILE_STATUS_ON_DIR); + assertEmptyDirStatus(status, Tristate.FALSE); } - @Test public void testCostOfCopyFromLocalFile() throws Throwable { describe("testCostOfCopyFromLocalFile"); @@ -361,19 +251,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"; + }, + with(INVOCATION_COPY_FROM_LOCAL_FILE, 1), + with(OBJECT_PUT_REQUESTS, 1), + with(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,268 +270,123 @@ public void testCostOfCopyFromLocalFile() throws Throwable { } } - private boolean reset(MetricDiff... diffs) { - for (MetricDiff diff : diffs) { - diff.reset(); - } - return true; - } - - @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(); - - 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(); - - // 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; - - int destDirDepth = directoriesInPath(destDir); - directoriesCreated.assertDiffEquals(state, 1); - deleteRequests.assertDiffEquals(state, 1); - directoriesDeleted.assertDiffEquals(state, 0); - fakeDirectoriesDeleted.assertDiffEquals(state, destDirDepth - 1); - - // create a new source file. - // Explicitly use a new path object to guarantee that the parent paths - // are different object instances - final Path srcFile2 = new Path(srcDir.toUri() + "/source2.txt"); - touch(fs, srcFile2); - - reset.call(); - - // rename the source file to the destination file. - // this tests the file rename path, not the dir rename path - // as srcFile2 exists, the parent dir of srcFilePath must not be created. - fs.rename(srcFilePath, destFilePath); - state = 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 - assertIsFile(destFilePath); - assertIsDirectory(srcDir); - assertPathDoesNotExist("should have gone in the rename", srcFilePath); - reset.call(); - - // rename the source file2 to the (no longer existing - // this tests the file rename path, not the dir rename path - // as srcFile2 exists, the parent dir of srcFilePath must not be created. - fs.rename(srcFile2, srcFilePath); - state = String.format("after rename(%s, %s) %s dest dir depth=%d", - srcFile2, srcFilePath, - summary, - destDirDepth); - - // here we expect there to be no fake directories - directoriesCreated.assertDiffEquals(state, 0); - // one for the renamed file only - deleteRequests.assertDiffEquals(state, 1); - directoriesDeleted.assertDiffEquals(state, 0); - fakeDirectoriesDeleted.assertDiffEquals(state, 0); - } - - private int directoriesInPath(Path path) { - return path.isRoot() ? 0 : 1 + directoriesInPath(path.getParent()); - } - @Test - public void testCostOfRootRename() throws Throwable { - describe("assert that a root directory rename doesn't" - + " do much in terms of parent dir operations"); + public void testDirProbes() throws Throwable { + describe("Test directory probe cost"); + assumeUnguarded(); S3AFileSystem fs = getFileSystem(); + // Create the empty directory. + Path emptydir = dir(methodPath()); - // unique name, so that even when run in parallel tests, there's no conflict - String uuid = UUID.randomUUID().toString(); - Path src = new Path("/src-" + uuid); - Path dest = new Path("/dest-" + uuid); + // head probe fails + interceptRawGetFileStatusFNFE(emptydir, false, + StatusProbeEnum.HEAD_ONLY, + FILE_STATUS_FILE_PROBE); - try { - MetricDiff deleteRequests = - new MetricDiff(fs, Statistic.OBJECT_DELETE_REQUESTS); - MetricDiff directoriesDeleted = - new MetricDiff(fs, Statistic.DIRECTORIES_DELETED); - MetricDiff fakeDirectoriesDeleted = - new MetricDiff(fs, Statistic.FAKE_DIRECTORIES_DELETED); - MetricDiff directoriesCreated = - new MetricDiff(fs, Statistic.DIRECTORIES_CREATED); - touch(fs, src); - fs.rename(src, dest); - Object summary = new Object() { - @Override - public String toString() { - return String.format("[%s, %s, %s, %s]", - directoriesCreated, directoriesDeleted, - deleteRequests, fakeDirectoriesDeleted); - } - }; - - String state = String.format("after touch(%s) %s", - src, summary); - touch(fs, src); - fs.rename(src, dest); - directoriesCreated.assertDiffEquals(state, 0); - - - state = String.format("after rename(%s, %s) %s", - src, dest, summary); - // here we expect there to be no fake directories - directoriesCreated.assertDiffEquals(state, 0); - // one for the renamed file only - deleteRequests.assertDiffEquals(state, 1); - directoriesDeleted.assertDiffEquals(state, 0); - fakeDirectoriesDeleted.assertDiffEquals(state, 0); - - // delete that destination file, assert only the file delete was issued - reset(deleteRequests, directoriesCreated, directoriesDeleted, - fakeDirectoriesDeleted); - - fs.delete(dest, false); - // here we expect there to be no fake directories - directoriesCreated.assertDiffEquals(state, 0); - // one for the deleted file - deleteRequests.assertDiffEquals(state, 1); - directoriesDeleted.assertDiffEquals(state, 0); - fakeDirectoriesDeleted.assertDiffEquals(state, 0); - } finally { - fs.delete(src, false); - fs.delete(dest, false); - } - } + // a LIST will find it and declare as empty + S3AFileStatus status = verifyRawInnerGetFileStatus(emptydir, true, + StatusProbeEnum.LIST_ONLY, + FILE_STATUS_DIR_PROBE); + assertEmptyDirStatus(status, Tristate.TRUE); - @Test - public void testDirProbes() throws Throwable { - describe("Test directory probe cost -raw only"); - S3AFileSystem fs = getFileSystem(); - assume("Unguarded FS only", !fs.hasMetadataStore()); - String dir = "testEmptyDirHeadProbe"; - Path emptydir = path(dir); - // Create the empty directory. - fs.mkdirs(emptydir); - - // metrics and assertions. - resetMetricDiffs(); - - 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 + interceptRawGetFileStatusFNFE(emptydir, false, + EnumSet.noneOf(StatusProbeEnum.class), + NO_IO); // 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, () -> + interceptRaw(FileNotFoundException.class, "", + NO_IO, () -> 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 = verifyRaw(FILE_STATUS_DIR_PROBE, () -> + fs.s3GetFileStatus(emptydir, + emptyDirTrailingSlash, + StatusProbeEnum.LIST_ONLY, + null, + true)); assertEquals(emptydir, status.getPath()); + assertEmptyDirStatus(status, Tristate.TRUE); } @Test - public void testCreateCost() throws Throwable { - describe("Test file creation cost -raw only"); + public void testNeedEmptyDirectoryProbeRequiresList() throws Throwable { S3AFileSystem fs = getFileSystem(); - assume("Unguarded FS only", !fs.hasMetadataStore()); - resetMetricDiffs(); - Path testFile = path("testCreateCost"); + 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"); + 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, + CREATE_FILE_NO_OVERWRITE); // but when true: only the directory checks take place. - try (FSDataOutputStream out = fs.create(testFile, true)) { - verifyOperationCount(1, 1); - } + create(testFile, true, CREATE_FILE_OVERWRITE); + } + + @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. + interceptRaw(FileAlreadyExistsException.class, "", + FILE_STATUS_FILE_PROBE, + () -> 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. + interceptRaw(FileAlreadyExistsException.class, "", + GET_FILE_STATUS_ON_DIR_MARKER, + () -> 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, + 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); + + // 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)); } @Test @@ -656,15 +400,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); + verifyRaw(GET_FILE_STATUS_ON_DIR + .plus(LIST_OPERATION), + () -> fs.globStatus(basePath.suffix("/*"))); } @Test @@ -678,14 +422,14 @@ 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); + 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/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/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..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 @@ -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 " + - "in a recursive listing", - preDeleteUndelimited.getObjectSummaries(), - postDeleteUndelimited.getObjectSummaries() - ); + assertObjectSummariesEqual("InconsistentAmazonS3Client added back objects" + + " incorrectly in a recursive listing", + preDeleteUndelimited, postDeleteUndelimited, + stripTombstones); assertListSizeEqual("InconsistentAmazonS3Client added back prefixes incorrectly " + "in a recursive listing", @@ -586,6 +586,24 @@ 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 +700,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/ITestS3GuardOutOfBandOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardOutOfBandOperations.java index 32ead7f3fed71..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 @@ -56,7 +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.DEFAULT_METADATASTORE_METADATA_TTL; +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.METADATASTORE_AUTHORITATIVE; import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_METADATA_TTL; import static org.apache.hadoop.fs.s3a.Constants.RETRY_INTERVAL; @@ -169,12 +170,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 +237,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 +863,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/S3ATestConstants.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java index 118c9ee773a6b..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}. @@ -218,4 +223,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 aa5979dbf751e..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,14 @@ 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, + DEFAULT_DIRECTORY_MARKER_POLICY); + conf.set(DIRECTORY_MARKER_POLICY, directoryRetention); + return conf; } @@ -882,7 +890,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); } /** @@ -1441,4 +1450,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/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..194cd645c0714 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestDirectoryMarkerPolicy.java @@ -0,0 +1,163 @@ +/* + * 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.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. + */ +@RunWith(Parameterized.class) +public class TestDirectoryMarkerPolicy extends AbstractHadoopTestBase { + + @Parameterized.Parameters(name = "{0}") + public static Collection data() { + return Arrays.asList(new Object[][]{ + { + DirectoryPolicy.MarkerPolicy.Delete, + FAIL_IF_INVOKED, + false, false + }, + { + DirectoryPolicy.MarkerPolicy.Keep, + FAIL_IF_INVOKED, + true, true + }, + { + DirectoryPolicy.MarkerPolicy.Authoritative, + AUTH_PATH_ONLY, + false, true + } + }); + } + + private final DirectoryPolicy directoryPolicy; + + private final boolean expectNonAuthDelete; + + private final boolean expectAuthDelete; + + public TestDirectoryMarkerPolicy( + final DirectoryPolicy.MarkerPolicy markerPolicy, + final Predicate authoritativeness, + final boolean expectNonAuthDelete, + final boolean expectAuthDelete) { + this.directoryPolicy = newPolicy(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 DirectoryPolicy newPolicy( + DirectoryPolicy.MarkerPolicy markerPolicy, + Predicate authoritativeness) { + return new DirectoryPolicyImpl(markerPolicy, 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"); + + /** + * 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 { + 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 { + 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 { + 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/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java new file mode 100644 index 0000000000000..db0542ddc94a6 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java @@ -0,0 +1,637 @@ +/* + * 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.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.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.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? + */ + 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. */ + private boolean isGuarded; + + private boolean isRaw; + + private boolean isAuthoritative; + + private boolean isNonAuth; + + private boolean isKeeping; + + private 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); + } + removeBaseAndBucketOverrides(bucketName, conf, + DIRECTORY_MARKER_POLICY, + METADATASTORE_AUTHORITATIVE, + AUTHORITATIVE_PATH); + // directory marker options + 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 cost expected cost + * @return path to new object. + */ + protected Path buildFile(Path path, + boolean overwrite, + boolean recursive, + OperationCost cost) throws Exception { + resetStatistics(); + verifyRaw(cost, () -> { + 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); + } + + /** + * Create then close the file. + * @param path path + * @param overwrite overwrite flag + * @param cost expected cost + + * @return path to new object. + */ + protected Path create(Path path, boolean overwrite, + OperationCost cost) throws Exception { + return verifyRaw(cost, () -> + 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 interceptRaw( + Class clazz, + String text, + OperationCost cost, + Callable eval) throws Exception { + return verifyMetricsIntercepting(clazz, text, eval, whenRaw(cost)); + } + + /** + * Declare the expected cost on any FS. + * @param cost costs to expect + * @return a probe. + */ + protected OperationCostValidator.ExpectedProbe always( + OperationCost cost) { + return expect(true, cost); + } + + /** + * Declare the expected cost on a raw FS. + * @param cost costs to expect + * @return a probe. + */ + protected OperationCostValidator.ExpectedProbe whenRaw( + OperationCost cost) { + return expect(isRaw(), cost); + } + + /** + * Declare the expected cost on a guarded FS. + * @param cost costs to expect + * @return a probe. + */ + protected OperationCostValidator.ExpectedProbe whenGuarded( + OperationCost cost) { + return expect(isGuarded(), cost); + } + + /** + * Declare the expected cost on a guarded auth FS. + * @param cost costs to expect + * @return a probe. + */ + protected OperationCostValidator.ExpectedProbe whenAuthoritative( + OperationCost cost) { + return expect(isAuthoritative(), cost); + } + + + /** + * Declare the expected cost on a guarded nonauth FS. + * @param cost costs to expect + * @return a probe. + */ + 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 cost expected cost + * @param eval closure to evaluate + * @param return type of closure + * @return the result of the evaluation + */ + protected T verifyRaw( + OperationCost cost, + Callable eval) throws Exception { + return verifyMetrics(eval, whenRaw(cost)); + } + + /** + * Execute {@code S3AFileSystem#innerGetFileStatus(Path, boolean, Set)} + * for the given probes. + * 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 cost expected cost + * @return the status + */ + public S3AFileStatus verifyRawInnerGetFileStatus( + Path path, + boolean needEmptyDirectoryFlag, + Set probes, + OperationCost cost) throws Exception { + return verifyRaw(cost, () -> + innerGetFileStatus(getFileSystem(), + path, + needEmptyDirectoryFlag, + probes)); + } + + /** + * Execute {@code S3AFileSystem#innerGetFileStatus(Path, boolean, Set)} + * for the given probes -expect a FileNotFoundException, + * 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 cost expected cost + */ + + public void interceptRawGetFileStatusFNFE( + Path path, + boolean needEmptyDirectoryFlag, + Set probes, + OperationCost cost) throws Exception { + interceptRaw(FileNotFoundException.class, "", + cost, () -> + 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 cost expected cost on a Raw FS. + */ + 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) + .isEqualTo(expected); + } + + /** + * Probe for a path being a file. + * Metrics are only checked on unguarded stores. + * @param path path + * @param expected expected outcome + * @param cost expected cost on a Raw FS. + */ + 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) + .isEqualTo(expected); + } + + /** + * A metric diff which must always hold. + * @param stat metric source + * @param expected expected value. + * @return the diff. + */ + protected OperationCostValidator.ExpectedProbe with( + final Statistic stat, final int expected) { + return probe(stat, expected); + } + + /** + * A metric diff which must hold when the fs is unguarded. + * @param stat metric source + * @param expected expected value. + * @return the diff. + */ + protected OperationCostValidator.ExpectedProbe withWhenRaw( + final Statistic stat, final int expected) { + return probe(isRaw(), stat, expected); + } + + /** + * A metric diff which must hold when the fs is guarded. + * @param stat metric source + * @param expected expected value. + * @return the diff. + */ + protected OperationCostValidator.ExpectedProbe withWhenGuarded( + final Statistic stat, + final int expected) { + return probe(isGuarded(), stat, expected); + } + + /** + * A metric diff which must hold when the fs is guarded + authoritative. + * @param stat metric source + * @param expected expected value. + * @return the diff. + */ + protected OperationCostValidator.ExpectedProbe withWhenAuthoritative( + final Statistic stat, + final int expected) { + return probe(isAuthoritative(), stat, expected); + } + + /** + * A metric diff which must hold when the fs is guarded + authoritative. + * @param stat metric source + * @param expected expected value. + * @return the diff. + */ + protected OperationCostValidator.ExpectedProbe withWhenNonauth( + final Statistic stat, + final int expected) { + return probe(isNonAuth(), stat, expected); + } + + /** + * 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 withWhenKeeping( + final Statistic stat, + final int expected) { + return probe(isKeepingMarkers(), stat, expected); + } + + /** + * 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 withWhenDeleting( + final Statistic stat, + final int expected) { + return probe(isDeleting(), stat, expected); + } + + /** + * 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 + */ + 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/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..ed56802ddfec1 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestDirectoryMarkerListing.java @@ -0,0 +1,824 @@ +/* + * 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.FileNotFoundException; +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; + +import com.amazonaws.AmazonClientException; +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; + +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.S3AFileSystem; +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; +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 -the bucket settings are changed + * appropriately. + */ +@RunWith(Parameterized.class) +public class ITestDirectoryMarkerListing extends AbstractS3ATestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestDirectoryMarkerListing.class); + + 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"; + + /** + * 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} + *

+ * Older releases: yes. + *

+ * The full marker-optimized releases: no. + */ + private static final boolean RENAME_COPIES_MARKERS = false; + + /** + * Test configuration name. + */ + private final String name; + + /** + * Does this test configuration keep markers? + */ + private final boolean keepMarkers; + + /** + * Is this FS deleting markers? + */ + 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; + + /** + * 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, + DIRECTORY_MARKER_POLICY); + conf.set(DIRECTORY_MARKER_POLICY, + keepMarkers + ? DIRECTORY_MARKER_POLICY_KEEP + : DIRECTORY_MARKER_POLICY_DELETE); + return conf; + } + + /** + * The setup phase includes creating the test objects. + */ + @Override + public void setup() throws Exception { + super.setup(); + S3AFileSystem fs = getFileSystem(); + assume("unguarded FS only", + !fs.hasMetadataStore()); + s3client = fs.getAmazonS3ClientForTesting("markers"); + + bucket = fs.getBucket(); + Path base = new Path(methodPath(), "base"); + + createTestObjects(base); + } + + /** + * Teardown deletes the objects created before + * the superclass does the directory cleanup. + */ + @Override + public void teardown() throws Exception { + if (s3client != null) { + deleteObject(markerKey); + deleteObject(markerKeySlash); + deleteObject(markerPeerKey); + deleteObject(fileKeyUnderMarker); + } + // do this ourselves to avoid audits teardown failing + // when surplus markers are found + deleteTestDirInTeardown(); + super.teardown(); + } + + /** + * 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, MARKER_PEER); + markerPeerKey = fs.pathToKey(markerPeer); + markerKey = fs.pathToKey(markerDir); + markerKeySlash = markerKey + "/"; + 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 testMarkerExists() throws Throwable { + 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); + head(fileKeyUnderMarker); + } + + /* + ================================================================= + The listing operations + ================================================================= + */ + + @Test + 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 testListFilesMarkerDirRecursive() throws Throwable { + List statuses = toList( + getFileSystem().listFiles(markerDir, true)); + assertContainsFileUnderMarkerOnly(statuses); + } + + /** + * Path listing above the base dir MUST only find the file + * and not the marker. + */ + @Test + public void testListStatusBaseDirRecursive() throws Throwable { + List statuses = toList( + getFileSystem().listFiles(basePath, true)); + assertContainsExactlyStatusOfPaths(statuses, filePathUnderMarker, + markerPeer); + } + + @Test + public void testGlobStatusBaseDirRecursive() throws Throwable { + Path escapedPath = new Path(escape(basePath.toUri().getPath())); + List statuses = + exec("glob", () -> + 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(escapedPath, "*")))); + assertContainsFileUnderMarkerOnly(statuses); + } + + /** + * 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 testListLocatedStatusBaseDir() throws Throwable { + List statuses = + exec("listLocatedStatus", () -> + toList(getFileSystem().listLocatedStatus(basePath))); + + assertContainsExactlyStatusOfPaths(statuses, markerPeer, markerDir); + } + + /** + * Call {@code listLocatedStatus(markerDir)}; expect + * the file entry only. + */ + @Test + public void testListLocatedStatusMarkerDir() throws Throwable { + List statuses = + exec("listLocatedStatus", () -> + toList(getFileSystem().listLocatedStatus(markerDir))); + + assertContainsFileUnderMarkerOnly(statuses); + } + + + /* + ================================================================= + Creation Rejection + ================================================================= + */ + + @Test + public void testCreateNoOverwriteMarkerDir() throws Throwable { + describe("create no-overwrite over the marker dir fails"); + head(markerKeySlash); + intercept(FileAlreadyExistsException.class, () -> + exec("create", () -> + getFileSystem().create(markerDir, false))); + // dir is still there. + head(markerKeySlash); + } + + @Test + public void testCreateNoOverwriteFile() throws Throwable { + describe("create-no-overwrite on the file fails"); + + head(fileKeyUnderMarker); + intercept(FileAlreadyExistsException.class, () -> + exec("create", () -> + getFileSystem().create(filePathUnderMarker, false))); + assertTestObjectsExist(); + } + + @Test + public void testCreateFileNoOverwrite() throws Throwable { + describe("verify the createFile() API also fails"); + head(fileKeyUnderMarker); + intercept(FileAlreadyExistsException.class, () -> + exec("create", () -> + getFileSystem().createFile(filePathUnderMarker) + .overwrite(false) + .build())); + 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. + ================================================================= + */ + + /** + * 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 testRenameBase() throws Throwable { + describe("rename base directory"); + + 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 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 testRenameUnderMarkerDir() throws Throwable { + describe("directory rename under an existing marker"); + String file = "sourceFile"; + Path srcDir = new Path(basePath, "srcdir"); + mkdirs(srcDir); + Path src = new Path(srcDir, file); + String srcKey = toKey(src); + 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, file)); + 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 file = "sourceFile"; + Path srcDir = new Path(basePath, "srcdir"); + mkdirs(srcDir); + Path src = new Path(srcDir, file); + String srcKey = toKey(src); + put(srcKey, file); + 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 { + head(markerKeySlash); + } + } + + /** + * 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 testRenameEmptyDirOverMarker() throws Throwable { + describe("rename an empty directory over the marker"); + S3AFileSystem fs = getFileSystem(); + String dir = "sourceDir"; + Path src = new Path(basePath, dir); + 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(); + } + + /* + ================================================================= + Utility methods and assertions. + ================================================================= + */ + + /** + * Assert the test objects exist. + */ + 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)); + } + /** + * Delete an object. + * @param key key + * @param content string + */ + private void deleteObject(final String key) throws Exception { + exec("DELETE " + key, () -> { + s3client.deleteObject(bucket, key); + return "deleted " + key; + }); + } + + /** + * Issue a HEAD request. + * @param key + * @return a description of the object. + */ + 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()); + } + + /** + * 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); + } + } + + /** + * 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 objects to match that of the paths. + * @param statuses status object list + * @param paths ordered varargs list of paths + * @param type of status objects + */ + private void assertContainsExactlyStatusOfPaths( + List statuses, Path... paths) { + + 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++) { + assertEquals("Path mismatch at element " + i + " in " + summary, + paths[i], statuses.get(i).getPath()); + } + } + + /** + * 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); + } + + /** + * 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); + } + + /** + * 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()); + } + + /** + * 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); + } + + /** + * 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) { + LOG.info("{}\t{}", c++, t); + } + 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 + ") failed", + getFileSystem().rename(src, dest)); + } + + /** + * 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 new file mode 100644 index 0000000000000..d3d976e928940 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java @@ -0,0 +1,218 @@ +/* + * 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.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.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. + */ +@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); + } + + @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. + * The parent dir must be found and declared as empty. + *

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

+ */ + @Test + public void testDeleteSingleFileInDir() throws Throwable { + describe("delete a file"); + 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(); + }, + 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), + withWhenRaw(OBJECT_LIST_REQUESTS, + FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L), + with(DIRECTORIES_DELETED, 0), + with(FILES_DELETED, 1), + + // keeping: create no parent dirs or delete parents + withWhenKeeping(DIRECTORIES_CREATED, 0), + withWhenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + + // deleting: create a parent and delete any of its parents + withWhenDeleting(DIRECTORIES_CREATED, 1), + withWhenDeleting(OBJECT_DELETE_REQUESTS, + DELETE_OBJECT_REQUEST + + DELETE_MARKER_REQUEST) + ); + // there is an empty dir for a parent + S3AFileStatus status = verifyRawInnerGetFileStatus(dir, true, + 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), + withWhenRaw(OBJECT_LIST_REQUESTS, + FILESTATUS_FILE_PROBE_L + FILESTATUS_DIR_PROBE_L), + with(DIRECTORIES_DELETED, 0), + with(FILES_DELETED, 1), + + // no need to create a parent + with(DIRECTORIES_CREATED, 0), + + // keeping: create no parent dirs or delete parents + withWhenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + + // deleting: create a parent and delete any of its parents + withWhenDeleting(OBJECT_DELETE_REQUESTS, + DELETE_OBJECT_REQUEST)); + } + + @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(); + }, + 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 + withWhenDeleting(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(); + }, + with(DIRECTORIES_CREATED, 0), + with(DIRECTORIES_DELETED, 0), + // keeping: no delete operations. + withWhenKeeping(OBJECT_DELETE_REQUESTS, 0), + withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0), + // delete all possible fake dirs above the file + 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 new file mode 100644 index 0000000000000..85c70768356e6 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ARenameCost.java @@ -0,0 +1,207 @@ +/* + * 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.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.performance.OperationCost.*; + +/** + * 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), + whenRaw(RENAME_SINGLE_FILE_DIFFERENT_DIR), + with(DIRECTORIES_CREATED, 0), + with(DIRECTORIES_DELETED, 0), + // keeping: only the core delete operation is issued. + withWhenKeeping(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0), + // deleting: delete any fake marker above the destination. + withWhenDeleting(OBJECT_DELETE_REQUESTS, + DELETE_OBJECT_REQUEST + DELETE_MARKER_REQUEST), + withWhenDeleting(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 the same directory"); + + 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), + whenRaw(RENAME_SINGLE_FILE_SAME_DIR), + with(OBJECT_COPY_REQUESTS, 1), + with(DIRECTORIES_CREATED, 0), + with(OBJECT_DELETE_REQUESTS, DELETE_OBJECT_REQUEST), + with(FAKE_DIRECTORIES_DELETED, 0)); + } + + @Test + 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(); + + // 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(); + }, + whenRaw(FILE_STATUS_FILE_PROBE + .plus(GET_FILE_STATUS_FNFE) + .plus(COPY_OP)), + // here we expect there to be no fake directories + with(DIRECTORIES_CREATED, 0), + // one for the renamed file only + with(OBJECT_DELETE_REQUESTS, + DELETE_OBJECT_REQUEST), + // no directories are deleted: This is root + with(DIRECTORIES_DELETED, 0), + // no fake directories are deleted: This is root + with(FAKE_DIRECTORIES_DELETED, 0), + with(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(src, false); + return "after fs.delete(/dest) " + getMetricSummary(); + }, + 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 { + fs.delete(src, 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..46a6b712c49bf --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java @@ -0,0 +1,201 @@ +/* + * 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 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; + + /** 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; + + /** + * 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); + + /** 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; + + /** 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); + + /** + * 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); + + /** + * 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 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(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 and the copy. + */ + public static final OperationCost RENAME_SINGLE_FILE_SAME_DIR = + 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. + */ + 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 new file mode 100644 index 0000000000000..c351d1b185a32 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java @@ -0,0 +1,483 @@ +/* + * 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.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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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.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; + +/** + * 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 final class OperationCostValidator { + + private static final Logger LOG = + LoggerFactory.getLogger(OperationCostValidator.class); + + /** + * The empty probe: declared as disabled. + */ + 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<>(); + + /** + * 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.filesystem, 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 exec( + Callable eval, + ExpectedProbe... expectedA) throws Exception { + List expected = Arrays.asList(expectedA); + 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 = + "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); + } + 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(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 are not enabled in %s", pstr) + .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 intercepting( + Class clazz, + String text, + Callable eval, + ExpectedProbe... expected) throws Exception { + + return exec(() -> + 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 { + + /** + * Filesystem. + */ + private final S3AFileSystem filesystem; + + /** + * Metrics to create. + */ + private final List metrics = new ArrayList<>(); + + + /** + * 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 enabled whenever the expected + * value is greater than zero. + * @param statistic statistic to check. + * @param expected expected value. + * @return a probe. + */ + public static ExpectedProbe probe( + final Statistic statistic, + final int expected) { + return probe(expected >= 0, 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 enabled + ? new ProbeList(Arrays.asList(plist)) + : 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. + *

+ * 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; + } + + @Override + public String toString() { + String sb = "ExpectSingleStatistic{" + + statistic + + ", expected=" + expected + + ", enabled=" + isEnabled() + + '}'; + return sb; + } + } + + /** + * A list of probes; the verify operation + * 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; + } + + @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; + } + + @Override + public String toString() { + String pstr = probes.stream() + .map(Object::toString) + .collect(Collectors.joining(", ")); + 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; + } + } +} 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..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 @@ -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)); + intercept(ExitUtil.ExitException.class, () -> + 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/ITestS3GuardDDBRootOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardDDBRootOperations.java index b2e6b3e93a8b3..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 @@ -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; @@ -52,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 { @@ -82,9 +87,15 @@ protected int getTestTimeoutMillis() { protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); String bucketName = getTestBucketName(conf); + disableFilesystemCaching(conf); + 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. - removeBucketOverrides(bucketName, conf, ENABLE_MULTI_DELETE); conf.setTimeDuration( S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY, 0, 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/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/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..00e62d9491070 --- /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 new file mode 100644 index 0000000000000..4a81b1aba919b --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerTool.java @@ -0,0 +1,533 @@ +/* + * 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.IOException; +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.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.S3AFileSystem; + +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; + +/** + * Test the marker tool and use it to compare the behavior + * of keeping vs legacy S3A FS instances. + */ +public class ITestMarkerTool extends AbstractMarkerToolTest { + + protected static final Logger LOG = + LoggerFactory.getLogger(ITestMarkerTool.class); + + /** + * 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; + + + @Test + 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 testCleanMarkersFileLimit() throws Throwable { + describe("Clean markers under a keeping FS -with file limit"); + CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); + + // audit will be interrupted + markerTool(EXIT_INTERRUPTED, getDeletingFS(), + createdPaths.base, false, 0, 1, false); + } + + @Test + 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, UNLIMITED_LISTING, false); + + LOG.info("Auditing a directory expecting retained markers"); + markerTool(fs, createdPaths.base, false, + expectedMarkerCount); + + // 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)); + } + + @Test + public void testRenameKeepingFS() throws Throwable { + 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"); + + 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); + LOG.info("Auditing destination paths"); + verifyRenamed(dest, createdPaths); + } + + /** + * 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 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, 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); + assertIsDirectory(dest); + + // there are no markers + MarkerTool.ScanResult scanResult = markerTool(mixedFSDir2, dest, false, 0); + // there are exactly the files we want + Assertions.assertThat(scanResult) + .describedAs("Scan result %s", scanResult) + .extracting(s -> s.getTracker().getFilesFound()) + .isEqualTo(expectedFileCount); + 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); + } + + /** + * Marker tool with no args. + */ + @Test + public void testRunNoArgs() throws Throwable { + runToFailure(EXIT_USAGE, MARKERS); + } + + @Test + public void testRunWrongBucket() throws Throwable { + runToFailure(EXIT_NOT_FOUND, MARKERS, + AUDIT, + "s3a://this-bucket-does-not-exist-hopefully"); + } + + /** + * Run with a path that doesn't exist. + */ + @Test + public void testRunUnknownPath() throws Throwable { + runToFailure(EXIT_NOT_FOUND, MARKERS, + AUDIT, + methodPath()); + } + + /** + * Having both -audit and -clean on the command line is an error. + */ + @Test + public void testRunTooManyActions() throws Throwable { + runToFailure(EXIT_USAGE, MARKERS, + 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()); + final File audit = tempAuditFile(); + run(MARKERS, V, + AUDIT, + 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, MARKERS, V, + AUDIT, + m(OPT_OUT), audit, + createdPaths.base); + expectMarkersInOutput(audit, expectedMarkersWithBaseDir); + } + + @Test + public void testRunLimitedAudit() throws Throwable { + describe("Audit with a limited number of files (2)"); + CreatedPaths createdPaths = createPaths(getKeepingFS(), methodPath()); + runToFailure(EXIT_INTERRUPTED, + MARKERS, V, + m(OPT_LIMIT), 2, + CLEAN, + createdPaths.base); + run(MARKERS, V, + AUDIT, + createdPaths.base); + } + + /** + * Run an audit against the landsat bucket. + *

+ * This tests paging/scale against a larger bucket without + * worrying about setup costs. + */ + @Test + public void testRunLimitedLandsatAudit() throws Throwable { + describe("Audit a few thousand landsat objects"); + final File audit = tempAuditFile(); + + run(MARKERS, + AUDIT, + m(OPT_LIMIT), 3000, + m(OPT_OUT), audit, + LANDSAT_BUCKET); + readOutput(audit); + } + + @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()); + } + + @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()); + } + + @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()); + } + + @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()); + } + + /** + * 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; + expectedMarkersWithBaseDir = expectedMarkers + 1; + return r; + } + + /** + * 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)); + } + } + +} 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); + } + +}