-
Notifications
You must be signed in to change notification settings - Fork 2.9k
AWS: Add SupportBulkOperation interface. S3FileIO implementation will perform a batch deletion using RemoveObjects API #4052
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
33 changes: 33 additions & 0 deletions
33
api/src/main/java/org/apache/iceberg/io/BulkDeletionFailureException.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,33 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, | ||
| * software distributed under the License is distributed on an | ||
| * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| * KIND, either express or implied. See the License for the | ||
| * specific language governing permissions and limitations | ||
| * under the License. | ||
| */ | ||
|
|
||
| package org.apache.iceberg.io; | ||
|
|
||
| public class BulkDeletionFailureException extends RuntimeException { | ||
| private final int numberFailedObjects; | ||
|
|
||
| public BulkDeletionFailureException(int numberFailedObjects) { | ||
| super(String.format("Failed to delete %d files", numberFailedObjects)); | ||
| this.numberFailedObjects = numberFailedObjects; | ||
| } | ||
|
|
||
| public int numberFailedObjects() { | ||
| return numberFailedObjects; | ||
| } | ||
| } |
30 changes: 30 additions & 0 deletions
30
api/src/main/java/org/apache/iceberg/io/SupportsBulkOperations.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,30 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, | ||
| * software distributed under the License is distributed on an | ||
| * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| * KIND, either express or implied. See the License for the | ||
| * specific language governing permissions and limitations | ||
| * under the License. | ||
| */ | ||
|
|
||
| package org.apache.iceberg.io; | ||
|
|
||
| public interface SupportsBulkOperations { | ||
| /** | ||
| * Delete the files at the given paths. | ||
| * | ||
| * @param pathsToDelete The paths to delete | ||
| * @throws BulkDeletionFailureException in | ||
| */ | ||
| void deleteFiles(Iterable<String> pathsToDelete) throws BulkDeletionFailureException; | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Normally, we will retry deletes unless there is some exception that prevents it (like FileNotFound or Unauthorized). Why not do that here? Is that a follow-up issue?
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I was thinking it would be up to the provider of the S3 client who would configure the retry policy on the client. Is that something within the scope of FileIO? If so I think that's something we could tackle in a follow-on.
Someone could use a custom AwsClientFactory . The DefaultAwsClientFactory will create an S3 client with the default retry policy which would retry on the failures mentioned in https://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/retry/PredefinedRetryPolicies.SDKDefaultRetryCondition.html.
So basically 5xx errors like service unavailable, throttling, clock-skew etc would be retried. Failures such as the bucket not existing, or unauthorized 4xx errors would not be retried by default. @jackye1995 @rdblue thoughts?
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actually, I realized the current default implementation would be inconsistent with what I just mentioned. for default, we just loop over the files and delete, we don't surface the failure at the end; if there's a failure it will be surfaced immediately. I'm still leaning towards the deleteFiles semantic being a best effort deletion attempt on all files in the list (surfacing failures at the end). So I am more leaning towards changing the default implementation.
Let me know your thoughts if you agree on this semantic for deleteFiles @rdblue @jackye1995 @danielcweeks
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I feel like we should not rely on failure/exceptions and keep delete "best effort". The reason for this is that the cleanup being done using FileIO delete (whether it's for failed/speculative tasks or other maintenance type clean) up is not a requirement for iceberg to function and will lead to spurious errors and inconsistency (as we see the normal delete path doesn't report errors in multiple implementations).
Even in a delete failure case, the required action is to run the maintenance procedures to perform orphan file cleanup, so we end up propagating errors that could result in job failure due to issues that would normally not affect a job.
I think it's fine to log failed deletes for debugging purposes, but we shouldn't rely on the deletes.
Alternatively, we could keep this bulk functionality out of the FileIO implementation and add a "feature" interface like
SupportsBulkDeletethat can be used for S3 specifically. Not all implementations of FileIO have a concept of bulk delete (S3 is the only one I'm aware of). That would allow us to have a stricter contract around deletes for cleanup procedures.I'd actually prefer the latter as it keeps FileIO cleaner and we can separate contract behaviors without introducing inconsistency. Actions may need additional feature not available (e.g. listing) that we don't want to add to the FileIO contract directly.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@danielcweeks @jackye1995 @rdblue Updated the latest so that it's contained within a separate feature interface, let me know what you think!
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It looks like we're collecting all of the failed deletions in
failedDeletionsbut not really doing anything with that. Since the interface is intended to be iterable and could potentially have a lot of delete values, I would suggest just warn logging the path and keeping a count as opposed to keeping the full list of all the paths.Other than that, this looks good.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Makes sense. Updated to logging the paths which we failed to delete, and keeping just a count of the failures.
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@danielcweeks It looks like GCS does support batch deletion operations. https://cloud.google.com/storage/docs/batch.
I don't think this necessarily changes the approach considering we want to keep FileIO simple, but let me know your thoughts here.