Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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;
}
}
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;
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import java.security.MessageDigest;
import java.security.SecureRandom;
import java.util.Base64;
import java.util.List;
import java.util.UUID;
import javax.crypto.KeyGenerator;
import javax.crypto.SecretKey;
Expand All @@ -36,6 +37,7 @@
import org.apache.iceberg.aws.AwsProperties;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.io.OutputFile;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
Expand Down Expand Up @@ -67,6 +69,7 @@ public class TestS3FileIOIntegration {
private static byte[] contentBytes;
private static String content;
private static String kmsKeyArn;
private static int deletionBatchSize;
private String objectKey;
private String objectUri;

Expand All @@ -78,6 +81,7 @@ public static void beforeClass() {
bucketName = AwsIntegTestUtil.testBucketName();
prefix = UUID.randomUUID().toString();
contentBytes = new byte[1024 * 1024 * 10];
deletionBatchSize = 3;
content = new String(contentBytes, StandardCharsets.UTF_8);
kmsKeyArn = kms.createKey().keyMetadata().arn();
}
Expand Down Expand Up @@ -208,8 +212,49 @@ public void testClientFactorySerialization() throws Exception {
validateRead(fileIO2);
}

@Test
public void testDeleteFilesMultipleBatches() throws Exception {
S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, getDeletionTestProperties());
testDeleteFiles(deletionBatchSize * 2, s3FileIO);
}

@Test
public void testDeleteFilesLessThanBatchSize() throws Exception {
S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, getDeletionTestProperties());
testDeleteFiles(deletionBatchSize - 1, s3FileIO);
}

@Test
public void testDeleteFilesSingleBatchWithRemainder() throws Exception {
S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, getDeletionTestProperties());
testDeleteFiles(5, s3FileIO);
}

private AwsProperties getDeletionTestProperties() {
AwsProperties properties = new AwsProperties();
properties.setS3FileIoDeleteBatchSize(deletionBatchSize);
return properties;
}

private void testDeleteFiles(int numObjects, S3FileIO s3FileIO) throws Exception {
List<String> paths = Lists.newArrayList();
for (int i = 1; i <= numObjects; i++) {
String deletionKey = objectKey + "-deletion-" + i;
write(s3FileIO, String.format("s3://%s/%s", bucketName, deletionKey));
paths.add(String.format("s3://%s/%s", bucketName, deletionKey));
}
s3FileIO.deleteFiles(paths);
for (String path : paths) {
Assert.assertFalse(s3FileIO.newInputFile(path).exists());
}
}

private void write(S3FileIO s3FileIO) throws Exception {
OutputFile outputFile = s3FileIO.newOutputFile(objectUri);
write(s3FileIO, objectUri);
}

