singhpk234 commented on a change in pull request #4342:
URL: https://github.com/apache/iceberg/pull/4342#discussion_r837151207
##########
File path: aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java
##########
@@ -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: {}", path, e);
Review comment:
[minor] should we log both deleteTag as well as path :
```suggestion
LOG.warn("Failed to add delete tags: {} to {}",
awsProperties.s3DeleteTags().toString(), path, e);
```
##########
File path: aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java
##########
@@ -303,6 +303,32 @@
*/
public static final String S3_WRITE_TAGS_PREFIX = "s3.write.tags.";
+ /**
+ * Used by {@link S3FileIO} to tag objects when deleting. When this config
is set, objects are
+ * tagged with the configured key-value pairs before deletion. This is
considered a soft-delete,
+ * because users are able to configure tag-based object lifecycle policy at
bucket level to
+ * transition objects to different tiers.
+ * <p>
+ * For more details, see
https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-lifecycle-mgmt.html
+ * <p>
+ * Example: s3.delete.tags.my_key=my_val
+ */
+ public static final String S3_DELETE_TAGS_PREFIX = "s3.delete.tags.";
+
+ /**
+ * Number of threads to use for adding delete tags to S3 objects, default to
{@link
+ * Runtime#availableProcessors()}
+ */
+ public static final String S3FILEIO_DELETE_THREADS = "s3.delete.num-threads";
+
+ /**
+ * Determines if S3FileIO delete the object when io.delete() is called,
default to true. Once
Review comment:
Nit:
```suggestion
* Determines if S3FileIO deletes the object when io.delete() is called,
default to true. Once
```
##########
File path: aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java
##########
@@ -155,6 +190,31 @@ public void deleteFiles(Iterable<String> paths) throws
BulkDeletionFailureExcept
}
}
+ private void tagFileToDelete(String path, Set<Tag> deleteTags) throws
S3Exception {
+ S3URI location = new S3URI(path);
Review comment:
can we wire in s3BucketToAccessPointMapping, to use access point in tag
manipulation.
Adding Tag's via access point is an allowed operation :
https://docs.aws.amazon.com/AmazonS3/latest/userguide/access-points-usage-examples.html#add-tag-set-ap
i.e both GetObjectTagging as well as PutObjectTagging operations are allowed
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]