[
https://issues.apache.org/jira/browse/HADOOP-15628?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16596852#comment-16596852
]
Steve Jacobs commented on HADOOP-15628:
---------------------------------------
I think this is fixed in 3.1, at least from my look at the code. I don't have
an ability to test this from hive currently which is where we were seeing the
error.
On 8/29/18, 11:01 AM, "Steve Loughran (JIRA)" <[email protected]> wrote:
[
https://na01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FHADOOP-15628%3Fpage%3Dcom.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel%26focusedCommentId%3D16596602%23comment-16596602&data=02%7C01%7Csjacobs%40battelleecology.org%7C7cb8f13c8b334b1333a408d60dd100e9%7Cf44d2ab390994d85998610165a8619f5%7C0%7C0%7C636711588646499813&sdata=U5uohXfg71E2F9XDn0qrfhm%2Fhc0zu%2BgKBrxyyuLZmK0%3D&reserved=0
]
Steve Loughran commented on HADOOP-15628:
-----------------------------------------
[~steveatbat]: if you can do a patch for this ASAP we can still get it in
to 3.2. I don't see how it can be tested, other than regression testing & review
> S3A Filesystem does not check return from AmazonS3Client deleteObjects
> ----------------------------------------------------------------------
>
> Key: HADOOP-15628
> URL:
https://na01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FHADOOP-15628&data=02%7C01%7Csjacobs%40battelleecology.org%7C7cb8f13c8b334b1333a408d60dd100e9%7Cf44d2ab390994d85998610165a8619f5%7C0%7C0%7C636711588646499813&sdata=JZTQFE%2BG788%2FE%2BFEE283jv9Ieig4IWCdxKyLKahTJVk%3D&reserved=0
> Project: Hadoop Common
> Issue Type: Bug
> Components: fs/s3
> Affects Versions: 2.9.1, 2.8.4, 3.1.1, 3.0.3
> Environment: Hadoop 3.0.2 / Hadoop 2.8.3
> Hive 2.3.2 / Hive 2.3.3 / Hive 3.0.0
> Non-AWS S3 implementation
> Reporter: Steve Jacobs
> Priority: Minor
>
> Deletes in S3A that use the Multi-Delete functionality in the Amazon S3
api do not check to see if all objects have been succesfully delete. In the
event of a failure, the api will still return a 200 OK (which isn't checked
currently):
> [Delete Code from Hadoop
2.8|https://github.com/apache/hadoop/blob/a0da1ec01051108b77f86799dd5e97563b2a3962/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java#L574]
> {code:java}
> if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) {
> DeleteObjectsRequest deleteRequest =
> new DeleteObjectsRequest(bucket).withKeys(keysToDelete);
> s3.deleteObjects(deleteRequest);
> statistics.incrementWriteOps(1);
> keysToDelete.clear();
> }
> {code}
> This should be converted to use the DeleteObjectsResult class from the
S3Client:
> [Amazon Code
Example|https://docs.aws.amazon.com/AmazonS3/latest/dev/DeletingMultipleObjectsUsingJava.htm]
> {code:java}
> // Verify that the objects were deleted successfully.
> DeleteObjectsResult delObjRes =
s3Client.deleteObjects(multiObjectDeleteRequest); int successfulDeletes =
delObjRes.getDeletedObjects().size();
> System.out.println(successfulDeletes + " objects successfully deleted.");
> {code}
> Bucket policies can be misconfigured, and deletes will fail without
warning by S3A clients.
>
>
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
> S3A Filesystem does not check return from AmazonS3Client deleteObjects
> ----------------------------------------------------------------------
>
> Key: HADOOP-15628
> URL: https://issues.apache.org/jira/browse/HADOOP-15628
> Project: Hadoop Common
> Issue Type: Sub-task
> Components: fs/s3
> Affects Versions: 2.9.1, 2.8.4, 3.1.1, 3.0.3
> Environment: Hadoop 3.0.2 / Hadoop 2.8.3
> Hive 2.3.2 / Hive 2.3.3 / Hive 3.0.0
> Non-AWS S3 implementation
> Reporter: Steve Jacobs
> Priority: Minor
>
> Deletes in S3A that use the Multi-Delete functionality in the Amazon S3 api
> do not check to see if all objects have been succesfully delete. In the event
> of a failure, the api will still return a 200 OK (which isn't checked
> currently):
> [Delete Code from Hadoop
> 2.8|https://github.com/apache/hadoop/blob/a0da1ec01051108b77f86799dd5e97563b2a3962/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java#L574]
>
> {code:java}
> if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) {
> DeleteObjectsRequest deleteRequest =
> new DeleteObjectsRequest(bucket).withKeys(keysToDelete);
> s3.deleteObjects(deleteRequest);
> statistics.incrementWriteOps(1);
> keysToDelete.clear();
> }
> {code}
> This should be converted to use the DeleteObjectsResult class from the
> S3Client:
> [Amazon Code
> Example|https://docs.aws.amazon.com/AmazonS3/latest/dev/DeletingMultipleObjectsUsingJava.htm]
> {code:java}
> // Verify that the objects were deleted successfully.
> DeleteObjectsResult delObjRes =
> s3Client.deleteObjects(multiObjectDeleteRequest); int successfulDeletes =
> delObjRes.getDeletedObjects().size();
> System.out.println(successfulDeletes + " objects successfully deleted.");
> {code}
> Bucket policies can be misconfigured, and deletes will fail without warning
> by S3A clients.
>
>
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]