Skip to content

Commit 4c83040

Browse files
authored
Merge pull request apache#9 from passaro/HADOOP-18073-deleteObject
Hadoop 18073 delete object
2 parents 6114b54 + 4fe37e2 commit 4c83040

22 files changed

+336
-273
lines changed
Lines changed: 44 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,44 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.hadoop.fs.s3a;
20+
21+
import java.util.List;
22+
23+
import software.amazon.awssdk.services.s3.model.S3Error;
24+
25+
import org.apache.hadoop.classification.InterfaceAudience;
26+
import org.apache.hadoop.classification.InterfaceStability;
27+
28+
/**
29+
* Exception raised in {@link S3AFileSystem#deleteObjects} when
30+
* one or more of the keys could not be deleted.
31+
*/
32+
@InterfaceAudience.Public
33+
@InterfaceStability.Unstable
34+
public class MultiObjectDeleteException extends RuntimeException {
35+
36+
private final List<S3Error> errors;
37+
38+
public MultiObjectDeleteException(List<S3Error> errors) {
39+
super(errors.toString());
40+
this.errors = errors;
41+
}
42+
43+
public List<S3Error> errors() { return errors; }
44+
}

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

Lines changed: 73 additions & 59 deletions
Original file line numberDiff line numberDiff line change
@@ -54,9 +54,6 @@
5454
import com.amazonaws.SdkBaseException;
5555
import com.amazonaws.services.s3.AmazonS3;
5656
import com.amazonaws.services.s3.Headers;
57-
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
58-
import com.amazonaws.services.s3.model.DeleteObjectsResult;
59-
import com.amazonaws.services.s3.model.MultiObjectDeleteException;
6057
import com.amazonaws.services.s3.model.SelectObjectContentRequest;
6158
import com.amazonaws.services.s3.model.SelectObjectContentResult;
6259
import com.amazonaws.services.s3.model.UploadPartRequest;
@@ -82,13 +79,22 @@
8279
import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
8380
import software.amazon.awssdk.awscore.exception.AwsServiceException;
8481
import software.amazon.awssdk.core.sync.RequestBody;
82+
import software.amazon.awssdk.services.s3.S3Client;
8583
import software.amazon.awssdk.services.s3.model.CopyObjectRequest;
8684
import software.amazon.awssdk.services.s3.model.CopyObjectResponse;
85+
import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest;
86+
import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse;
87+
import software.amazon.awssdk.services.s3.model.GetObjectRequest;
88+
import software.amazon.awssdk.services.s3.model.GetObjectResponse;
8789
import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
8890
import software.amazon.awssdk.services.s3.model.HeadObjectResponse;
91+
import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
92+
import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
8993
import software.amazon.awssdk.services.s3.model.ObjectCannedACL;
94+
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
9095
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
9196
import software.amazon.awssdk.services.s3.model.PutObjectResponse;
97+
import software.amazon.awssdk.services.s3.model.S3Error;
9298
import software.amazon.awssdk.services.s3.model.StorageClass;
9399
import software.amazon.awssdk.transfer.s3.CompletedCopy;
94100
import software.amazon.awssdk.transfer.s3.CompletedFileUpload;
@@ -1297,10 +1303,13 @@ public S3Client getAmazonS3V2ClientForTesting(String reason) {
12971303
* Set the client -used in mocking tests to force in a different client.
12981304
* @param client client.
12991305
*/
1300-
protected void setAmazonS3Client(AmazonS3 client) {
1301-
Preconditions.checkNotNull(client, "client");
1302-
LOG.debug("Setting S3 client to {}", client);
1303-
s3 = client;
1306+
protected void setAmazonS3Client(Pair<AmazonS3, S3Client> client) {
1307+
Preconditions.checkNotNull(client.getLeft(), "client");
1308+
Preconditions.checkNotNull(client.getRight(), "clientV2");
1309+
LOG.debug("Setting S3 client to {}", client.getLeft());
1310+
LOG.debug("Setting S3V2 client to {}", client.getRight());
1311+
s3 = client.getLeft();
1312+
s3V2 = client.getRight();
13041313

13051314
// Need to use a new TransferManager that uses the new client.
13061315
// Also, using a new TransferManager requires a new threadpool as the old
@@ -2290,7 +2299,7 @@ public CopyObjectResponse copyFile(final String srcKey,
22902299

22912300
@Override
22922301
public void removeKeys(
2293-
final List<DeleteObjectsRequest.KeyVersion> keysToDelete,
2302+
final List<ObjectIdentifier> keysToDelete,
22942303
final boolean deleteFakeDir)
22952304
throws MultiObjectDeleteException, AmazonClientException, IOException {
22962305
auditSpan.activate();
@@ -2654,7 +2663,7 @@ protected HeadObjectResponse getObjectMetadata(String key,
26542663
changeTracker.processMetadata(headObjectResponse, operation);
26552664
}
26562665
return headObjectResponse;
2657-
} catch(AwsServiceException ase) {
2666+
} catch (AwsServiceException ase) {
26582667
if (!isObjectNotFound(ase)) {
26592668
// file not found is not considered a failure of the call,
26602669
// so only switch the duration tracker to update failure
@@ -2807,8 +2816,9 @@ protected void deleteObject(String key)
28072816
incrementStatistic(OBJECT_DELETE_OBJECTS);
28082817
trackDurationOfInvocation(getDurationTrackerFactory(),
28092818
OBJECT_DELETE_REQUEST.getSymbol(),
2810-
() -> s3.deleteObject(getRequestFactory()
2811-
.newDeleteObjectRequest(key)));
2819+
() -> s3V2.deleteObject(getRequestFactory()
2820+
.newDeleteObjectRequestBuilder(key)
2821+
.build()));
28122822
return null;
28132823
});
28142824
}
@@ -2870,40 +2880,43 @@ private void blockRootDelete(String key) throws InvalidRequestException {
28702880
* @return the AWS response
28712881
* @throws MultiObjectDeleteException one or more of the keys could not
28722882
* be deleted.
2873-
* @throws AmazonClientException amazon-layer failure.
2883+
* @throws AwsServiceException amazon-layer failure.
28742884
*/
28752885
@Retries.RetryRaw
2876-
private DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteRequest)
2877-
throws MultiObjectDeleteException, AmazonClientException, IOException {
2886+
private DeleteObjectsResponse deleteObjects(DeleteObjectsRequest deleteRequest)
2887+
throws MultiObjectDeleteException, AwsServiceException, IOException {
28782888
incrementWriteOperations();
28792889
BulkDeleteRetryHandler retryHandler =
28802890
new BulkDeleteRetryHandler(createStoreContext());
2881-
int keyCount = deleteRequest.getKeys().size();
2882-
try(DurationInfo ignored =
2891+
int keyCount = deleteRequest.delete().objects().size();
2892+
try (DurationInfo ignored =
28832893
new DurationInfo(LOG, false, "DELETE %d keys",
28842894
keyCount)) {
2885-
return invoker.retryUntranslated("delete",
2886-
DELETE_CONSIDERED_IDEMPOTENT,
2887-
(text, e, r, i) -> {
2888-
// handle the failure
2889-
retryHandler.bulkDeleteRetried(deleteRequest, e);
2890-
},
2891-
// duration is tracked in the bulk delete counters
2892-
trackDurationOfOperation(getDurationTrackerFactory(),
2893-
OBJECT_BULK_DELETE_REQUEST.getSymbol(), () -> {
2895+
DeleteObjectsResponse response =
2896+
invoker.retryUntranslated("delete", DELETE_CONSIDERED_IDEMPOTENT,
2897+
(text, e, r, i) -> {
2898+
// handle the failure
2899+
retryHandler.bulkDeleteRetried(deleteRequest, e);
2900+
},
2901+
// duration is tracked in the bulk delete counters
2902+
trackDurationOfOperation(getDurationTrackerFactory(),
2903+
OBJECT_BULK_DELETE_REQUEST.getSymbol(), () -> {
28942904
incrementStatistic(OBJECT_DELETE_OBJECTS, keyCount);
2895-
return s3.deleteObjects(deleteRequest);
2896-
}));
2897-
} catch (MultiObjectDeleteException e) {
2898-
// one or more of the keys could not be deleted.
2899-
// log and rethrow
2900-
List<MultiObjectDeleteException.DeleteError> errors = e.getErrors();
2901-
LOG.debug("Partial failure of delete, {} errors", errors.size(), e);
2902-
for (MultiObjectDeleteException.DeleteError error : errors) {
2903-
LOG.debug("{}: \"{}\" - {}",
2904-
error.getKey(), error.getCode(), error.getMessage());
2905+
return s3V2.deleteObjects(deleteRequest);
2906+
}));
2907+
2908+
if (!response.errors().isEmpty()) {
2909+
// one or more of the keys could not be deleted.
2910+
// log and then throw
2911+
List<S3Error> errors = response.errors();
2912+
LOG.debug("Partial failure of delete, {} errors", errors.size());
2913+
for (S3Error error : errors) {
2914+
LOG.debug("{}: \"{}\" - {}", error.key(), error.code(), error.message());
2915+
}
2916+
throw new MultiObjectDeleteException(errors);
29052917
}
2906-
throw e;
2918+
2919+
return response;
29072920
}
29082921
}
29092922

@@ -3104,56 +3117,57 @@ public void incrementPutProgressStatistics(String key, long bytes) {
31043117
* be deleted in a multiple object delete operation.
31053118
* The number of rejected objects will be added to the metric
31063119
* {@link Statistic#FILES_DELETE_REJECTED}.
3107-
* @throws AmazonClientException other amazon-layer failure.
3120+
* @throws AwsServiceException other amazon-layer failure.
31083121
*/
31093122
@Retries.RetryRaw
31103123
private void removeKeysS3(
3111-
List<DeleteObjectsRequest.KeyVersion> keysToDelete,
3124+
List<ObjectIdentifier> keysToDelete,
31123125
boolean deleteFakeDir)
3113-
throws MultiObjectDeleteException, AmazonClientException,
3114-
IOException {
3126+
throws MultiObjectDeleteException, AwsServiceException, IOException {
31153127
if (LOG.isDebugEnabled()) {
31163128
LOG.debug("Initiating delete operation for {} objects",
31173129
keysToDelete.size());
3118-
for (DeleteObjectsRequest.KeyVersion key : keysToDelete) {
3119-
LOG.debug(" {} {}", key.getKey(),
3120-
key.getVersion() != null ? key.getVersion() : "");
3130+
for (ObjectIdentifier objectIdentifier : keysToDelete) {
3131+
LOG.debug(" {} {}", objectIdentifier.key(),
3132+
objectIdentifier.versionId() != null ? objectIdentifier.versionId() : "");
31213133
}
31223134
}
31233135
if (keysToDelete.isEmpty()) {
31243136
// exit fast if there are no keys to delete
31253137
return;
31263138
}
3127-
for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) {
3128-
blockRootDelete(keyVersion.getKey());
3139+
for (ObjectIdentifier objectIdentifier : keysToDelete) {
3140+
blockRootDelete(objectIdentifier.key());
31293141
}
31303142
try {
31313143
if (enableMultiObjectsDelete) {
31323144
if (keysToDelete.size() <= pageSize) {
31333145
deleteObjects(getRequestFactory()
3134-
.newBulkDeleteRequest(keysToDelete));
3146+
.newBulkDeleteRequestBuilder(keysToDelete)
3147+
.build());
31353148
} else {
31363149
// Multi object deletion of more than 1000 keys is not supported
31373150
// by s3. So we are paging the keys by page size.
31383151
LOG.debug("Partitioning the keys to delete as it is more than " +
31393152
"page size. Number of keys: {}, Page size: {}",
31403153
keysToDelete.size(), pageSize);
3141-
for (List<DeleteObjectsRequest.KeyVersion> batchOfKeysToDelete :
3154+
for (List<ObjectIdentifier> batchOfKeysToDelete :
31423155
Lists.partition(keysToDelete, pageSize)) {
31433156
deleteObjects(getRequestFactory()
3144-
.newBulkDeleteRequest(batchOfKeysToDelete));
3157+
.newBulkDeleteRequestBuilder(batchOfKeysToDelete)
3158+
.build());
31453159
}
31463160
}
31473161
} else {
3148-
for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) {
3149-
deleteObject(keyVersion.getKey());
3162+
for (ObjectIdentifier objectIdentifier : keysToDelete) {
3163+
deleteObject(objectIdentifier.key());
31503164
}
31513165
}
31523166
} catch (MultiObjectDeleteException ex) {
31533167
// partial delete.
31543168
// Update the stats with the count of the actual number of successful
31553169
// deletions.
3156-
int rejected = ex.getErrors().size();
3170+
int rejected = ex.errors().size();
31573171
noteDeleted(keysToDelete.size() - rejected, deleteFakeDir);
31583172
incrementStatistic(FILES_DELETE_REJECTED, rejected);
31593173
throw ex;
@@ -3186,15 +3200,15 @@ private void noteDeleted(final int count, final boolean deleteFakeDir) {
31863200
* a mistaken attempt to delete the root directory.
31873201
* @throws MultiObjectDeleteException one or more of the keys could not
31883202
* be deleted in a multiple object delete operation.
3189-
* @throws AmazonClientException amazon-layer failure.
3203+
* @throws AwsServiceException amazon-layer failure.
31903204
* @throws IOException other IO Exception.
31913205
*/
31923206
@VisibleForTesting
31933207
@Retries.RetryRaw
31943208
public void removeKeys(
3195-
final List<DeleteObjectsRequest.KeyVersion> keysToDelete,
3209+
final List<ObjectIdentifier> keysToDelete,
31963210
final boolean deleteFakeDir)
3197-
throws MultiObjectDeleteException, AmazonClientException,
3211+
throws MultiObjectDeleteException, AwsServiceException,
31983212
IOException {
31993213
try (DurationInfo ignored = new DurationInfo(LOG, false,
32003214
"Deleting %d keys", keysToDelete.size())) {
@@ -4418,22 +4432,22 @@ private PutObjectOptions putOptionsForPath(Path path) {
44184432
*/
44194433
@Retries.RetryExceptionsSwallowed
44204434
private void deleteUnnecessaryFakeDirectories(Path path) {
4421-
List<DeleteObjectsRequest.KeyVersion> keysToRemove = new ArrayList<>();
4435+
List<ObjectIdentifier> keysToRemove = new ArrayList<>();
44224436
while (!path.isRoot()) {
44234437
String key = pathToKey(path);
44244438
key = (key.endsWith("/")) ? key : (key + "/");
44254439
LOG.trace("To delete unnecessary fake directory {} for {}", key, path);
4426-
keysToRemove.add(new DeleteObjectsRequest.KeyVersion(key));
4440+
keysToRemove.add(ObjectIdentifier.builder().key(key).build());
44274441
path = path.getParent();
44284442
}
44294443
try {
44304444
removeKeys(keysToRemove, true);
4431-
} catch(AmazonClientException | IOException e) {
4445+
} catch (AwsServiceException | IOException e) {
44324446
instrumentation.errorIgnored();
44334447
if (LOG.isDebugEnabled()) {
44344448
StringBuilder sb = new StringBuilder();
4435-
for(DeleteObjectsRequest.KeyVersion kv : keysToRemove) {
4436-
sb.append(kv.getKey()).append(",");
4449+
for (ObjectIdentifier objectIdentifier : keysToRemove) {
4450+
sb.append(objectIdentifier.key()).append(",");
44374451
}
44384452
LOG.debug("While deleting keys {} ", sb.toString(), e);
44394453
}

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -191,9 +191,11 @@ public static IOException translateExceptionV2(@Nullable String operation,
191191

192192
AwsServiceException ase = (AwsServiceException) exception;
193193

194-
int status = ase.statusCode();
194+
if (ase.awsErrorDetails() != null) {
195+
message = message + ":" + ase.awsErrorDetails().errorCode();
196+
}
195197
IOException ioe;
196-
message = message + ":" + ase.awsErrorDetails().errorCode();
198+
int status = ase.statusCode();
197199
switch (status) {
198200

199201
case 403:
@@ -215,7 +217,7 @@ public static IOException translateExceptionV2(@Nullable String operation,
215217
default:
216218
// no specific exit code. Choose an IOE subclass based on the class
217219
// of the caught exception
218-
ioe = new IOException();
220+
ioe = new IOException(message);
219221
break;
220222
}
221223

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java

Lines changed: 14 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -24,8 +24,6 @@
2424
import java.util.List;
2525
import java.util.Optional;
2626

27-
import com.amazonaws.services.s3.model.DeleteObjectRequest;
28-
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
2927
import com.amazonaws.services.s3.model.ListNextBatchOfObjectsRequest;
3028
import com.amazonaws.services.s3.model.ObjectListing;
3129
import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams;
@@ -38,19 +36,22 @@
3836
import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets;
3937
import org.apache.hadoop.fs.s3a.impl.PutObjectOptions;
4038

39+
import software.amazon.awssdk.services.s3.model.AbortMultipartUploadRequest;
4140
import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
4241
import software.amazon.awssdk.services.s3.model.CompletedPart;
4342
import software.amazon.awssdk.services.s3.model.CopyObjectRequest;
43+
import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest;
44+
import software.amazon.awssdk.services.s3.model.DeleteObjectRequest;
45+
import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest;
4446
import software.amazon.awssdk.services.s3.model.GetObjectRequest;
4547
import software.amazon.awssdk.services.s3.model.HeadObjectRequest;
4648
import software.amazon.awssdk.services.s3.model.HeadObjectResponse;
49+
import software.amazon.awssdk.services.s3.model.ListMultipartUploadsRequest;
4750
import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
4851
import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
49-
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
50-
import software.amazon.awssdk.services.s3.model.AbortMultipartUploadRequest;
51-
import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest;
52-
import software.amazon.awssdk.services.s3.model.ListMultipartUploadsRequest;
5352
import software.amazon.awssdk.services.s3.model.ObjectCannedACL;
53+
import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
54+
import software.amazon.awssdk.services.s3.model.PutObjectRequest;
5455
import software.amazon.awssdk.services.s3.model.StorageClass;
5556

5657
/**
@@ -276,18 +277,18 @@ ListObjectsV2Request.Builder newListObjectsV2RequestBuilder(String key,
276277
int maxKeys);
277278

278279
/**
279-
* Create a request to delete a single object.
280+
* Create a request builder to delete a single object.
280281
* @param key object to delete
281-
* @return the request
282+
* @return the request builder.
282283
*/
283-
DeleteObjectRequest newDeleteObjectRequest(String key);
284+
DeleteObjectRequest.Builder newDeleteObjectRequestBuilder(String key);
284285

285286
/**
286-
* Bulk delete request.
287+
* Create a request builder to delete objects in bulk.
287288
* @param keysToDelete list of keys to delete.
288-
* @return the request
289+
* @return the request builder.
289290
*/
290-
DeleteObjectsRequest newBulkDeleteRequest(
291-
List<DeleteObjectsRequest.KeyVersion> keysToDelete);
291+
DeleteObjectsRequest.Builder newBulkDeleteRequestBuilder(
292+
List<ObjectIdentifier> keysToDelete);
292293

293294
}

0 commit comments

Comments
 (0)