diff --git a/lib/trino-hdfs/src/main/java/io/trino/hdfs/s3/TrinoS3FileSystem.java b/lib/trino-hdfs/src/main/java/io/trino/hdfs/s3/TrinoS3FileSystem.java index 83701adecc16..fa6c7fa11967 100644 --- a/lib/trino-hdfs/src/main/java/io/trino/hdfs/s3/TrinoS3FileSystem.java +++ b/lib/trino-hdfs/src/main/java/io/trino/hdfs/s3/TrinoS3FileSystem.java @@ -63,6 +63,7 @@ import com.amazonaws.services.s3.model.KMSEncryptionMaterialsProvider; import com.amazonaws.services.s3.model.ListObjectsV2Request; import com.amazonaws.services.s3.model.ListObjectsV2Result; +import com.amazonaws.services.s3.model.MultiObjectDeleteException; import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.PartETag; import com.amazonaws.services.s3.model.PutObjectRequest; @@ -182,6 +183,7 @@ import static java.util.Objects.requireNonNull; import static java.util.concurrent.Executors.newCachedThreadPool; import static java.util.concurrent.TimeUnit.SECONDS; +import static java.util.stream.Collectors.joining; import static java.util.stream.Collectors.toList; import static org.apache.hadoop.fs.FSExceptionMessages.CANNOT_SEEK_PAST_EOF; import static org.apache.hadoop.fs.FSExceptionMessages.NEGATIVE_SEEK; @@ -778,6 +780,12 @@ public void deleteFiles(Collection paths) deletePaths(currentBatch); } } + catch (MultiObjectDeleteException e) { + String errors = e.getErrors().stream() + .map(error -> format("key: %s, versionId: %s, code: %s, message: %s", error.getKey(), error.getVersionId(), error.getCode(), error.getMessage())) + .collect(joining(", ")); + throw new IOException("Exception while batch deleting paths: %s".formatted(errors), e); + } catch (AmazonClientException e) { throw new IOException("Exception while batch deleting paths", e); }