private void write(S3FileIO s3FileIO, String uri) throws Exception {
OutputFile outputFile = s3FileIO.newOutputFile(uri);
OutputStream outputStream = outputFile.create();
IoUtils.copy(new ByteArrayInputStream(contentBytes), outputStream);
outputStream.close();
Expand Down
35 changes: 35 additions & 0 deletions aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java
Original file line number Diff line number Diff line change
Expand Up @@ -172,6 +172,25 @@ public class AwsProperties implements Serializable {
public static final String S3_CHECKSUM_ENABLED = "s3.checksum-enabled";
public static final boolean S3_CHECKSUM_ENABLED_DEFAULT = false;

/**
* Configure the batch size used when deleting multiple files from a given S3 bucket
*/
public static final String S3FILEIO_DELETE_BATCH_SIZE = "s3.delete.batch-size";

/**
* Default batch size used when deleting files.
* <p>
* Refer to https://github.com/apache/hadoop/commit/56dee667707926f3796c7757be1a133a362f05c9
* for more details on why this value was chosen.
*/
public static final int S3FILEIO_DELETE_BATCH_SIZE_DEFAULT = 250;

/**
* Max possible batch size for deletion. Currently, a max of 1000 keys can be deleted in one batch.
* https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html
*/
public static final int S3FILEIO_DELETE_BATCH_SIZE_MAX = 1000;

/**
* DynamoDB table name for {@link DynamoDbCatalog}
*/
Expand Down Expand Up @@ -236,6 +255,7 @@ public class AwsProperties implements Serializable {
private String s3FileIoSseMd5;
private int s3FileIoMultipartUploadThreads;
private int s3FileIoMultiPartSize;
private int s3FileIoDeleteBatchSize;
private double s3FileIoMultipartThresholdFactor;
private String s3fileIoStagingDirectory;
private ObjectCannedACL s3FileIoAcl;
Expand All @@ -256,6 +276,7 @@ public AwsProperties() {
this.s3FileIoMultipartUploadThreads = Runtime.getRuntime().availableProcessors();
this.s3FileIoMultiPartSize = S3FILEIO_MULTIPART_SIZE_DEFAULT;
this.s3FileIoMultipartThresholdFactor = S3FILEIO_MULTIPART_THRESHOLD_FACTOR_DEFAULT;
this.s3FileIoDeleteBatchSize = S3FILEIO_DELETE_BATCH_SIZE_DEFAULT;
this.s3fileIoStagingDirectory = System.getProperty("java.io.tmpdir");
this.isS3ChecksumEnabled = S3_CHECKSUM_ENABLED_DEFAULT;

Expand Down Expand Up @@ -310,6 +331,12 @@ public AwsProperties(Map<String, String> properties) {
this.isS3ChecksumEnabled = PropertyUtil.propertyAsBoolean(properties, S3_CHECKSUM_ENABLED,
S3_CHECKSUM_ENABLED_DEFAULT);

this.s3FileIoDeleteBatchSize = PropertyUtil.propertyAsInt(properties, S3FILEIO_DELETE_BATCH_SIZE,
S3FILEIO_DELETE_BATCH_SIZE_DEFAULT);
Preconditions.checkArgument(s3FileIoDeleteBatchSize > 0 &&
s3FileIoDeleteBatchSize <= S3FILEIO_DELETE_BATCH_SIZE_MAX,
String.format("Deletion batch size must be between 1 and %s", S3FILEIO_DELETE_BATCH_SIZE_MAX));

this.dynamoDbTableName = PropertyUtil.propertyAsString(properties, DYNAMODB_TABLE_NAME,
DYNAMODB_TABLE_NAME_DEFAULT);
}
Expand All @@ -326,6 +353,14 @@ public String s3FileIoSseKey() {
return s3FileIoSseKey;
}

public int s3FileIoDeleteBatchSize() {
return s3FileIoDeleteBatchSize;
}

public void setS3FileIoDeleteBatchSize(int deleteBatchSize) {
this.s3FileIoDeleteBatchSize = deleteBatchSize;
}

public void setS3FileIoSseKey(String sseKey) {
this.s3FileIoSseKey = sseKey;
}
Expand Down
77 changes: 76 additions & 1 deletion aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,23 +19,36 @@

package org.apache.iceberg.aws.s3;

import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
import org.apache.iceberg.aws.AwsClientFactories;
import org.apache.iceberg.aws.AwsProperties;
import org.apache.iceberg.common.DynConstructors;
import org.apache.iceberg.io.BulkDeletionFailureException;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.io.OutputFile;
import org.apache.iceberg.io.SupportsBulkOperations;
import org.apache.iceberg.metrics.MetricsContext;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.relocated.com.google.common.collect.Multimaps;
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.PropertyUtil;
import org.apache.iceberg.util.SerializableSupplier;
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.ObjectIdentifier;
import software.amazon.awssdk.services.s3.model.Tag;

/**
Expand All @@ -45,7 +58,7 @@
* URIs with schemes s3a, s3n, https are also treated as s3 file paths.
* Using this FileIO with other schemes will result in {@link org.apache.iceberg.exceptions.ValidationException}.
*/
public class S3FileIO implements FileIO {
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";

Expand Down Expand Up @@ -105,6 +118,68 @@ public void deleteFile(String path) {
client().deleteObject(deleteRequest);
}

/**
* Deletes the given paths in a batched manner.
* <p>
* The paths are grouped by bucket, and deletion is triggered when we either reach the configured batch size
* or have a final remainder batch for each bucket.
*
* @param paths paths to delete
*/
@Override
public void deleteFiles(Iterable<String> paths) throws BulkDeletionFailureException {
SetMultimap<String, String> bucketToObjects = Multimaps.newSetMultimap(Maps.newHashMap(), Sets::newHashSet);
int numberOfFailedDeletions = 0;
for (String path : paths) {
S3URI location = new S3URI(path);
String bucket = location.bucket();
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.

numberOfFailedDeletions += failedDeletionsForBatch.size();
failedDeletionsForBatch.forEach(failedPath -> LOG.warn("Failed to delete object at path {}", failedPath));
bucketToObjects.removeAll(bucket);
}
bucketToObjects.get(bucket).add(objectKey);
}

// Delete the remainder
for (Map.Entry<String, Collection<String>> bucketToObjectsEntry : bucketToObjects.asMap().entrySet()) {
final String bucket = bucketToObjectsEntry.getKey();
final Collection<String> objects = bucketToObjectsEntry.getValue();
List<String> failedDeletions = deleteObjectsInBucket(bucket, objects);
failedDeletions.forEach(failedPath -> LOG.warn("Failed to delete object at path {}", failedPath));
numberOfFailedDeletions += failedDeletions.size();
}

if (numberOfFailedDeletions > 0) {
throw new BulkDeletionFailureException(numberOfFailedDeletions);
}
}

private List<String> deleteObjectsInBucket(String bucket, Collection<String> objects) {
if (!objects.isEmpty()) {
List<ObjectIdentifier> objectIds = objects
.stream()
.map(objectKey -> ObjectIdentifier.builder().key(objectKey).build())
.collect(Collectors.toList());
DeleteObjectsRequest deleteObjectsRequest = DeleteObjectsRequest.builder()
.bucket(bucket)
.delete(Delete.builder().objects(objectIds).build())
.build();
DeleteObjectsResponse response = client().deleteObjects(deleteObjectsRequest);
if (response.hasErrors()) {
return response.errors()
.stream()
.map(error -> String.format("s3://%s/%s", bucket, error.key()))
.collect(Collectors.toList());
}
}

return Lists.newArrayList();
}

private S3Client client() {
if (client == null) {
client = s3.get();
Expand Down
20 changes: 20 additions & 0 deletions aws/src/test/java/org/apache/iceberg/aws/TestAwsProperties.java
Original file line number Diff line number Diff line change
Expand Up @@ -97,4 +97,24 @@ public void testS3MultipartThresholdFactorLessThanOne() {
() -> new AwsProperties(map));
}

@Test
public void testS3FileIoDeleteBatchSizeTooLarge() {
Map<String, String> map = Maps.newHashMap();
map.put(AwsProperties.S3FILEIO_DELETE_BATCH_SIZE, "2000");
AssertHelpers.assertThrows("should not accept batch size greater than 1000",
IllegalArgumentException.class,
"Deletion batch size must be between 1 and 1000",
() -> new AwsProperties(map));
}

@Test
public void testS3FileIoDeleteBatchSizeTooSmall() {
Map<String, String> map = Maps.newHashMap();
map.put(AwsProperties.S3FILEIO_DELETE_BATCH_SIZE, "0");
AssertHelpers.assertThrows("should not accept batch size less than 1",
IllegalArgumentException.class,
"Deletion batch size must be between 1 and 1000",
() -> new AwsProperties(map));
}

}
Loading