diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md index 9440b4e050a72..0b09f02b93982 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md @@ -220,21 +220,21 @@ directory contains many thousands of files. Consider a directory `"/d"` with the contents: - a - part-0000001 - part-0000002 - ... - part-9999999 + a + part-0000001 + part-0000002 + ... + part-9999999 If the number of files is such that HDFS returns a partial listing in each response, then, if a listing `listStatus("/d")` takes place concurrently with the operation `rename("/d/a","/d/z"))`, the result may be one of: - [a, part-0000001, ... , part-9999999] - [part-0000001, ... , part-9999999, z] - [a, part-0000001, ... , part-9999999, z] - [part-0000001, ... , part-9999999] + [a, part-0000001, ... , part-9999999] + [part-0000001, ... , part-9999999, z] + [a, part-0000001, ... , part-9999999, z] + [part-0000001, ... , part-9999999] While this situation is likely to be a rare occurrence, it MAY happen. In HDFS these inconsistent views are only likely when listing a directory with many children. @@ -964,7 +964,7 @@ A path referring to a file is removed, return value: `True` Deleting an empty root does not change the filesystem state and may return true or false. - if isDir(FS, p) and isRoot(p) and children(FS, p) == {} : + if isRoot(p) and children(FS, p) == {} : FS ' = FS result = (undetermined) @@ -973,6 +973,9 @@ There is no consistent return code from an attempt to delete the root directory. Implementations SHOULD return true; this avoids code which checks for a false return value from overreacting. +*Object Stores*: see [Object Stores: root directory deletion](#object-stores-rm-root). + + ##### Empty (non-root) directory `recursive == False` Deleting an empty directory that is not root will remove the path from the FS and @@ -986,7 +989,7 @@ return true. ##### Recursive delete of non-empty root directory Deleting a root path with children and `recursive==True` - can do one of two things. +can generally have three outcomes: 1. The POSIX model assumes that if the user has the correct permissions to delete everything, @@ -1004,6 +1007,8 @@ filesystem is desired. FS' = FS result = False +1. Object Stores: see [Object Stores: root directory deletion](#object-stores-rm-root). + HDFS has the notion of *Protected Directories*, which are declared in the option `fs.protected.directories`. Any attempt to delete such a directory or a parent thereof raises an `AccessControlException`. Accordingly, any @@ -1019,6 +1024,23 @@ Any filesystem client which interacts with a remote filesystem which lacks such a security model, MAY reject calls to `delete("/", true)` on the basis that it makes it too easy to lose data. + +### Object Stores: root directory deletion + +Some of the object store based filesystem implementations always return +false when deleting the root, leaving the state of the store unchanged. + + if isRoot(p) : + FS ' = FS + result = False + +This is irrespective of the recursive flag status or the state of the directory. + +This is a simplification which avoids the inevitably non-atomic scan and delete +of the contents of the store. It also avoids any confusion about whether +the operation actually deletes that specific store/container itself, and +adverse consequences of the simpler permissions models of stores. + ##### Recursive delete of non-root directory Deleting a non-root path with children `recursive==true` diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java index 647ffdf9996c5..36ce71ac7f525 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java @@ -189,7 +189,7 @@ public void testListEmptyRootDirectory() throws IOException { Path root = new Path("/"); FileStatus[] statuses = fs.listStatus(root); for (FileStatus status : statuses) { - ContractTestUtils.assertDeleted(fs, status.getPath(), true); + ContractTestUtils.assertDeleted(fs, status.getPath(), false,true, false); } FileStatus[] rootListStatus = fs.listStatus(root); assertEquals("listStatus on empty root-directory returned found: " diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java index b7b67390aa77c..7bd984069bf9c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java @@ -674,7 +674,8 @@ public static void touch(FileSystem fs, /** * Delete a file/dir and assert that delete() returned true * and that the path no longer exists. This variant rejects - * all operations on root directories. + * all operations on root directories and requires the target path + * to exist before the deletion operation. * @param fs filesystem * @param file path to delete * @param recursive flag to enable recursive delete @@ -688,20 +689,41 @@ public static void assertDeleted(FileSystem fs, /** * Delete a file/dir and assert that delete() returned true - * and that the path no longer exists. This variant rejects - * all operations on root directories + * and that the path no longer exists. + * This variant requires the target path + * to exist before the deletion operation. + * @param fs filesystem + * @param file path to delete + * @param recursive flag to enable recursive delete + * @param allowRootOperations can the root dir be deleted? + * @throws IOException IO problems + */ + public static void assertDeleted(FileSystem fs, + Path file, + boolean recursive, + boolean allowRootOperations) throws IOException { + assertDeleted(fs, file, true, recursive, allowRootOperations); + } + + /** + * Delete a file/dir and assert that delete() returned true + * and that the path no longer exists. * @param fs filesystem * @param file path to delete + * @param requirePathToExist check for the path existing first? * @param recursive flag to enable recursive delete * @param allowRootOperations can the root dir be deleted? * @throws IOException IO problems */ public static void assertDeleted(FileSystem fs, Path file, + boolean requirePathToExist, boolean recursive, boolean allowRootOperations) throws IOException { rejectRootOperation(file, allowRootOperations); - assertPathExists(fs, "about to be deleted file", file); + if (requirePathToExist) { + assertPathExists(fs, "about to be deleted file", file); + } boolean deleted = fs.delete(file, recursive); String dir = ls(fs, file.getParent()); assertTrue("Delete failed on " + file + ": " + dir, deleted); 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 3c4f5bfd50ae1..aa00ae8fdc6ce 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 @@ -2307,30 +2307,16 @@ private boolean innerDelete(S3AFileStatus status, boolean recursive) * Implements the specific logic to reject root directory deletion. * The caller must return the result of this call, rather than * attempt to continue with the delete operation: deleting root - * directories is never allowed. This method simply implements - * the policy of when to return an exit code versus raise an exception. + * directories is never allowed. * @param status filesystem status * @param recursive recursive flag from command * @return a return code for the operation - * @throws PathIOException if the operation was explicitly rejected. */ private boolean rejectRootDirectoryDelete(S3AFileStatus status, - boolean recursive) throws IOException { - LOG.info("s3a delete the {} root directory. Path: {}. Recursive: {}", - bucket, status.getPath(), recursive); - boolean emptyRoot = status.isEmptyDirectory() == Tristate.TRUE; - if (emptyRoot) { - return true; - } - if (recursive) { - LOG.error("Cannot delete root path: {}", status.getPath()); - return false; - } else { - // reject - String msg = "Cannot delete root path: " + status.getPath(); - LOG.error(msg); - throw new PathIOException(bucket, msg); - } + boolean recursive) { + LOG.error("S3A: Cannot delete the {} root directory. Path: {}. Recursive: " + + "{}", bucket, status.getPath(), recursive); + return false; } /** @@ -2623,7 +2609,8 @@ S3AFileStatus innerGetFileStatus(final Path f, // Check MetadataStore, if any. PathMetadata pm = null; if (hasMetadataStore()) { - pm = S3Guard.getWithTtl(metadataStore, path, ttlTimeProvider); + pm = S3Guard.getWithTtl(metadataStore, path, ttlTimeProvider, + needEmptyDirectoryFlag); } Set tombstones = Collections.emptySet(); if (pm != null) { 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 8e5c4d2ed7012..5feb4b9b1c941 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 @@ -712,12 +712,15 @@ static void patchLastUpdated( * @param ms metastore * @param path path to look up. * @param timeProvider nullable time provider + * @param needEmptyDirectoryFlag if true, implementation will + * return known state of directory emptiness. * @return the metadata or null if there as no entry. * @throws IOException failure. */ public static PathMetadata getWithTtl(MetadataStore ms, Path path, - @Nullable ITtlTimeProvider timeProvider) throws IOException { - final PathMetadata pathMetadata = ms.get(path); + @Nullable ITtlTimeProvider timeProvider, + final boolean needEmptyDirectoryFlag) throws IOException { + final PathMetadata pathMetadata = ms.get(path, needEmptyDirectoryFlag); // if timeProvider is null let's return with what the ms has if (timeProvider == null) { LOG.debug("timeProvider is null, returning pathMetadata as is"); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java index 9ef3763bfc39a..dc82f05e252f4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.contract.AbstractFSContract; import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.junit.Ignore; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,6 +63,11 @@ public S3AFileSystem getFileSystem() { return (S3AFileSystem) super.getFileSystem(); } + @Override + @Ignore("S3 always return false when non-recursively remove root dir") + public void testRmNonEmptyRootDirNonRecursive() throws Throwable { + } + /** * This is overridden to allow for eventual consistency on listings, * but only if the store does not have S3Guard protecting it. @@ -69,9 +75,6 @@ public S3AFileSystem getFileSystem() { @Override public void testListEmptyRootDirectory() throws IOException { int maxAttempts = 10; - if (getFileSystem().hasMetadataStore()) { - maxAttempts = 1; - } describe("Listing root directory; for consistency allowing " + maxAttempts + " attempts"); for (int attempt = 1; attempt <= maxAttempts; ++attempt) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java index c086597355f47..c85af29697f61 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardEmptyDirs.java @@ -20,16 +20,30 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.io.InputStream; +import java.util.stream.Stream; + +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.model.ListObjectsV2Request; +import com.amazonaws.services.s3.model.ListObjectsV2Result; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.S3ObjectSummary; +import org.assertj.core.api.Assertions; +import org.junit.Test; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.fs.s3a.s3guard.DDBPathMetadata; +import org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore; import org.apache.hadoop.fs.s3a.s3guard.MetadataStore; import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore; -import org.junit.Assume; -import org.junit.Test; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeFilesystemHasMetadatastore; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.getStatusWithEmptyDirFlag; /** * Test logic around whether or not a directory is empty, with S3Guard enabled. @@ -84,7 +98,7 @@ private S3AFileStatus getEmptyDirStatus(Path dir) throws IOException { @Test public void testEmptyDirs() throws Exception { S3AFileSystem fs = getFileSystem(); - Assume.assumeTrue(fs.hasMetadataStore()); + assumeFilesystemHasMetadatastore(getFileSystem()); MetadataStore configuredMs = fs.getMetadataStore(); Path existingDir = path("existing-dir"); Path existingFile = path("existing-dir/existing-file"); @@ -126,4 +140,139 @@ public void testEmptyDirs() throws Exception { configuredMs.forgetMetadata(existingDir); } } + + /** + * Test tombstones don't get in the way of a listing of the + * root dir. + * This test needs to create a path which appears first in the listing, + * and an entry which can come later. To allow the test to proceed + * while other tests are running, the filename "0000" is used for that + * deleted entry. + */ + @Test + public void testTombstonesAndEmptyDirectories() throws Throwable { + S3AFileSystem fs = getFileSystem(); + assumeFilesystemHasMetadatastore(getFileSystem()); + + // Create the first and last files. + Path base = path(getMethodName()); + // use something ahead of all the ASCII alphabet characters so + // even during parallel test runs, this test is expected to work. + String first = "0000"; + Path firstPath = new Path(base, first); + + // this path is near the bottom of the ASCII string space. + // This isn't so critical. + String last = "zzzz"; + Path lastPath = new Path(base, last); + touch(fs, firstPath); + touch(fs, lastPath); + // Delete first entry (+assert tombstone) + assertDeleted(firstPath, false); + DynamoDBMetadataStore ddbMs = getRequiredDDBMetastore(fs); + DDBPathMetadata firstMD = ddbMs.get(firstPath); + assertNotNull("No MD for " + firstPath, firstMD); + assertTrue("Not a tombstone " + firstMD, + firstMD.isDeleted()); + // PUT child to store going past the FS entirely. + // This is not going to show up on S3Guard. + Path child = new Path(firstPath, "child"); + StoreContext ctx = fs.createStoreContext(); + String childKey = ctx.pathToKey(child); + String baseKey = ctx.pathToKey(base) + "/"; + AmazonS3 s3 = fs.getAmazonS3ClientForTesting("LIST"); + String bucket = ctx.getBucket(); + try { + createEmptyObject(fs, childKey); + + // Do a list + ListObjectsV2Request listReq = new ListObjectsV2Request() + .withBucketName(bucket) + .withPrefix(baseKey) + .withMaxKeys(10) + .withDelimiter("/"); + ListObjectsV2Result listing = s3.listObjectsV2(listReq); + + // the listing has the first path as a prefix, because of the child + Assertions.assertThat(listing.getCommonPrefixes()) + .describedAs("The prefixes of a LIST of %s", base) + .contains(baseKey + first + "/"); + + // and the last file is one of the files + Stream files = listing.getObjectSummaries() + .stream() + .map(S3ObjectSummary::getKey); + Assertions.assertThat(files) + .describedAs("The files of a LIST of %s", base) + .contains(baseKey + last); + + // verify absolutely that the last file exists + assertPathExists("last file", lastPath); + + boolean isDDB = fs.getMetadataStore() instanceof DynamoDBMetadataStore; + // if DDB is the metastore, then we expect no FS requests to be made + // at all. + S3ATestUtils.MetricDiff listMetric = new S3ATestUtils.MetricDiff(fs, + Statistic.OBJECT_LIST_REQUESTS); + S3ATestUtils.MetricDiff getMetric = new S3ATestUtils.MetricDiff(fs, + Statistic.OBJECT_METADATA_REQUESTS); + // do a getFile status with empty dir flag + S3AFileStatus status = getStatusWithEmptyDirFlag(fs, base); + assertNonEmptyDir(status); + if (isDDB) { + listMetric.assertDiffEquals( + "FileSystem called S3 LIST rather than use DynamoDB", + 0); + getMetric.assertDiffEquals( + "FileSystem called S3 GET rather than use DynamoDB", + 0); + LOG.info("Verified that DDB directory status was accepted"); + } + + } finally { + // try to recover from the defective state. + s3.deleteObject(bucket, childKey); + fs.delete(lastPath, true); + ddbMs.forgetMetadata(firstPath); + } + } + + protected void assertNonEmptyDir(final S3AFileStatus status) { + assertEquals("Should not be empty dir: " + status, Tristate.FALSE, + status.isEmptyDirectory()); + } + + /** + * Get the DynamoDB metastore; assume false if it is of a different + * type. + * @return extracted and cast metadata store. + */ + @SuppressWarnings("ConstantConditions") + private DynamoDBMetadataStore getRequiredDDBMetastore(S3AFileSystem fs) { + MetadataStore ms = fs.getMetadataStore(); + assume("Not a DynamoDBMetadataStore: " + ms, + ms instanceof DynamoDBMetadataStore); + return (DynamoDBMetadataStore) ms; + } + + /** + * From {@code S3AFileSystem.createEmptyObject()}. + * @param fs filesystem + * @param key key + */ + private void createEmptyObject(S3AFileSystem fs, String key) { + final InputStream im = new InputStream() { + @Override + public int read() { + return -1; + } + }; + + PutObjectRequest putObjectRequest = fs.newPutObjectRequest(key, + fs.newObjectMetadata(0L), + im); + AmazonS3 s3 = fs.getAmazonS3ClientForTesting("PUT"); + s3.putObject(putObjectRequest); + } + } 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 a789e83d1dda9..c65f6bbdc7e3c 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 @@ -505,6 +505,16 @@ public static boolean isMetadataStoreAuthoritative(Configuration conf) { Constants.DEFAULT_METADATASTORE_AUTHORITATIVE); } + /** + * Require a filesystem to have a metadata store; skip test + * if not. + * @param fs filesystem to check + */ + public static void assumeFilesystemHasMetadatastore(S3AFileSystem fs) { + assume("Filesystem does not have a metastore", + fs.hasMetadataStore()); + } + /** * Reset all metrics in a list. * @param metrics metrics to reset @@ -818,6 +828,22 @@ public static T terminateService(final T service) { return null; } + /** + * Get a file status from S3A with the {@code needEmptyDirectoryFlag} + * state probed. + * This accesses a package-private method in the + * S3A filesystem. + * @param fs filesystem + * @param dir directory + * @return a status + * @throws IOException + */ + public static S3AFileStatus getStatusWithEmptyDirFlag( + final S3AFileSystem fs, + final Path dir) throws IOException { + return fs.innerGetFileStatus(dir, true); + } + /** * Helper class to do diffs of metrics. */ 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 74c406978e3e9..e2c95e194bbf6 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 @@ -234,7 +234,7 @@ public void test_400_rm_root_recursive() throws Throwable { assertDeleted(file, false); - assertTrue("Root directory delete failed", + assertFalse("Root directory delete failed", fs.delete(root, true)); ContractTestUtils.touch(fs, file2); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java index bb5557e597308..90962ada117ab 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3Guard.java @@ -154,7 +154,7 @@ public void testGetWithTtlExpired() throws Exception { pm.setLastUpdated(100L); MetadataStore ms = mock(MetadataStore.class); - when(ms.get(path)).thenReturn(pm); + when(ms.get(path, false)).thenReturn(pm); ITtlTimeProvider timeProvider = mock(ITtlTimeProvider.class); @@ -162,7 +162,8 @@ public void testGetWithTtlExpired() throws Exception { when(timeProvider.getMetadataTtl()).thenReturn(1L); // act - final PathMetadata pmExpired = S3Guard.getWithTtl(ms, path, timeProvider); + final PathMetadata pmExpired = S3Guard.getWithTtl(ms, path, timeProvider, + false); // assert assertNull(pmExpired); @@ -178,7 +179,7 @@ public void testGetWithTtlNotExpired() throws Exception { pm.setLastUpdated(100L); MetadataStore ms = mock(MetadataStore.class); - when(ms.get(path)).thenReturn(pm); + when(ms.get(path, false)).thenReturn(pm); ITtlTimeProvider timeProvider = mock(ITtlTimeProvider.class); @@ -187,7 +188,7 @@ public void testGetWithTtlNotExpired() throws Exception { // act final PathMetadata pmNotExpired = - S3Guard.getWithTtl(ms, path, timeProvider); + S3Guard.getWithTtl(ms, path, timeProvider, false); // assert assertNotNull(pmNotExpired); @@ -205,7 +206,7 @@ public void testGetWithZeroLastUpdatedNotExpired() throws Exception { pm.setLastUpdated(0L); MetadataStore ms = mock(MetadataStore.class); - when(ms.get(path)).thenReturn(pm); + when(ms.get(path, false)).thenReturn(pm); ITtlTimeProvider timeProvider = mock(ITtlTimeProvider.class); @@ -213,7 +214,8 @@ public void testGetWithZeroLastUpdatedNotExpired() throws Exception { when(timeProvider.getMetadataTtl()).thenReturn(2L); // act - final PathMetadata pmExpired = S3Guard.getWithTtl(ms, path, timeProvider); + final PathMetadata pmExpired = S3Guard.getWithTtl(ms, path, timeProvider, + false); // assert assertNotNull(pmExpired);