-
Notifications
You must be signed in to change notification settings - Fork 3k
AWS: Add S3 delete tagging #4342
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
Changes from all commits
11d7023
9d3ce68
28d5e07
2e45a5a
1d1cdd4
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,6 +23,7 @@ | |
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Set; | ||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
| import java.util.stream.Collectors; | ||
| import org.apache.iceberg.aws.AwsClientFactories; | ||
|
|
@@ -40,14 +41,22 @@ | |
| import org.apache.iceberg.relocated.com.google.common.collect.SetMultimap; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Sets; | ||
| import org.apache.iceberg.util.SerializableSupplier; | ||
| import org.apache.iceberg.util.Tasks; | ||
| import org.apache.iceberg.util.ThreadPools; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
| import software.amazon.awssdk.services.s3.S3Client; | ||
| import software.amazon.awssdk.services.s3.model.Delete; | ||
| import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; | ||
| import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; | ||
| import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse; | ||
| import software.amazon.awssdk.services.s3.model.GetObjectTaggingRequest; | ||
| import software.amazon.awssdk.services.s3.model.GetObjectTaggingResponse; | ||
| import software.amazon.awssdk.services.s3.model.ObjectIdentifier; | ||
| import software.amazon.awssdk.services.s3.model.PutObjectTaggingRequest; | ||
| import software.amazon.awssdk.services.s3.model.S3Exception; | ||
| import software.amazon.awssdk.services.s3.model.Tag; | ||
| import software.amazon.awssdk.services.s3.model.Tagging; | ||
|
|
||
| /** | ||
| * FileIO implementation backed by S3. | ||
|
|
@@ -59,6 +68,7 @@ | |
| public class S3FileIO implements FileIO, SupportsBulkOperations { | ||
| private static final Logger LOG = LoggerFactory.getLogger(S3FileIO.class); | ||
| private static final String DEFAULT_METRICS_IMPL = "org.apache.iceberg.hadoop.HadoopMetricsContext"; | ||
| private static volatile ExecutorService executorService; | ||
|
|
||
| private SerializableSupplier<S3Client> s3; | ||
| private AwsProperties awsProperties; | ||
|
|
@@ -108,6 +118,18 @@ public OutputFile newOutputFile(String path) { | |
|
|
||
| @Override | ||
| public void deleteFile(String path) { | ||
| if (awsProperties.s3DeleteTags() != null && !awsProperties.s3DeleteTags().isEmpty()) { | ||
| try { | ||
| tagFileToDelete(path, awsProperties.s3DeleteTags()); | ||
| } catch (S3Exception e) { | ||
| LOG.warn("Failed to add delete tags: {} to {}", awsProperties.s3DeleteTags(), path, e); | ||
| } | ||
| } | ||
|
|
||
| if (!awsProperties.isS3DeleteEnabled()) { | ||
| return; | ||
jackye1995 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| S3URI location = new S3URI(path, awsProperties.s3BucketToAccessPointMapping()); | ||
| DeleteObjectRequest deleteRequest = | ||
| DeleteObjectRequest.builder().bucket(location.bucket()).key(location.key()).build(); | ||
|
|
@@ -125,6 +147,20 @@ public void deleteFile(String path) { | |
| */ | ||
| @Override | ||
| public void deleteFiles(Iterable<String> paths) throws BulkDeletionFailureException { | ||
| if (awsProperties.s3DeleteTags() != null && !awsProperties.s3DeleteTags().isEmpty()) { | ||
| Tasks.foreach(paths) | ||
| .noRetry() | ||
| .executeWith(executorService()) | ||
| .suppressFailureWhenFinished() | ||
| .onFailure((path, exc) -> LOG.warn("Failed to add delete tags: {} to {}", | ||
| awsProperties.s3DeleteTags(), path, exc)) | ||
| .run(path -> tagFileToDelete(path, awsProperties.s3DeleteTags())); | ||
| } | ||
|
|
||
| if (!awsProperties.isS3DeleteEnabled()) { | ||
| return; | ||
| } | ||
|
|
||
| SetMultimap<String, String> bucketToObjects = Multimaps.newSetMultimap(Maps.newHashMap(), Sets::newHashSet); | ||
| int numberOfFailedDeletions = 0; | ||
| for (String path : paths) { | ||
|
|
@@ -155,6 +191,31 @@ public void deleteFiles(Iterable<String> paths) throws BulkDeletionFailureExcept | |
| } | ||
| } | ||
|
|
||
| private void tagFileToDelete(String path, Set<Tag> deleteTags) throws S3Exception { | ||
| S3URI location = new S3URI(path, awsProperties.s3BucketToAccessPointMapping()); | ||
| String bucket = location.bucket(); | ||
| String objectKey = location.key(); | ||
| GetObjectTaggingRequest getObjectTaggingRequest = GetObjectTaggingRequest.builder() | ||
| .bucket(bucket) | ||
| .key(objectKey) | ||
| .build(); | ||
| GetObjectTaggingResponse getObjectTaggingResponse = client() | ||
| .getObjectTagging(getObjectTaggingRequest); | ||
| // Get existing tags, if any and then add the delete tags | ||
| Set<Tag> tags = Sets.newHashSet(); | ||
| if (getObjectTaggingResponse.hasTagSet()) { | ||
| tags.addAll(getObjectTaggingResponse.tagSet()); | ||
| } | ||
|
|
||
| tags.addAll(deleteTags); | ||
| PutObjectTaggingRequest putObjectTaggingRequest = PutObjectTaggingRequest.builder() | ||
rajarshisarkar marked this conversation as resolved.
Show resolved
Hide resolved
rajarshisarkar marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| .bucket(bucket) | ||
| .key(objectKey) | ||
| .tagging(Tagging.builder().tagSet(tags).build()) | ||
| .build(); | ||
| client().putObjectTagging(putObjectTaggingRequest); | ||
| } | ||
|
|
||
| private List<String> deleteObjectsInBucket(String bucket, Collection<String> objects) { | ||
| if (!objects.isEmpty()) { | ||
| List<ObjectIdentifier> objectIds = objects | ||
|
|
@@ -184,6 +245,19 @@ private S3Client client() { | |
| return client; | ||
| } | ||
|
|
||
| private ExecutorService executorService() { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We should probably shutdown the
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks, I have made the changes.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think we cannot close it, because it's a static variable and shared across FileIOs? Which is similar to the one in
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I feel not closing is fine. We can keep it as static considering parity with |
||
| if (executorService == null) { | ||
| synchronized (S3FileIO.class) { | ||
| if (executorService == null) { | ||
| executorService = ThreadPools.newWorkerPool( | ||
| "iceberg-s3fileio-delete", awsProperties.s3FileIoDeleteThreads()); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| return executorService; | ||
| } | ||
|
|
||
| @Override | ||
| public void initialize(Map<String, String> properties) { | ||
| this.awsProperties = new AwsProperties(properties); | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.