Skip to content

Conversation

@amogh-jahagirdar
Copy link
Contributor

@amogh-jahagirdar amogh-jahagirdar commented Feb 6, 2022

Starting a draft PR for performing batch deletion for S3 objects. Related issue: #4012

This can be useful for the expire snapshots and remove orphan files operations. In this PR we update the FileIO interface and add a S3 implementation to perform the batch removal. Leaving it in draft until I add some unit tests for the batch removal implementation for S3, wanted to get some initial feedback on the code.

In other PRs we can tackle updating the actions for expiring snapshots and removing orphan files. My thoughts there are we do have a separate batching mechanism within the action implementation (for an action a user would specify a batch size (defaulting to 1). Tasks looks like it is generic and for the parameter we could partition the given input list into batches, and then we could pass in a function which accepts a list of strings for performing the batch deletion. This would only be done in the case the batch size is greater than 1. Any thoughts?

@szehon-ho @dramaticlly @jackye1995

Copy link
Contributor

@dramaticlly dramaticlly left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you @amogh-jahagirdar , left some nitpicky comments

Copy link
Contributor

@jackye1995 jackye1995 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for working on this! I think we need some integration tests especially for:
(1) verify the limit of S3 batch delete, if 1000 batch size is good for common usage
(2) do we need to dedupe on client side for the paths input. Maybe we can just ignore, and then catch object not found error that is more efficient.

@amogh-jahagirdar
Copy link
Contributor Author

amogh-jahagirdar commented Feb 17, 2022

Sorry for the delay folks. Few updates:

1.) updated the PR with some integration tests and more unit tests.

2.) The deletion batch size is configurable through s3.delete.batch-size.

3.) The default is 250 instead of 1000. Tbh I think some rigorous benchmarking should be done here. I set to 250 mostly mimicking the similar change done in hadoop-aws apache/hadoop@56dee66 which also used to perform batch deletions in 1000 until encountering major throttling issues. For reference if there are N keys in a batch, this will uses N requests in your throughput calculation done by S3 for controlling throttling. S3 limitations are 3500 TPS per prefix. So if we did 1000, in the worst case where most of the keys fall in the same prefix (if somebody has a hive-like file structure) then we would easily hit this limitation easily. If prefixes are better distributed we could get more throughput, but don't think we should rely on this assumption.

4.) I do not think for the S3 case we need to worry about any de-duping. If the same key is passed in DeleteObjects multiple times, there are no failures. Also after the delete marker is set on the object, if deleteObjects is called later, the call still does not fail (DeleteObjects does not fail if the passed in keys do not exist. It's a no-op)

@dramaticlly @jackye1995 Let me know your thoughts!

Copy link
Contributor

@dramaticlly dramaticlly left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM, thank for the changes

Copy link
Contributor

@jackye1995 jackye1995 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

overall this looks good to me!

@amogh-jahagirdar amogh-jahagirdar force-pushed the s3-bulk-delete branch 2 times, most recently from fd1c5c3 to 7490989 Compare February 19, 2022 01:10
@amogh-jahagirdar amogh-jahagirdar changed the title Add deleteFiles to FileIO. S3FileIO implementation will perform a batch deletion using RemoveObjects API API: Add deleteFiles to FileIO. S3FileIO implementation will perform a batch deletion using RemoveObjects API Feb 19, 2022
@amogh-jahagirdar
Copy link
Contributor Author

amogh-jahagirdar commented Feb 19, 2022

For some reason I had the "core" label on the commit message, but this is API and AWS module updates. Updated the commit message to reflect this.

@amogh-jahagirdar amogh-jahagirdar force-pushed the s3-bulk-delete branch 3 times, most recently from 4ca27c3 to 77271fd Compare February 19, 2022 01:21
Copy link
Contributor

@jackye1995 jackye1995 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Mostly looks good to me, just a few nitpicking. Thanks for working for this!

String objectKey = location.key();
Set<String> objectsInBucket = bucketToObjects.get(bucket);
if (objectsInBucket.size() == awsProperties.s3FileIoDeleteBatchSize()) {
List<String> failedDeletionsForBatch = deleteObjectsInBucket(bucket, objectsInBucket);
Copy link
Contributor

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?

Copy link
Contributor Author

@amogh-jahagirdar amogh-jahagirdar Feb 24, 2022

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?

Copy link
Contributor Author

@amogh-jahagirdar amogh-jahagirdar Mar 1, 2022

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

Copy link
Contributor

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 SupportsBulkDelete that 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.

Copy link
Contributor Author

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!

Copy link
Contributor

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 failedDeletions but 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.

Copy link
Contributor Author

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.

Copy link
Contributor Author

@amogh-jahagirdar amogh-jahagirdar Mar 15, 2022

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.

@amogh-jahagirdar amogh-jahagirdar force-pushed the s3-bulk-delete branch 2 times, most recently from 7406df4 to 4ac850a Compare February 24, 2022 18:34
@danielcweeks danielcweeks self-requested a review March 1, 2022 18:39
@amogh-jahagirdar amogh-jahagirdar changed the title API: Add deleteFiles to FileIO. S3FileIO implementation will perform a batch deletion using RemoveObjects API AWS: Add SupportBulkOperation interface. S3FileIO implementation will perform a batch deletion using RemoveObjects API Mar 9, 2022
@amogh-jahagirdar
Copy link
Contributor Author

@rdblue @danielcweeks @jackye1995 Sorry for the delay, I've updated the PR. Some fundamental changes from before:

1.) No deleteFiles on FileIO.

2.) public BulkOperation interface which is currently only implemented for S3FileIO. batch deletion exception are only for the S3 case (package private).

It does make it a bit cumbersome when we want to do batch deletion in removing orphan files, because we would have to do instanceOf check.

@amogh-jahagirdar amogh-jahagirdar force-pushed the s3-bulk-delete branch 5 times, most recently from 514af24 to 6c3bbd0 Compare March 15, 2022 04:31
… perform a batch deletion using RemoveObjects API
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants