|
@@ -135,6 +135,7 @@ import org.apache.hadoop.security.token.DelegationTokenIssuer;
|
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
|
import org.apache.hadoop.util.DurationInfo;
|
|
import org.apache.hadoop.util.DurationInfo;
|
|
import org.apache.hadoop.util.LambdaUtils;
|
|
import org.apache.hadoop.util.LambdaUtils;
|
|
|
|
+import org.apache.hadoop.util.Lists;
|
|
import org.apache.hadoop.util.Preconditions;
|
|
import org.apache.hadoop.util.Preconditions;
|
|
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
@@ -225,6 +226,7 @@ import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDura
|
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation;
|
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation;
|
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier;
|
|
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier;
|
|
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
|
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
|
|
|
+import static org.apache.hadoop.util.Preconditions.checkArgument;
|
|
import static org.apache.hadoop.util.functional.RemoteIterators.typeCastingRemoteIterator;
|
|
import static org.apache.hadoop.util.functional.RemoteIterators.typeCastingRemoteIterator;
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -550,6 +552,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
|
|
|
|
pageSize = intOption(getConf(), BULK_DELETE_PAGE_SIZE,
|
|
pageSize = intOption(getConf(), BULK_DELETE_PAGE_SIZE,
|
|
BULK_DELETE_PAGE_SIZE_DEFAULT, 0);
|
|
BULK_DELETE_PAGE_SIZE_DEFAULT, 0);
|
|
|
|
+ checkArgument(pageSize <= InternalConstants.MAX_ENTRIES_TO_DELETE,
|
|
|
|
+ "page size out of range: %s", pageSize);
|
|
listing = new Listing(listingOperationCallbacks, createStoreContext());
|
|
listing = new Listing(listingOperationCallbacks, createStoreContext());
|
|
} catch (AmazonClientException e) {
|
|
} catch (AmazonClientException e) {
|
|
// amazon client exception: stop all services then throw the translation
|
|
// amazon client exception: stop all services then throw the translation
|
|
@@ -2026,14 +2030,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public DeleteObjectsResult removeKeys(
|
|
|
|
- final List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
|
|
|
- final boolean deleteFakeDir,
|
|
|
|
- final boolean quiet)
|
|
|
|
|
|
+ public void removeKeys(
|
|
|
|
+ final List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
|
|
|
+ final boolean deleteFakeDir)
|
|
throws MultiObjectDeleteException, AmazonClientException, IOException {
|
|
throws MultiObjectDeleteException, AmazonClientException, IOException {
|
|
auditSpan.activate();
|
|
auditSpan.activate();
|
|
- return S3AFileSystem.this.removeKeys(keysToDelete, deleteFakeDir,
|
|
|
|
- quiet);
|
|
|
|
|
|
+ S3AFileSystem.this.removeKeys(keysToDelete, deleteFakeDir);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -2818,10 +2820,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
* @param keysToDelete collection of keys to delete on the s3-backend.
|
|
* @param keysToDelete collection of keys to delete on the s3-backend.
|
|
* if empty, no request is made of the object store.
|
|
* if empty, no request is made of the object store.
|
|
* @param deleteFakeDir indicates whether this is for deleting fake dirs
|
|
* @param deleteFakeDir indicates whether this is for deleting fake dirs
|
|
- * @param quiet should a bulk query be quiet, or should its result list
|
|
|
|
- * all deleted keys?
|
|
|
|
- * @return the deletion result if a multi object delete was invoked
|
|
|
|
- * and it returned without a failure.
|
|
|
|
* @throws InvalidRequestException if the request was rejected due to
|
|
* @throws InvalidRequestException if the request was rejected due to
|
|
* a mistaken attempt to delete the root directory.
|
|
* a mistaken attempt to delete the root directory.
|
|
* @throws MultiObjectDeleteException one or more of the keys could not
|
|
* @throws MultiObjectDeleteException one or more of the keys could not
|
|
@@ -2831,10 +2829,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
* @throws AmazonClientException other amazon-layer failure.
|
|
* @throws AmazonClientException other amazon-layer failure.
|
|
*/
|
|
*/
|
|
@Retries.RetryRaw
|
|
@Retries.RetryRaw
|
|
- private DeleteObjectsResult removeKeysS3(
|
|
|
|
- List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
|
|
|
- boolean deleteFakeDir,
|
|
|
|
- boolean quiet)
|
|
|
|
|
|
+ private void removeKeysS3(
|
|
|
|
+ List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
|
|
|
+ boolean deleteFakeDir)
|
|
throws MultiObjectDeleteException, AmazonClientException,
|
|
throws MultiObjectDeleteException, AmazonClientException,
|
|
IOException {
|
|
IOException {
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
@@ -2847,16 +2844,28 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
}
|
|
}
|
|
if (keysToDelete.isEmpty()) {
|
|
if (keysToDelete.isEmpty()) {
|
|
// exit fast if there are no keys to delete
|
|
// exit fast if there are no keys to delete
|
|
- return null;
|
|
|
|
|
|
+ return;
|
|
}
|
|
}
|
|
for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) {
|
|
for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) {
|
|
blockRootDelete(keyVersion.getKey());
|
|
blockRootDelete(keyVersion.getKey());
|
|
}
|
|
}
|
|
- DeleteObjectsResult result = null;
|
|
|
|
try {
|
|
try {
|
|
if (enableMultiObjectsDelete) {
|
|
if (enableMultiObjectsDelete) {
|
|
- result = deleteObjects(
|
|
|
|
- getRequestFactory().newBulkDeleteRequest(keysToDelete, quiet));
|
|
|
|
|
|
+ if (keysToDelete.size() <= pageSize) {
|
|
|
|
+ deleteObjects(getRequestFactory()
|
|
|
|
+ .newBulkDeleteRequest(keysToDelete));
|
|
|
|
+ } else {
|
|
|
|
+ // Multi object deletion of more than 1000 keys is not supported
|
|
|
|
+ // by s3. So we are paging the keys by page size.
|
|
|
|
+ LOG.debug("Partitioning the keys to delete as it is more than " +
|
|
|
|
+ "page size. Number of keys: {}, Page size: {}",
|
|
|
|
+ keysToDelete.size(), pageSize);
|
|
|
|
+ for (List<DeleteObjectsRequest.KeyVersion> batchOfKeysToDelete :
|
|
|
|
+ Lists.partition(keysToDelete, pageSize)) {
|
|
|
|
+ deleteObjects(getRequestFactory()
|
|
|
|
+ .newBulkDeleteRequest(batchOfKeysToDelete));
|
|
|
|
+ }
|
|
|
|
+ }
|
|
} else {
|
|
} else {
|
|
for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) {
|
|
for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) {
|
|
deleteObject(keyVersion.getKey());
|
|
deleteObject(keyVersion.getKey());
|
|
@@ -2872,7 +2881,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
throw ex;
|
|
throw ex;
|
|
}
|
|
}
|
|
noteDeleted(keysToDelete.size(), deleteFakeDir);
|
|
noteDeleted(keysToDelete.size(), deleteFakeDir);
|
|
- return result;
|
|
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -2889,7 +2897,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Invoke {@link #removeKeysS3(List, boolean, boolean)}.
|
|
|
|
|
|
+ * Invoke {@link #removeKeysS3(List, boolean)}.
|
|
* If a {@code MultiObjectDeleteException} is raised, the
|
|
* If a {@code MultiObjectDeleteException} is raised, the
|
|
* relevant statistics are updated.
|
|
* relevant statistics are updated.
|
|
*
|
|
*
|
|
@@ -2910,35 +2918,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
final boolean deleteFakeDir)
|
|
final boolean deleteFakeDir)
|
|
throws MultiObjectDeleteException, AmazonClientException,
|
|
throws MultiObjectDeleteException, AmazonClientException,
|
|
IOException {
|
|
IOException {
|
|
- removeKeys(keysToDelete, deleteFakeDir,
|
|
|
|
- true);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Invoke {@link #removeKeysS3(List, boolean, boolean)}.
|
|
|
|
- * @param keysToDelete collection of keys to delete on the s3-backend.
|
|
|
|
- * if empty, no request is made of the object store.
|
|
|
|
- * @param deleteFakeDir indicates whether this is for deleting fake dirs.
|
|
|
|
- * @param quiet should a bulk query be quiet, or should its result list
|
|
|
|
- * all deleted keys
|
|
|
|
- * @return the deletion result if a multi object delete was invoked
|
|
|
|
- * and it returned without a failure, else null.
|
|
|
|
- * @throws InvalidRequestException if the request was rejected due to
|
|
|
|
- * a mistaken attempt to delete the root directory.
|
|
|
|
- * @throws MultiObjectDeleteException one or more of the keys could not
|
|
|
|
- * be deleted in a multiple object delete operation.
|
|
|
|
- * @throws AmazonClientException amazon-layer failure.
|
|
|
|
- * @throws IOException other IO Exception.
|
|
|
|
- */
|
|
|
|
- @Retries.RetryRaw
|
|
|
|
- private DeleteObjectsResult removeKeys(
|
|
|
|
- final List<DeleteObjectsRequest.KeyVersion> keysToDelete,
|
|
|
|
- final boolean deleteFakeDir,
|
|
|
|
- final boolean quiet)
|
|
|
|
- throws MultiObjectDeleteException, AmazonClientException, IOException {
|
|
|
|
try (DurationInfo ignored = new DurationInfo(LOG, false,
|
|
try (DurationInfo ignored = new DurationInfo(LOG, false,
|
|
- "Deleting %d keys", keysToDelete.size())) {
|
|
|
|
- return removeKeysS3(keysToDelete, deleteFakeDir, quiet);
|
|
|
|
|
|
+ "Deleting %d keys", keysToDelete.size())) {
|
|
|
|
+ removeKeysS3(keysToDelete, deleteFakeDir);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|