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);