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 2383da9c104c7..c4577cc253784 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -51,6 +51,7 @@ 200000 + @@ -122,6 +123,7 @@ ${fs.s3a.scale.test.huge.filesize} ${fs.s3a.scale.test.huge.partitionsize} ${fs.s3a.scale.test.timeout} + ${fs.s3a.directory.marker} @@ -162,6 +164,7 @@ ${fs.s3a.s3guard.test.enabled} ${fs.s3a.s3guard.test.authoritative} ${fs.s3a.s3guard.test.implementation} + ${fs.s3a.directory.marker} ${test.integration.timeout} @@ -214,6 +217,7 @@ ${fs.s3a.s3guard.test.enabled} ${fs.s3a.s3guard.test.implementation} ${fs.s3a.s3guard.test.authoritative} + ${fs.s3a.directory.marker} @@ -268,6 +272,7 @@ ${fs.s3a.s3guard.test.enabled} ${fs.s3a.s3guard.test.implementation} ${fs.s3a.s3guard.test.authoritative} + ${fs.s3a.directory.marker} ${fs.s3a.scale.test.timeout} @@ -331,6 +336,44 @@ + + + keep-markers + + + markers + keep + + + + keep + + + + delete-markers + + + markers + delete + + + + delete + + + + auth-markers + + + markers + authoritative + + + + authoritative + + + diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index 22a0b45f1c7a5..a738fe2465880 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -953,4 +953,40 @@ private Constants() { * Value: {@value} seconds. */ public static final int THREAD_POOL_SHUTDOWN_DELAY_SECONDS = 30; + + /** + * Policy for directory markers. + * This is a new feature of HADOOP-13230 which addresses + * some scale, performance and permissions issues -but + * at the risk of backwards compatibility. + */ + public static final String DIRECTORY_MARKER_POLICY = + "fs.s3a.directory.markers"; + + /** + * Retain directory markers. + * Value: {@value}. + */ + public static final String DIRECTORY_MARKER_POLICY_KEEP = + "keep"; + + /** + * Delete directory markers. This is the backwards compatible option. + * Value: {@value}. + */ + public static final String DIRECTORY_MARKER_POLICY_DELETE = + "delete"; + + /** + * Retain directory markers in authoritative directory trees only. + * Value: {@value}. + */ + public static final String DIRECTORY_MARKER_POLICY_AUTHORITATIVE = + "authoritative"; + + /** + * Default retention policy. + */ + public static final String DEFAULT_DIRECTORY_MARKER_POLICY = + DIRECTORY_MARKER_POLICY_DELETE; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Listing.java index 9c2f67dd884ec..32c5a8306f13c 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 @@ -125,12 +125,27 @@ public FileStatusListingIterator createFileStatusListingIterator( Listing.FileStatusAcceptor acceptor, RemoteIterator providedStatus) throws IOException { return new FileStatusListingIterator( - new ObjectListingIterator(listPath, request), + createObjectListingIterator(listPath, request), filter, acceptor, providedStatus); } + /** + * Create an object listing iterator against a path, with a given + * list object request. + * @param listPath path of the listing + * @param request initial request to make + * @return the iterator + * @throws IOException IO Problems + */ + @Retries.RetryRaw + public ObjectListingIterator createObjectListingIterator( + final Path listPath, + final S3ListRequest request) throws IOException { + return new ObjectListingIterator(listPath, request); + } + /** * Create a located status iterator over a file status iterator. * @param statusIterator an iterator over the remote status entries diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index e5b08f1bb903e..cd7d8ba955b82 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -67,7 +67,6 @@ import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.PutObjectRequest; import com.amazonaws.services.s3.model.PutObjectResult; -import com.amazonaws.services.s3.model.S3ObjectSummary; import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; import com.amazonaws.services.s3.model.SSECustomerKey; import com.amazonaws.services.s3.model.UploadPartRequest; @@ -104,6 +103,8 @@ import org.apache.hadoop.fs.s3a.impl.ContextAccessors; import org.apache.hadoop.fs.s3a.impl.CopyOutcome; import org.apache.hadoop.fs.s3a.impl.DeleteOperation; +import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; +import org.apache.hadoop.fs.s3a.impl.DirectoryPolicyImpl; import org.apache.hadoop.fs.s3a.impl.InternalConstants; import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport; import org.apache.hadoop.fs.s3a.impl.OperationCallbacks; @@ -293,6 +294,16 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private final S3AFileSystem.OperationCallbacksImpl operationCallbacks = new OperationCallbacksImpl(); + /** + * Directory policy. + */ + private DirectoryPolicy directoryPolicy; + + /** + * Context accessors for re-use. + */ + private final ContextAccessors contextAccessors = new ContextAccessorsImpl(); + /** Add any deprecated keys. */ @SuppressWarnings("deprecation") private static void addDeprecatedKeys() { @@ -450,6 +461,12 @@ public void initialize(URI name, Configuration originalConf) DEFAULT_S3GUARD_DISABLED_WARN_LEVEL); S3Guard.logS3GuardDisabled(LOG, warnLevel, bucket); } + // directory policy, which will look at authoritative paths + // if needed + directoryPolicy = new DirectoryPolicyImpl(conf, + this::allowAuthoritative); + LOG.debug("Directory marker retention policy is {}", + directoryPolicy); initMultipartUploads(conf); @@ -1274,7 +1291,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, @@ -1282,10 +1299,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); } } @@ -1420,10 +1449,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"); @@ -1524,11 +1556,22 @@ public void deleteObjectAtPath(final Path path, final boolean isFile, final BulkOperationState operationState) throws IOException { - once("delete", key, () -> + once("delete", path.toString(), () -> S3AFileSystem.this.deleteObjectAtPath(path, key, isFile, operationState)); } + @Override + @Retries.RetryTranslated + public void deleteDirectoryMarkers(final Path path, + final String key, + final BulkOperationState operationState) + throws IOException { + if (!keepDirectoryMarkers(path)) { + deleteUnnecessaryFakeDirectories(path, operationState); + } + } + @Override @Retries.RetryTranslated public RemoteIterator listFilesAndEmptyDirectories( @@ -1574,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(destCreated)) { + deleteUnnecessaryFakeDirectories(destParent, null); + } maybeCreateFakeParentDirectory(sourceRenamed); } } @@ -1874,6 +1919,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")) { @@ -2507,7 +2553,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); } @@ -2523,7 +2569,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); } } @@ -2552,7 +2598,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); @@ -2560,7 +2606,9 @@ public FileStatus[] innerListStatus(Path f) throws FileNotFoundException, entryPoint(INVOCATION_LIST_STATUS); List result; - final FileStatus fileStatus = getFileStatus(path); + entryPoint(INVOCATION_GET_FILE_STATUS); + final S3AFileStatus fileStatus = innerGetFileStatus(path, false, + StatusProbeEnum.ALL); if (fileStatus.isDirectory()) { if (!key.isEmpty()) { @@ -2592,7 +2640,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; } @@ -2703,9 +2751,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); } } @@ -2725,11 +2774,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; @@ -2739,7 +2792,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()) { @@ -2800,7 +2853,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. @@ -2811,9 +2865,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); @@ -2824,72 +2879,96 @@ S3AFileStatus innerGetFileStatus(final Path f, // Skip going to s3 if the file checked is a directory. Because if the // dest is also a directory, there's no difference. - if (!pm.getFileStatus().isDirectory() && + if (!msStatus.isDirectory() && !allowAuthoritative && probes.contains(StatusProbeEnum.Head)) { // a file has been found in a non-auth path and the caller has not said // they only care about directories LOG.debug("Metadata for {} found in the non-auth metastore.", path); - final long msModTime = pm.getFileStatus().getModificationTime(); - - S3AFileStatus s3AFileStatus; - try { - s3AFileStatus = s3GetFileStatus(path, key, probes, tombstones); - } catch (FileNotFoundException fne) { - s3AFileStatus = null; - } - if (s3AFileStatus == null) { - LOG.warn("Failed to find file {}. Either it is not yet visible, or " - + "it has been deleted.", path); - } else { - final long s3ModTime = s3AFileStatus.getModificationTime(); - - if(s3ModTime > msModTime) { - LOG.debug("S3Guard metadata for {} is outdated;" - + " s3modtime={}; msModTime={} updating metastore", - path, s3ModTime, msModTime); - return S3Guard.putAndReturn(metadataStore, s3AFileStatus, - ttlTimeProvider); + // TODO: if the timestamp of the pm is close to "now", we don't need to + // TODO: bother with a check of S3. that means: + // TODO: one of : status modtime is close to now, + // TODO: or pm.getLastUpdated() == now + long validTime = ttlTimeProvider.getNow() - ttlTimeProvider.getMetadataTtl(); + final long msModTime = msStatus.getModificationTime(); + + if (msModTime < validTime) { + LOG.debug("Metastore entry is out of date, probing S3"); + try { + S3AFileStatus s3AFileStatus = s3GetFileStatus(path, key, probes, + tombstones, needEmptyDirectoryFlag); + final long s3ModTime = s3AFileStatus.getModificationTime(); + + if (s3ModTime > msModTime) { + // there's new data in S3 + LOG.debug("S3Guard metadata for {} is outdated;" + + " s3modtime={}; msModTime={} updating metastore", + path, s3ModTime, msModTime); + // add to S3Guard and return the value + // note that the checks for empty dir status below can be skipped + // because the call to s3GetFileStatus include the checks there + return S3Guard.putAndReturn(metadataStore, s3AFileStatus, + ttlTimeProvider); + } else { + // the modtime of the data is the same as/older than the s3guard + // value either an old object has been found, or the existing one + // was retrieved in both cases -refresh the S3Guard entry so the + // record's TTL is updated. + S3Guard.refreshEntry(metadataStore, pm, s3AFileStatus, + ttlTimeProvider); + } + } catch (FileNotFoundException fne) { + // the attempt to refresh the record failed because there was + // no entry. Either it is a new file not visible, or it + // has been deleted (and therefore S3Guard is out of sync with S3) + LOG.warn("Failed to find file {}. Either it is not yet visible, or " + + "it has been deleted.", path); } } } - S3AFileStatus msStatus = pm.getFileStatus(); if (needEmptyDirectoryFlag && msStatus.isDirectory()) { + // the caller needs to know if a directory is empty, + // and that this is a directory. if (pm.isEmptyDirectory() != Tristate.UNKNOWN) { // We have a definitive true / false from MetadataStore, we are done. return msStatus; } else { + // execute a S3Guard listChildren command to list tombstones under the + // path. + // This list will be used in the forthcoming s3GetFileStatus call. DirListingMetadata children = S3Guard.listChildrenWithTtl(metadataStore, path, ttlTimeProvider, allowAuthoritative); if (children != null) { tombstones = children.listTombstones(); } - LOG.debug("MetadataStore doesn't know if dir is empty, using S3."); + LOG.debug("MetadataStore doesn't know if {} is empty, using S3.", + path); } } else { // Either this is not a directory, or we don't care if it is empty return msStatus; } - // If the metadata store has no children for it and it's not listed in - // S3 yet, we'll assume the empty directory is true; - S3AFileStatus s3FileStatus; + // now issue the S3 getFileStatus call. try { - s3FileStatus = s3GetFileStatus(path, key, probes, tombstones); + S3AFileStatus s3FileStatus = s3GetFileStatus(path, key, probes, tombstones, + true); + // entry was found, so save in S3Guard and return the final value. + return S3Guard.putAndReturn(metadataStore, s3FileStatus, + ttlTimeProvider); } catch (FileNotFoundException e) { + // If the metadata store has no children for it and it's not listed in + // S3 yet, we'll conclude that it is an empty directory return S3AFileStatus.fromFileStatus(msStatus, Tristate.TRUE, null, null); } - // entry was found, save in S3Guard - return S3Guard.putAndReturn(metadataStore, s3FileStatus, - ttlTimeProvider); } else { // there was no entry in S3Guard // retrieve the data and update the metadata store in the process. return S3Guard.putAndReturn(metadataStore, - s3GetFileStatus(path, key, probes, tombstones), + s3GetFileStatus(path, key, probes, tombstones, needEmptyDirectoryFlag), ttlTimeProvider); } } @@ -2944,6 +3023,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. @@ -2953,88 +3034,85 @@ S3AFileStatus innerGetFileStatus(final Path f, S3AFileStatus s3GetFileStatus(final Path path, final String key, final Set probes, - @Nullable Set tombstones) throws IOException { - if (!key.isEmpty()) { - if (probes.contains(StatusProbeEnum.Head) && !key.endsWith("/")) { - try { - // look for the simple file - ObjectMetadata meta = getObjectMetadata(key); - LOG.debug("Found exact file: normal file {}", key); - return new S3AFileStatus(meta.getContentLength(), - dateToLong(meta.getLastModified()), - path, - getDefaultBlockSize(path), - username, - meta.getETag(), - meta.getVersionId()); - } catch (AmazonServiceException e) { - // if the response is a 404 error, it just means that there is - // no file at that path...the remaining checks will be needed. - if (e.getStatusCode() != SC_404 || isUnknownBucket(e)) { - throw translateException("getFileStatus", path, e); - } - } catch (AmazonClientException e) { + @Nullable final Set tombstones, + final boolean needEmptyDirectoryFlag) throws IOException { + LOG.debug("S3GetFileStatus {}", path); + Preconditions.checkArgument(!needEmptyDirectoryFlag + || probes.contains(StatusProbeEnum.List), + "s3GetFileStatus(%s) wants to know if a directory is empty but" + + " does not request a list probe", path); + + if (!key.isEmpty() && !key.endsWith("/") + && probes.contains(StatusProbeEnum.Head)) { + try { + // look for the simple file + ObjectMetadata meta = getObjectMetadata(key); + LOG.debug("Found exact file: normal file {}", key); + return new S3AFileStatus(meta.getContentLength(), + dateToLong(meta.getLastModified()), + path, + getDefaultBlockSize(path), + username, + meta.getETag(), + meta.getVersionId()); + } catch (AmazonServiceException e) { + // if the response is a 404 error, it just means that there is + // no file at that path...the remaining checks will be needed. + if (e.getStatusCode() != SC_404 || isUnknownBucket(e)) { throw translateException("getFileStatus", path, e); } - } - - // Either a normal file was not found or the probe was skipped. - // because the key ended in "/" or it was not in the set of probes. - // Look for the dir marker - if (probes.contains(StatusProbeEnum.DirMarker)) { - String newKey = maybeAddTrailingSlash(key); - try { - ObjectMetadata meta = getObjectMetadata(newKey); - - if (objectRepresentsDirectory(newKey, meta.getContentLength())) { - LOG.debug("Found file (with /): fake directory"); - return new S3AFileStatus(Tristate.TRUE, path, username); - } else { - LOG.warn("Found file (with /): real file? should not happen: {}", - key); - - return new S3AFileStatus(meta.getContentLength(), - dateToLong(meta.getLastModified()), - path, - getDefaultBlockSize(path), - username, - meta.getETag(), - meta.getVersionId()); - } - } catch (AmazonServiceException e) { - if (e.getStatusCode() != SC_404 || isUnknownBucket(e)) { - throw translateException("getFileStatus", newKey, e); - } - } catch (AmazonClientException e) { - throw translateException("getFileStatus", newKey, e); - } + } catch (AmazonClientException e) { + throw translateException("getFileStatus", path, e); } } // execute the list if (probes.contains(StatusProbeEnum.List)) { try { + // this will find a marker dir / as well as an entry. + // When making a simple "is this a dir check" all is good. + // but when looking for an empty dir, we need to verify there are no + // children, so ask for two entries, so as to find + // a child String dirKey = maybeAddTrailingSlash(key); - S3ListRequest request = createListObjectsRequest(dirKey, "/", 1); + // list size is dir marker + at least one non-tombstone entry + // there's a corner case: more tombstones than you have in a + // single page list. We assume that if you have been deleting + // that many files, then the AWS listing will have purged some + // by the time of listing so that the response includes some + // which have not. + + int listSize; + if (tombstones == null) { + // no tombstones so look for a marker and at least one child. + listSize = 2; + } else { + // build a listing > tombstones. If the caller has many thousands + // of tombstones this won't work properly, which is why pruning + // of expired tombstones matters. + listSize = Math.min(2 + tombstones.size(), Math.max(2, maxKeys)); + } + S3ListRequest request = createListObjectsRequest(dirKey, "/", + listSize); + // execute the request + S3ListResult listResult = listObjects(request); - S3ListResult objects = listObjects(request); - Collection prefixes = objects.getCommonPrefixes(); - Collection summaries = objects.getObjectSummaries(); - if (!isEmptyOfKeys(prefixes, tombstones) || - !isEmptyOfObjects(summaries, tombstones)) { + if (listResult.hasPrefixesOrObjects(contextAccessors, tombstones)) { if (LOG.isDebugEnabled()) { - LOG.debug("Found path as directory (with /): {}/{}", - prefixes.size(), summaries.size()); - - for (S3ObjectSummary summary : summaries) { - LOG.debug("Summary: {} {}", summary.getKey(), summary.getSize()); - } - for (String prefix : prefixes) { - LOG.debug("Prefix: {}", prefix); - } + LOG.debug("Found path as directory (with /)"); + listResult.logAtDebug(LOG); } - + // At least one entry has been found. + // If looking for an empty directory, the marker must exist but no children. + // So the listing must contain the marker entry only. + if (needEmptyDirectoryFlag + && listResult.representsEmptyDirectory( + contextAccessors, dirKey, tombstones)) { + return new S3AFileStatus(Tristate.TRUE, path, username); + } + // either an empty directory is not needed, or the + // listing does not meet the requirements. return new S3AFileStatus(Tristate.FALSE, path, username); } else if (key.isEmpty()) { LOG.debug("Found root directory"); @@ -3053,48 +3131,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. @@ -3109,7 +3145,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; @@ -3512,6 +3548,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); @@ -3645,14 +3682,14 @@ private Optional generateSSECustomerKey() { /** * Perform post-write actions. - * Calls {@link #deleteUnnecessaryFakeDirectories(Path)} and then + * Calls {@link #deleteUnnecessaryFakeDirectories(Path, BulkOperationState)} and then * updates any metastore. * This operation MUST be called after any PUT/multipart PUT completes * successfully. * * The operations actions include *
    - *
  1. Calling {@link #deleteUnnecessaryFakeDirectories(Path)}
  2. + *
  3. Calling {@link #deleteUnnecessaryFakeDirectories(Path, BulkOperationState)}
  4. *
  5. Updating any metadata store with details on the newly created * object.
  6. *
@@ -3677,12 +3714,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; @@ -3741,13 +3785,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); @@ -3757,7 +3814,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()) { @@ -4018,25 +4075,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; + } } /** @@ -4769,6 +4842,15 @@ public StoreContext createStoreContext() { .build(); } + /** + * Get the operation callbacks for this FS. + * @return callbacks for operations. + */ + @InterfaceAudience.Private + public OperationCallbacks getOperationCallbacks() { + return operationCallbacks; + } + /** * The implementation of context accessors. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java index 1a0d2c3378ca6..abce86d410272 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListRequest.java @@ -24,11 +24,15 @@ /** * API version-independent container for S3 List requests. */ -public class S3ListRequest { - private ListObjectsRequest v1Request; - private ListObjectsV2Request v2Request; +public final class S3ListRequest { - protected S3ListRequest(ListObjectsRequest v1, ListObjectsV2Request v2) { + private static final String DESCRIPTION + = "List %s:/%s delimiter=%s keys=%d requester pays=%s"; + + private final ListObjectsRequest v1Request; + private final ListObjectsV2Request v2Request; + + private S3ListRequest(ListObjectsRequest v1, ListObjectsV2Request v2) { v1Request = v1; v2Request = v2; } @@ -70,11 +74,15 @@ public ListObjectsV2Request getV2() { @Override public String toString() { if (isV1()) { - return String.format("List %s:/%s", - v1Request.getBucketName(), v1Request.getPrefix()); + return String.format(DESCRIPTION, + v1Request.getBucketName(), v1Request.getPrefix(), + v1Request.getDelimiter(), v1Request.getMaxKeys(), + v1Request.isRequesterPays()); } else { - return String.format("List %s:/%s", - v2Request.getBucketName(), v2Request.getPrefix()); + return String.format(DESCRIPTION, + v2Request.getBucketName(), v2Request.getPrefix(), + v2Request.getDelimiter(), v2Request.getMaxKeys(), + v2Request.isRequesterPays()); } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java index e8aff329070ef..551a8c4375151 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ListResult.java @@ -18,11 +18,18 @@ package org.apache.hadoop.fs.s3a; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + import com.amazonaws.services.s3.model.ListObjectsV2Result; import com.amazonaws.services.s3.model.ObjectListing; import com.amazonaws.services.s3.model.S3ObjectSummary; +import org.slf4j.Logger; -import java.util.List; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.impl.ContextAccessors; /** * API version-independent container for S3 List responses. @@ -92,6 +99,109 @@ public List getCommonPrefixes() { } else { return v2Result.getCommonPrefixes(); } + } + + /** + * Is the list of object summaries empty + * after accounting for tombstone markers (if provided)? + * @param accessors callback for key to path mapping. + * @param tombstones Set of tombstone markers, or null if not applicable. + * @return false if summaries contains objects not accounted for by + * tombstones. + */ + public boolean isEmptyOfObjects( + final ContextAccessors accessors, + final Set tombstones) { + if (tombstones == null) { + return getObjectSummaries().isEmpty(); + } + return isEmptyOfKeys(accessors, + objectSummaryKeys(), + tombstones); + } + + /** + * Get the list of keys in the object summary. + * @return a possibly empty list + */ + private List objectSummaryKeys() { + return getObjectSummaries().stream() + .map(S3ObjectSummary::getKey) + .collect(Collectors.toList()); + } + + /** + * Does this listing have prefixes or objects? + * @param accessors callback for key to path mapping. + * @param tombstones Set of tombstone markers, or null if not applicable. + * @return true if the reconciled list is non-empty + */ + public boolean hasPrefixesOrObjects( + final ContextAccessors accessors, + final Set tombstones) { + + return !isEmptyOfKeys(accessors, getCommonPrefixes(), tombstones) + || !isEmptyOfObjects(accessors, tombstones); + } + + /** + * Helper function to determine if a collection of keys is empty + * after accounting for tombstone markers (if provided). + * @param accessors callback for key to path mapping. + * @param keys Collection of path (prefixes / directories or keys). + * @param tombstones Set of tombstone markers, or null if not applicable. + * @return true if the list is considered empty. + */ + public boolean isEmptyOfKeys( + final ContextAccessors accessors, + final Collection keys, + final Set tombstones) { + if (tombstones == null) { + return keys.isEmpty(); + } + for (String key : keys) { + Path qualified = accessors.keyToPath(key); + if (!tombstones.contains(qualified)) { + return false; + } + } + return true; + } + /** + * Does this listing represent an empty directory? + * @param contextAccessors callback for key to path mapping. + * @param dirKey directory key + * @param tombstones Set of tombstone markers, or null if not applicable. + * @return true if the list is considered empty. + */ + public boolean representsEmptyDirectory( + final ContextAccessors contextAccessors, + final String dirKey, + final Set tombstones) { + // If looking for an empty directory, the marker must exist but + // no children. + // So the listing must contain the marker entry only as an object, + // and prefixes is null + List keys = objectSummaryKeys(); + return keys.size() == 1 && keys.contains(dirKey) + && getCommonPrefixes().isEmpty(); + } + + /** + * dump the result at debug level only. + * @param log log to use + */ + public void logAtDebug(Logger log) { + Collection prefixes = getCommonPrefixes(); + Collection summaries = getObjectSummaries(); + log.debug("Prefix count = {}; object count={}", + prefixes.size(), summaries.size()); + for (S3ObjectSummary summary : summaries) { + log.debug("Summary: {} {}", summary.getKey(), summary.getSize()); + } + for (String prefix : prefixes) { + log.debug("Prefix: {}", prefix); + } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java new file mode 100644 index 0000000000000..af785eb9b268e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DirMarkerTracker.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.impl; + +import java.util.Map; +import java.util.TreeMap; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; + +/** + * Tracks directory markers which have been reported in object listings. + * This is needed for auditing and cleanup, including during rename + * operations. + *

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

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

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

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

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