|
@@ -42,7 +42,6 @@ import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
-import org.apache.hadoop.fs.FileStatus;
|
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
|
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
|
@@ -58,6 +57,7 @@ import static org.apache.hadoop.fs.s3a.S3ATestUtils.MetricDiff;
|
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
|
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
|
|
import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles;
|
|
import static org.apache.hadoop.fs.s3a.S3AUtils.applyLocatedFiles;
|
|
import static org.apache.hadoop.fs.s3a.Statistic.FILES_DELETE_REJECTED;
|
|
import static org.apache.hadoop.fs.s3a.Statistic.FILES_DELETE_REJECTED;
|
|
|
|
+import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_OBJECTS;
|
|
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUESTS;
|
|
import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUESTS;
|
|
import static org.apache.hadoop.fs.s3a.auth.RoleModel.Effects;
|
|
import static org.apache.hadoop.fs.s3a.auth.RoleModel.Effects;
|
|
import static org.apache.hadoop.fs.s3a.auth.RoleModel.Statement;
|
|
import static org.apache.hadoop.fs.s3a.auth.RoleModel.Statement;
|
|
@@ -72,6 +72,7 @@ import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
|
|
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion;
|
|
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion;
|
|
import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.extractUndeletedPaths;
|
|
import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.extractUndeletedPaths;
|
|
import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.removeUndeletedPaths;
|
|
import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.removeUndeletedPaths;
|
|
|
|
+import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.toPathList;
|
|
import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.assertFileCount;
|
|
import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.assertFileCount;
|
|
import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.extractCause;
|
|
import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.extractCause;
|
|
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
|
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
|
@@ -331,27 +332,37 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
|
|
removeBucketOverrides(bucketName, conf,
|
|
removeBucketOverrides(bucketName, conf,
|
|
MAX_THREADS,
|
|
MAX_THREADS,
|
|
MAXIMUM_CONNECTIONS,
|
|
MAXIMUM_CONNECTIONS,
|
|
- S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY);
|
|
|
|
|
|
+ S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY,
|
|
|
|
+ DIRECTORY_MARKER_POLICY,
|
|
|
|
+ BULK_DELETE_PAGE_SIZE);
|
|
conf.setInt(MAX_THREADS, EXECUTOR_THREAD_COUNT);
|
|
conf.setInt(MAX_THREADS, EXECUTOR_THREAD_COUNT);
|
|
conf.setInt(MAXIMUM_CONNECTIONS, EXECUTOR_THREAD_COUNT * 2);
|
|
conf.setInt(MAXIMUM_CONNECTIONS, EXECUTOR_THREAD_COUNT * 2);
|
|
// turn off prune delays, so as to stop scale tests creating
|
|
// turn off prune delays, so as to stop scale tests creating
|
|
// so much cruft that future CLI prune commands take forever
|
|
// so much cruft that future CLI prune commands take forever
|
|
conf.setInt(S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY, 0);
|
|
conf.setInt(S3GUARD_DDB_BACKGROUND_SLEEP_MSEC_KEY, 0);
|
|
|
|
+ // use the keep policy to ensure that surplus markers exist
|
|
|
|
+ // to complicate failures
|
|
|
|
+ conf.set(DIRECTORY_MARKER_POLICY, DIRECTORY_MARKER_POLICY_KEEP);
|
|
|
|
+ // set the delete page size to its maximum to ensure that all
|
|
|
|
+ // entries are included in the same large delete, even on
|
|
|
|
+ // scale runs. This is needed for assertions on the result.
|
|
|
|
+ conf.setInt(BULK_DELETE_PAGE_SIZE, 1_000);
|
|
return conf;
|
|
return conf;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
* Create a unique path, which includes method name,
|
|
* Create a unique path, which includes method name,
|
|
- * multidelete flag and a random UUID.
|
|
|
|
|
|
+ * multidelete flag and a timestamp.
|
|
* @return a string to use for paths.
|
|
* @return a string to use for paths.
|
|
* @throws IOException path creation failure.
|
|
* @throws IOException path creation failure.
|
|
*/
|
|
*/
|
|
private Path uniquePath() throws IOException {
|
|
private Path uniquePath() throws IOException {
|
|
|
|
+ long now = System.currentTimeMillis();
|
|
return path(
|
|
return path(
|
|
- String.format("%s-%s-%04d",
|
|
|
|
|
|
+ String.format("%s-%s-%06d.%03d",
|
|
getMethodName(),
|
|
getMethodName(),
|
|
multiDelete ? "multi" : "single",
|
|
multiDelete ? "multi" : "single",
|
|
- System.currentTimeMillis() % 10000));
|
|
|
|
|
|
+ now / 1000, now % 1000));
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -477,8 +488,11 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
|
|
|
|
|
|
// create a set of files
|
|
// create a set of files
|
|
// this is done in parallel as it is 10x faster on a long-haul test run.
|
|
// this is done in parallel as it is 10x faster on a long-haul test run.
|
|
- List<Path> createdFiles = createFiles(fs, readOnlyDir, dirDepth, fileCount,
|
|
|
|
- dirCount);
|
|
|
|
|
|
+ List<Path> dirs = new ArrayList<>(dirCount);
|
|
|
|
+ List<Path> createdFiles = createDirsAndFiles(fs, readOnlyDir, dirDepth,
|
|
|
|
+ fileCount, dirCount,
|
|
|
|
+ new ArrayList<>(fileCount),
|
|
|
|
+ dirs);
|
|
// are they all there?
|
|
// are they all there?
|
|
int expectedFileCount = createdFiles.size();
|
|
int expectedFileCount = createdFiles.size();
|
|
assertFileCount("files ready to rename", roleFS,
|
|
assertFileCount("files ready to rename", roleFS,
|
|
@@ -495,26 +509,36 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
|
|
MultiObjectDeleteException.class, deniedException);
|
|
MultiObjectDeleteException.class, deniedException);
|
|
final List<Path> undeleted
|
|
final List<Path> undeleted
|
|
= extractUndeletedPaths(mde, fs::keyToQualifiedPath);
|
|
= extractUndeletedPaths(mde, fs::keyToQualifiedPath);
|
|
|
|
+
|
|
|
|
+ List<Path> expectedUndeletedFiles = new ArrayList<>(createdFiles);
|
|
|
|
+ if (getFileSystem().getDirectoryMarkerPolicy()
|
|
|
|
+ .keepDirectoryMarkers(readOnlyDir)) {
|
|
|
|
+ // directory markers are being retained,
|
|
|
|
+ // so will also be in the list of undeleted files
|
|
|
|
+ expectedUndeletedFiles.addAll(dirs);
|
|
|
|
+ }
|
|
Assertions.assertThat(undeleted)
|
|
Assertions.assertThat(undeleted)
|
|
.as("files which could not be deleted")
|
|
.as("files which could not be deleted")
|
|
- .hasSize(expectedFileCount)
|
|
|
|
- .containsAll(createdFiles)
|
|
|
|
- .containsExactlyInAnyOrderElementsOf(createdFiles);
|
|
|
|
|
|
+ .containsExactlyInAnyOrderElementsOf(expectedUndeletedFiles);
|
|
}
|
|
}
|
|
LOG.info("Result of renaming read-only files is as expected",
|
|
LOG.info("Result of renaming read-only files is as expected",
|
|
deniedException);
|
|
deniedException);
|
|
assertFileCount("files in the source directory", roleFS,
|
|
assertFileCount("files in the source directory", roleFS,
|
|
readOnlyDir, expectedFileCount);
|
|
readOnlyDir, expectedFileCount);
|
|
// now lets look at the destination.
|
|
// now lets look at the destination.
|
|
- // even with S3Guard on, we expect the destination to match that of our
|
|
|
|
|
|
+ // even with S3Guard on, we expect the destination to match that of
|
|
// the remote state.
|
|
// the remote state.
|
|
// the test will exist
|
|
// the test will exist
|
|
describe("Verify destination directory exists");
|
|
describe("Verify destination directory exists");
|
|
- FileStatus st = roleFS.getFileStatus(writableDir);
|
|
|
|
- assertTrue("Not a directory: " + st,
|
|
|
|
- st.isDirectory());
|
|
|
|
|
|
+ assertIsDirectory(writableDir);
|
|
assertFileCount("files in the dest directory", roleFS,
|
|
assertFileCount("files in the dest directory", roleFS,
|
|
writableDir, expectedFileCount);
|
|
writableDir, expectedFileCount);
|
|
|
|
+ // all directories in the source tree must still exist,
|
|
|
|
+ // which for S3Guard means no tombstone markers were added
|
|
|
|
+ LOG.info("Verifying all directories still exist");
|
|
|
|
+ for (Path dir : dirs) {
|
|
|
|
+ assertIsDirectory(dir);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
@Test
|
|
@Test
|
|
@@ -611,9 +635,14 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
|
|
|
|
|
|
// the full FS
|
|
// the full FS
|
|
S3AFileSystem fs = getFileSystem();
|
|
S3AFileSystem fs = getFileSystem();
|
|
-
|
|
|
|
- List<Path> readOnlyFiles = createFiles(fs, readOnlyDir,
|
|
|
|
- dirDepth, fileCount, dirCount);
|
|
|
|
|
|
+ StoreContext storeContext = fs.createStoreContext();
|
|
|
|
+
|
|
|
|
+ List<Path> dirs = new ArrayList<>(dirCount);
|
|
|
|
+ List<Path> readOnlyFiles = createDirsAndFiles(
|
|
|
|
+ fs, readOnlyDir, dirDepth,
|
|
|
|
+ fileCount, dirCount,
|
|
|
|
+ new ArrayList<>(fileCount),
|
|
|
|
+ dirs);
|
|
List<Path> deletableFiles = createFiles(fs,
|
|
List<Path> deletableFiles = createFiles(fs,
|
|
writableDir, dirDepth, fileCount, dirCount);
|
|
writableDir, dirDepth, fileCount, dirCount);
|
|
|
|
|
|
@@ -625,20 +654,31 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
|
|
readOnlyFiles.stream(),
|
|
readOnlyFiles.stream(),
|
|
deletableFiles.stream())
|
|
deletableFiles.stream())
|
|
.collect(Collectors.toList());
|
|
.collect(Collectors.toList());
|
|
|
|
+ List<MultiObjectDeleteSupport.KeyPath> keyPaths = allFiles.stream()
|
|
|
|
+ .map(path ->
|
|
|
|
+ new MultiObjectDeleteSupport.KeyPath(
|
|
|
|
+ storeContext.pathToKey(path),
|
|
|
|
+ path,
|
|
|
|
+ false))
|
|
|
|
+ .collect(Collectors.toList());
|
|
|
|
|
|
// this set can be deleted by the role FS
|
|
// this set can be deleted by the role FS
|
|
MetricDiff rejectionCount = new MetricDiff(roleFS, FILES_DELETE_REJECTED);
|
|
MetricDiff rejectionCount = new MetricDiff(roleFS, FILES_DELETE_REJECTED);
|
|
MetricDiff deleteVerbCount = new MetricDiff(roleFS, OBJECT_DELETE_REQUESTS);
|
|
MetricDiff deleteVerbCount = new MetricDiff(roleFS, OBJECT_DELETE_REQUESTS);
|
|
|
|
+ MetricDiff deleteObjectCount = new MetricDiff(roleFS,
|
|
|
|
+ OBJECT_DELETE_OBJECTS);
|
|
|
|
|
|
describe("Trying to delete read only directory");
|
|
describe("Trying to delete read only directory");
|
|
AccessDeniedException ex = expectDeleteForbidden(readOnlyDir);
|
|
AccessDeniedException ex = expectDeleteForbidden(readOnlyDir);
|
|
if (multiDelete) {
|
|
if (multiDelete) {
|
|
// multi-delete status checks
|
|
// multi-delete status checks
|
|
extractCause(MultiObjectDeleteException.class, ex);
|
|
extractCause(MultiObjectDeleteException.class, ex);
|
|
|
|
+ deleteVerbCount.assertDiffEquals("Wrong delete request count", 1);
|
|
|
|
+ deleteObjectCount.assertDiffEquals("Number of keys in delete request",
|
|
|
|
+ readOnlyFiles.size());
|
|
rejectionCount.assertDiffEquals("Wrong rejection count",
|
|
rejectionCount.assertDiffEquals("Wrong rejection count",
|
|
readOnlyFiles.size());
|
|
readOnlyFiles.size());
|
|
- deleteVerbCount.assertDiffEquals("Wrong delete count", 1);
|
|
|
|
- reset(rejectionCount, deleteVerbCount);
|
|
|
|
|
|
+ reset(rejectionCount, deleteVerbCount, deleteObjectCount);
|
|
}
|
|
}
|
|
// all the files are still there? (avoid in scale test due to cost)
|
|
// all the files are still there? (avoid in scale test due to cost)
|
|
if (!scaleTest) {
|
|
if (!scaleTest) {
|
|
@@ -649,16 +689,20 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
|
|
ex = expectDeleteForbidden(basePath);
|
|
ex = expectDeleteForbidden(basePath);
|
|
if (multiDelete) {
|
|
if (multiDelete) {
|
|
// multi-delete status checks
|
|
// multi-delete status checks
|
|
- extractCause(MultiObjectDeleteException.class, ex);
|
|
|
|
deleteVerbCount.assertDiffEquals("Wrong delete count", 1);
|
|
deleteVerbCount.assertDiffEquals("Wrong delete count", 1);
|
|
MultiObjectDeleteException mde = extractCause(
|
|
MultiObjectDeleteException mde = extractCause(
|
|
MultiObjectDeleteException.class, ex);
|
|
MultiObjectDeleteException.class, ex);
|
|
- final List<Path> undeleted
|
|
|
|
- = removeUndeletedPaths(mde, allFiles, fs::keyToQualifiedPath);
|
|
|
|
|
|
+ List<MultiObjectDeleteSupport.KeyPath> undeletedKeyPaths =
|
|
|
|
+ removeUndeletedPaths(mde, keyPaths, storeContext::keyToPath);
|
|
|
|
+ final List<Path> undeleted = toPathList(
|
|
|
|
+ undeletedKeyPaths);
|
|
|
|
+ deleteObjectCount.assertDiffEquals(
|
|
|
|
+ "Wrong count of objects in delete request",
|
|
|
|
+ allFiles.size());
|
|
Assertions.assertThat(undeleted)
|
|
Assertions.assertThat(undeleted)
|
|
.as("files which could not be deleted")
|
|
.as("files which could not be deleted")
|
|
.containsExactlyInAnyOrderElementsOf(readOnlyFiles);
|
|
.containsExactlyInAnyOrderElementsOf(readOnlyFiles);
|
|
- Assertions.assertThat(allFiles)
|
|
|
|
|
|
+ Assertions.assertThat(toPathList(keyPaths))
|
|
.as("files which were deleted")
|
|
.as("files which were deleted")
|
|
.containsExactlyInAnyOrderElementsOf(deletableFiles);
|
|
.containsExactlyInAnyOrderElementsOf(deletableFiles);
|
|
rejectionCount.assertDiffEquals("Wrong rejection count",
|
|
rejectionCount.assertDiffEquals("Wrong rejection count",
|
|
@@ -677,7 +721,26 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
|
|
|
|
|
|
Assertions.assertThat(readOnlyListing)
|
|
Assertions.assertThat(readOnlyListing)
|
|
.as("ReadOnly directory " + directoryList)
|
|
.as("ReadOnly directory " + directoryList)
|
|
- .containsAll(readOnlyFiles);
|
|
|
|
|
|
+ .containsExactlyInAnyOrderElementsOf(readOnlyFiles);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Verifies the logic of handling directory markers in
|
|
|
|
+ * delete operations, specifically:
|
|
|
|
+ * <ol>
|
|
|
|
+ * <li>all markers above empty directories MUST be deleted</li>
|
|
|
|
+ * <li>all markers above non-empty directories MUST NOT be deleted</li>
|
|
|
|
+ * </ol>
|
|
|
|
+ * As the delete list may include subdirectories, we need to work up from
|
|
|
|
+ * the bottom of the list of deleted files before probing the parents,
|
|
|
|
+ * that being done by a s3guard get(path, need-empty-directory) call.
|
|
|
|
+ * <p></p>
|
|
|
|
+ * This is pretty sensitive code.
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testSubdirDeleteFailures() throws Throwable {
|
|
|
|
+ describe("Multiobject delete handling of directorYesFory markers");
|
|
|
|
+ assume("Multiobject delete only", multiDelete);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -771,7 +834,7 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Parallel-touch a set of files in the destination directory.
|
|
|
|
|
|
+ * Build a set of files in a directory tree.
|
|
* @param fs filesystem
|
|
* @param fs filesystem
|
|
* @param destDir destination
|
|
* @param destDir destination
|
|
* @param depth file depth
|
|
* @param depth file depth
|
|
@@ -784,12 +847,48 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
|
|
final int depth,
|
|
final int depth,
|
|
final int fileCount,
|
|
final int fileCount,
|
|
final int dirCount) throws IOException {
|
|
final int dirCount) throws IOException {
|
|
- List<CompletableFuture<Path>> futures = new ArrayList<>(fileCount);
|
|
|
|
- List<Path> paths = new ArrayList<>(fileCount);
|
|
|
|
- List<Path> dirs = new ArrayList<>(fileCount);
|
|
|
|
|
|
+ return createDirsAndFiles(fs, destDir, depth, fileCount, dirCount,
|
|
|
|
+ new ArrayList<Path>(fileCount),
|
|
|
|
+ new ArrayList<Path>(dirCount));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Build a set of files in a directory tree.
|
|
|
|
+ * @param fs filesystem
|
|
|
|
+ * @param destDir destination
|
|
|
|
+ * @param depth file depth
|
|
|
|
+ * @param fileCount number of files to create.
|
|
|
|
+ * @param dirCount number of dirs to create at each level
|
|
|
|
+ * @param paths [out] list of file paths created
|
|
|
|
+ * @param dirs [out] list of directory paths created.
|
|
|
|
+ * @return the list of files created.
|
|
|
|
+ */
|
|
|
|
+ public static List<Path> createDirsAndFiles(final FileSystem fs,
|
|
|
|
+ final Path destDir,
|
|
|
|
+ final int depth,
|
|
|
|
+ final int fileCount,
|
|
|
|
+ final int dirCount,
|
|
|
|
+ final List<Path> paths,
|
|
|
|
+ final List<Path> dirs) throws IOException {
|
|
buildPaths(paths, dirs, destDir, depth, fileCount, dirCount);
|
|
buildPaths(paths, dirs, destDir, depth, fileCount, dirCount);
|
|
|
|
+ List<CompletableFuture<Path>> futures = new ArrayList<>(paths.size()
|
|
|
|
+ + dirs.size());
|
|
|
|
+
|
|
|
|
+ // create directories. With dir marker retention, that adds more entries
|
|
|
|
+ // to cause deletion issues
|
|
|
|
+ try (DurationInfo ignore =
|
|
|
|
+ new DurationInfo(LOG, "Creating %d directories", dirs.size())) {
|
|
|
|
+ for (Path path : dirs) {
|
|
|
|
+ futures.add(submit(EXECUTOR, () ->{
|
|
|
|
+ fs.mkdirs(path);
|
|
|
|
+ return path;
|
|
|
|
+ }));
|
|
|
|
+ }
|
|
|
|
+ waitForCompletion(futures);
|
|
|
|
+ }
|
|
|
|
+
|
|
try (DurationInfo ignore =
|
|
try (DurationInfo ignore =
|
|
- new DurationInfo(LOG, "Creating %d files", fileCount)) {
|
|
|
|
|
|
+ new DurationInfo(LOG, "Creating %d files", paths.size())) {
|
|
for (Path path : paths) {
|
|
for (Path path : paths) {
|
|
futures.add(put(fs, path, path.getName()));
|
|
futures.add(put(fs, path, path.getName()));
|
|
}
|
|
}
|