From 507d7c65865cc0e933143f2e23a9ffcddd219daa Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 24 Jan 2024 16:29:37 +0000 Subject: [PATCH 01/19] HADOOP-18679. Add API for bulk/paged object deletion A more minimal design that is easier to use and implement. Caller creates a BulkOperation; they get the page size of it and then submit batches to delete of less than that size. The outcome of each call contains a list of failures. S3A implementation to show how straightforward it is. Even with the single entry page size, it is still more efficient to use this as it doesn't try to recreate a parent dir or perform any probes to see if it is a directory: it maps straight to a DELETE call. Change-Id: Ibe8737e7933fe03d39070e7138a710b50c3d60c2 --- .../java/org/apache/hadoop/fs/BulkDelete.java | 143 ++++++++++++++++++ .../apache/hadoop/fs/BulkDeleteSource.java | 55 +++++++ .../hadoop/fs/CommonPathCapabilities.java | 6 + .../fs/statistics/StoreStatisticNames.java | 3 + .../apache/hadoop/fs/s3a/S3AFileSystem.java | 65 +++++++- .../org/apache/hadoop/fs/s3a/Statistic.java | 4 + .../fs/s3a/impl/BulkDeleteOperation.java | 117 ++++++++++++++ .../s3a/impl/MultiObjectDeleteException.java | 20 ++- 8 files changed, 406 insertions(+), 7 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java new file mode 100644 index 0000000000000..4f55b25d9c7ee --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java @@ -0,0 +1,143 @@ +/* + * 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.hadoop.fs; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +/** + * API for bulk deletion of objects/files, + * but not directories. + * After use, call {@code close()} to release any resources and + * to guarantee store IOStatistics are updated. + *

+ * Callers MUST have no expectation that parent directories will exist after the + * operation completes; if an object store needs to explicitly look for and create + * directory markers, that step will be omitted. + *

+ *

+ * Be aware that on some stores (AWS S3) each object listed in a bulk delete counts + * against the write IOPS limit; large page sizes are counterproductive here. + * @see HADOOP-16823. + * Large DeleteObject requests are their own Thundering Herd + *

+ */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public interface BulkDelete extends IOStatisticsSource, Closeable { + + /** + * The maximum number of objects/files to delete in a single request. + * @return a number greater than or equal to zero. + */ + int pageSize(); + + /** + * Base path of a bulk delete operation. + * All paths submitted in {@link #bulkDelete(List)} must be under this path. + */ + Path basePath(); + + /** + * Delete a list of files/objects. + *

+ * @param paths list of paths which must be absolute and under the base path. + * provided in {@link #basePath()}. + * @throws IOException IO problems including networking, authentication and more. + * @throws IllegalArgumentException if a path argument is invalid. + */ + BulkDeleteOutcome bulkDelete(List paths) + throws IOException, IllegalArgumentException; + + /** + * The outcome: a list of paths which failed to delete. + * An empty list means all files were successfully deleted. + * There are no guarantees about the ordering of the list. + * Reasons for failures are not provided. + * File Not Found is not a failure. + */ + class BulkDeleteOutcome { + + /** + * List of paths which failed to delete. + */ + private final List failures; + + /** + * Constructor. + * @param failures list of failures. This must be non-null. + */ + public BulkDeleteOutcome(final List failures) { + this.failures = failures; + } + + /** + * Get the list of failures. + * @return a possibly empty list of failures. + */ + public List getFailures() { + return failures; + } + } + + class BulkDeleteOutcomeElement { + private final Path path; + private final String error; + private final Exception exception; + + public BulkDeleteOutcomeElement( + final Path path, + final String error, + final Exception exception) { + this.path = path; + this.error = error; + this.exception = exception; + } + + public Path getPath() { + return path; + } + + public String getError() { + return error; + } + + public Exception getException() { + return exception; + } + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java new file mode 100644 index 0000000000000..9d7dc66b41842 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java @@ -0,0 +1,55 @@ +/* + * 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.hadoop.fs; + +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Interface for bulk deletion. + * Filesystems which support bulk deletion should implement this interface + * and MUST also declare their support in the path capability + * {@link CommonPathCapabilities#BULK_DELETE}. + * Exporting the interface does not guarantee that the operation is supported; + * returning a {@link BulkDelete} object from the call {@link #createBulkDelete(Path)} + * is. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public interface BulkDeleteSource { + + /** + * Create a bulk delete operation. + * There is no network IO at this point, simply the creation of + * a bulk delete object. + * A path must be supplied to ensure that on viewfs and similar filesystems, + * @param path path to delete under. + * @return the bulk delete. + * @throws UnsupportedOperationException the filesystem does not support delete under that path. + * @throws IllegalArgumentException path not valid. + * @throws IOException problems resolving paths + */ + default BulkDelete createBulkDelete(Path path) + throws UnsupportedOperationException, IllegalArgumentException, IOException { + throw new UnsupportedOperationException("Bulk delete not supported"); + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonPathCapabilities.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonPathCapabilities.java index 9ec07cbe966e9..2005f0ae3be31 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonPathCapabilities.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonPathCapabilities.java @@ -181,4 +181,10 @@ private CommonPathCapabilities() { */ public static final String DIRECTORY_LISTING_INCONSISTENT = "fs.capability.directory.listing.inconsistent"; + + /** + * Capability string to probe for bulk delete: {@value}. + */ + public static final String BULK_DELETE = "fs.capability.bulk.delete"; + } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java index 19ee9d1414ecf..d7d4bc45dba08 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java @@ -46,6 +46,9 @@ public final class StoreStatisticNames { /** {@value}. */ public static final String OP_APPEND = "op_append"; + /** {@value}. */ + public static final String OP_BULK_DELETE = "op_bulk-delete"; + /** {@value}. */ public static final String OP_COPY_FROM_LOCAL_FILE = "op_copy_from_local_file"; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 0e2ae0f74dd0a..5fa910632c1cb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -103,6 +103,8 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.BulkDelete; +import org.apache.hadoop.fs.BulkDeleteSource; import org.apache.hadoop.fs.CommonPathCapabilities; import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.CreateFlag; @@ -120,6 +122,7 @@ import org.apache.hadoop.fs.s3a.commit.magic.InMemoryMagicCommitTracker; import org.apache.hadoop.fs.s3a.impl.AWSCannedACL; import org.apache.hadoop.fs.s3a.impl.AWSHeaders; +import org.apache.hadoop.fs.s3a.impl.BulkDeleteOperation; import org.apache.hadoop.fs.s3a.impl.BulkDeleteRetryHandler; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.ConfigurationHelper; @@ -283,7 +286,8 @@ @InterfaceStability.Evolving public class S3AFileSystem extends FileSystem implements StreamCapabilities, AWSPolicyProvider, DelegationTokenProvider, IOStatisticsSource, - AuditSpanSource, ActiveThreadSpanSource { + AuditSpanSource, ActiveThreadSpanSource, + BulkDeleteSource { /** * Default blocksize as used in blocksize and FS status queries. @@ -3403,6 +3407,13 @@ private void removeKeysS3( // exit fast if there are no keys to delete return; } + if (keysToDelete.size() == 1) { + // single object is a single delete call. + // this is more informative in server logs and may be more efficient.. + deleteObject(keysToDelete.get(0).key()); + noteDeleted(1, deleteFakeDir); + return; + } for (ObjectIdentifier objectIdentifier : keysToDelete) { blockRootDelete(objectIdentifier.key()); } @@ -5481,7 +5492,11 @@ public boolean hasPathCapability(final Path path, final String capability) case STORE_CAPABILITY_DIRECTORY_MARKER_AWARE: return true; - // multi object delete flag + // this is always true, even if multi object + // delete is disabled -the page size is simply reduced to 1. + case CommonPathCapabilities.BULK_DELETE: + return true; + case ENABLE_MULTI_DELETE: return enableMultiObjectsDelete; @@ -5768,4 +5783,50 @@ public boolean isMultipartUploadEnabled() { return isMultipartUploadEnabled; } + @Override + public BulkDelete createBulkDelete(final Path path) + throws IllegalArgumentException, IOException { + + final Path p = makeQualified(path); + final AuditSpanS3A span = createSpan("bulkdelete", p.toString(), null); + return new BulkDeleteOperation( + createStoreContext(), + createBulkDeleteCallbacks(span), + p, + enableMultiObjectsDelete ? 1: pageSize, + span); + } + + /** + * Override point for mocking. + * @param span span for operations. + * @return an instance of the Bulk Delette callbacks. + */ + protected BulkDeleteOperation.BulkDeleteOperationCallbacks createBulkDeleteCallbacks( + final AuditSpanS3A span) { + return new BulkDeleteOperationCallbacksImpl(span); + } + + /** + * Callbacks for the bulk delete operation. + */ + protected class BulkDeleteOperationCallbacksImpl implements + BulkDeleteOperation.BulkDeleteOperationCallbacks { + + /** span for operations. */ + private final AuditSpan span; + + protected BulkDeleteOperationCallbacksImpl(AuditSpan span) { + this.span = span; + } + + @Override + @Retries.RetryTranslated + public void bulkDelete(final List keys) + throws MultiObjectDeleteException, IOException, IllegalArgumentException { + span.activate(); + once("bulkDelete", "", () -> + S3AFileSystem.this.removeKeys(keys, false)); + } + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java index ce3af3de803a4..7102623996e95 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java @@ -103,6 +103,10 @@ public enum Statistic { StoreStatisticNames.OP_ACCESS, "Calls of access()", TYPE_DURATION), + INVOCATION_BULK_DELETE( + StoreStatisticNames.OP_BULK_DELETE, + "Calls of bulk delete()", + TYPE_COUNTER), INVOCATION_COPY_FROM_LOCAL_FILE( StoreStatisticNames.OP_COPY_FROM_LOCAL_FILE, "Calls of copyFromLocalFile()", diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java new file mode 100644 index 0000000000000..f4d0fea16e7b9 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java @@ -0,0 +1,117 @@ +/* + * 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.hadoop.fs.s3a.impl; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; + +import org.apache.hadoop.fs.BulkDelete; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.Retries; +import org.apache.hadoop.fs.store.audit.AuditSpan; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.util.Preconditions.checkArgument; + +/** + * S3A Implementation of the {@link BulkDelete} interface. + */ +public class BulkDeleteOperation extends AbstractStoreOperation implements BulkDelete { + + private final BulkDeleteOperationCallbacks callbacks; + + private final Path basePath; + + private final int pageSize; + + public BulkDeleteOperation(final StoreContext storeContext, + final BulkDeleteOperationCallbacks callbacks, + final Path basePath, + final int pageSize, + final AuditSpan span) { + super(storeContext, span); + this.callbacks = requireNonNull(callbacks); + this.basePath = requireNonNull(basePath); + checkArgument(pageSize > 0, "Page size must be greater than 0"); + this.pageSize = pageSize; + } + + @Override + public int pageSize() { + return pageSize; + } + + @Override + public Path basePath() { + return basePath; + } + + @Override + public BulkDeleteOutcome bulkDelete(final List paths) + throws IOException, IllegalArgumentException { + requireNonNull(paths); + checkArgument(paths.size() <= pageSize, + "Number of paths (%d) is larger than the page size (%d)", paths.size(), pageSize); + + final List objects = paths.stream().map(p -> { + checkArgument(p.isAbsolute(), "Path %s is not absolute", p); + final String k = getStoreContext().pathToKey(p); + return ObjectIdentifier.builder().key(k).build(); + }).collect(Collectors.toList()); + try { + callbacks.bulkDelete(objects); + } catch (MultiObjectDeleteException e) { + final List outcomeElements = e.errors() + .stream() + .map(error -> new BulkDeleteOutcomeElement( + getStoreContext().keyToPath(error.key()), + MultiObjectDeleteException.errorToString(error), + null)) + .collect(Collectors.toList()); + return new BulkDeleteOutcome(outcomeElements); + } + return new BulkDeleteOutcome(Collections.emptyList()); + } + + @Override + public void close() throws IOException { + + } + + public interface BulkDeleteOperationCallbacks { + + /** + * Attempt a bulk delete operation. + * @param keys key list + * @throws MultiObjectDeleteException one or more of the keys could not + * be deleted in a multiple object delete operation. + * @throws AwsServiceException amazon-layer failure. + * @throws IOException other IO Exception. + * @throws IllegalArgumentException illegal arguments + */ + @Retries.RetryTranslated + void bulkDelete(final List keys) + throws MultiObjectDeleteException, IOException, IllegalArgumentException; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteException.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteException.java index 72ead1fb151fc..14ad559ead293 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteException.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MultiObjectDeleteException.java @@ -118,11 +118,7 @@ public IOException translateException(final String message) { String exitCode = ""; for (S3Error error : errors()) { String code = error.code(); - String item = String.format("%s: %s%s: %s%n", code, error.key(), - (error.versionId() != null - ? (" (" + error.versionId() + ")") - : ""), - error.message()); + String item = errorToString(error); LOG.info(item); result.append(item); if (exitCode == null || exitCode.isEmpty() || ACCESS_DENIED.equals(code)) { @@ -136,4 +132,18 @@ public IOException translateException(final String message) { return new AWSS3IOException(result.toString(), this); } } + + /** + * Convert an error to a string. + * @param error error from a delete request + * @return string value + */ + public static String errorToString(final S3Error error) { + String code = error.code(); + return String.format("%s: %s%s: %s%n", code, error.key(), + (error.versionId() != null + ? (" (" + error.versionId() + ")") + : ""), + error.message()); + } } From fbef70b8ff85941d796d57eaf5e9d3795ed6dfaf Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 9 Feb 2024 13:47:27 +0000 Subject: [PATCH 02/19] HADOOP-18679. Bulk Delete: utility methods Add methods in FileUtil to take an FS, cast to a BulkDeleteSource then perform the pageSize/bulkDelete operations. This is to make reflection based access straightforward: no new interfaces or types to work with, just two new methods with type-erased lists. Change-Id: I2d7b1bf8198422de635253fc087ca551a8bc6609 --- .../java/org/apache/hadoop/fs/BulkDelete.java | 4 +- .../apache/hadoop/fs/BulkDeleteSource.java | 11 +++- .../java/org/apache/hadoop/fs/FileUtil.java | 62 +++++++++++++++++++ 3 files changed, 73 insertions(+), 4 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java index 4f55b25d9c7ee..94dbb303863c2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java @@ -26,6 +26,8 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import static java.util.Objects.requireNonNull; + /** * API for bulk deletion of objects/files, * but not directories. @@ -101,7 +103,7 @@ class BulkDeleteOutcome { * @param failures list of failures. This must be non-null. */ public BulkDeleteOutcome(final List failures) { - this.failures = failures; + this.failures = requireNonNull(failures); } /** diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java index 9d7dc66b41842..d4cffb39bdf7c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java @@ -36,20 +36,25 @@ @InterfaceStability.Unstable public interface BulkDeleteSource { + /** + * Exception message for the case where bulk delete is not supported. + */ + String BULK_DELETE_NOT_SUPPORTED = "Bulk delete not supported"; + /** * Create a bulk delete operation. * There is no network IO at this point, simply the creation of * a bulk delete object. - * A path must be supplied to ensure that on viewfs and similar filesystems, + * A path must be supplied to assist in link resolution. * @param path path to delete under. * @return the bulk delete. - * @throws UnsupportedOperationException the filesystem does not support delete under that path. + * @throws UnsupportedOperationException bulk delete under that path is not supported. * @throws IllegalArgumentException path not valid. * @throws IOException problems resolving paths */ default BulkDelete createBulkDelete(Path path) throws UnsupportedOperationException, IllegalArgumentException, IOException { - throw new UnsupportedOperationException("Bulk delete not supported"); + throw new UnsupportedOperationException(BULK_DELETE_NOT_SUPPORTED); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java index fa87bb48aaa69..049a76da2da83 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java @@ -55,6 +55,7 @@ import java.util.jar.Attributes; import java.util.jar.JarOutputStream; import java.util.jar.Manifest; +import java.util.stream.Collectors; import java.util.zip.GZIPInputStream; import org.apache.commons.collections.map.CaseInsensitiveMap; @@ -2108,4 +2109,65 @@ public static void maybeIgnoreMissingDirectory(FileSystem fs, LOG.info("Ignoring missing directory {}", path); LOG.debug("Directory missing", e); } + + /** + * Get the maximum number of objects/files to delete in a single request. + * @param fs filesystem + * @param path path to delete under. + * @return a number greater than or equal to zero. + * @throws UnsupportedOperationException bulk delete under that path is not supported. + * @throws IllegalArgumentException path not valid. + * @throws IOException problems resolving paths + */ + public static int bulkDeletePageSize(FileSystem fs, Path path) throws IOException { + try (BulkDelete bulk = toBulkDeleteSource(fs).createBulkDelete(path)) { + return bulk.pageSize(); + } + } + + /** + * Convert a filesystem to a bulk delete source. + * @param fs filesystem + * @return cast fs. + * @throws UnsupportedOperationException FS doesn't implement the interface. + */ + private static BulkDeleteSource toBulkDeleteSource(final FileSystem fs) { + if (!(fs instanceof BulkDeleteSource)) { + throw new UnsupportedOperationException(BulkDeleteSource.BULK_DELETE_NOT_SUPPORTED); + } + return (BulkDeleteSource) fs; + } + + /** + * Delete a list of files/objects. + *
    + *
  • Files must be under the path provided in {@code base}.
  • + *
  • The size of the list must be equal to or less than the page size.
  • + *
  • Directories are not supported; the outcome of attempting to delete + * directories is undefined (ignored; undetected, listed as failures...).
  • + *
  • The operation is not atomic.
  • + *
  • The operation is treated as idempotent: network failures may + * trigger resubmission of the request -any new objects created under a + * path in the list may then be deleted.
  • + *
  • There is no guarantee that any parent directories exist after this call. + *
  • + *
+ * @param fs filesystem + * @param base path to delete under. + * @param paths list of paths which must be absolute and under the base path. + * @return a list of all the paths which couldn't be deleted for a reason other than "not found". + * @throws UnsupportedOperationException bulk delete under that path is not supported. + * @throws IOException IO problems including networking, authentication and more. + * @throws IllegalArgumentException if a path argument is invalid. + */ + public static List bulkDelete(FileSystem fs, Path base, List paths) + throws IOException { + try (BulkDelete bulk = toBulkDeleteSource(fs).createBulkDelete(base)) { + final BulkDelete.BulkDeleteOutcome outcome = bulk.bulkDelete(paths); + return outcome.getFailures().stream() + .map(BulkDelete.BulkDeleteOutcomeElement::getPath) + .collect(Collectors.toList()); + } + } + } From 910dd76b5eb8cc8abc262c9a6bd86c453f7dc689 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 15 Feb 2024 16:05:13 +0000 Subject: [PATCH 03/19] HADOOP-18679. ?: values the wrong way round Change-Id: Ib098c07cc1f7747ed1a3131b252656c96c520a75 --- .../src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 5fa910632c1cb..d6bcf9b918d28 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -5793,7 +5793,7 @@ public BulkDelete createBulkDelete(final Path path) createStoreContext(), createBulkDeleteCallbacks(span), p, - enableMultiObjectsDelete ? 1: pageSize, + enableMultiObjectsDelete ? pageSize : 1, span); } From 07df662c36f2c8d82de7aac08897345344451aae Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 13 Mar 2024 18:59:52 +0000 Subject: [PATCH 04/19] HADOOP-18679. Bulk Delete: initial S3AStore interface/impl Using this PR to start with the initial design, implementation and services offered by having lower-level interaction with S3 pushed down into an S3AStore class, with interface/impl split. The bulk delete callbacks now to talk to the store, *not* s3afs, with some minor changes in behaviour (IllegalArgumentException is raised if root paths / are to be deleted) Mock tests are failing; I expected that: they are always brittle. What next? get this in and then move lower level fs ops over a method calling s3client at a time, or in groups, as appropriate. The metric of success are: * all callback classes created in S3A FS can work through the store * no s3client direct invocation in S3AFS Change-Id: Ib5bc58991533fd5d13e78426e88b884b6ae5205c --- .../java/org/apache/hadoop/fs/BulkDelete.java | 4 +- .../apache/hadoop/util/functional/Tuples.java | 87 ++++ .../apache/hadoop/fs/s3a/S3AFileSystem.java | 165 +++---- .../apache/hadoop/fs/s3a/S3AInternals.java | 12 +- .../org/apache/hadoop/fs/s3a/S3AStore.java | 119 +++++ .../fs/s3a/impl/BulkDeleteOperation.java | 15 +- .../BulkDeleteOperationCallbacksImpl.java | 94 ++++ .../hadoop/fs/s3a/impl/S3AStoreBuilder.java | 104 +++++ .../hadoop/fs/s3a/impl/S3AStoreImpl.java | 412 ++++++++++++++++++ .../fs/s3a/impl/StoreContextFactory.java | 32 ++ .../tools/hadoop-aws/aws_sdk_upgrade.md | 1 + .../hadoop/fs/s3a/AbstractS3AMockTest.java | 3 +- .../hadoop/fs/s3a/TestS3ADeleteOnExit.java | 3 +- .../contract/ITestAbfsContractBulkDelete.java | 32 ++ 14 files changed, 953 insertions(+), 130 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/Tuples.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperationCallbacksImpl.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextFactory.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsContractBulkDelete.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java index 94dbb303863c2..af454bc49a8a8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java @@ -38,9 +38,9 @@ * operation completes; if an object store needs to explicitly look for and create * directory markers, that step will be omitted. *

- *

* Be aware that on some stores (AWS S3) each object listed in a bulk delete counts - * against the write IOPS limit; large page sizes are counterproductive here. + * against the write IOPS limit; large page sizes are counterproductive here, as + * are attempts at parallel submissions across multiple threads. * @see HADOOP-16823. * Large DeleteObject requests are their own Thundering Herd *

diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/Tuples.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/Tuples.java new file mode 100644 index 0000000000000..ed80c1daca726 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/Tuples.java @@ -0,0 +1,87 @@ +/* + * 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.hadoop.util.functional; + +import java.util.Map; + +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Tuple support. + * This allows for tuples to be passed around as part of the public API without + * committing to a third-party library tuple implementation. + */ +@InterfaceStability.Unstable +public final class Tuples { + + private Tuples() { + } + + /** + * Create a 2-tuple. + * @param key element 1 + * @param value element 2 + * @return a tuple. + * @param element 1 type + * @param element 2 type + */ + public static Map.Entry pair(final K key, final V value) { + return new Tuple<>(key, value); + } + + /** + * Simple tuple class: uses the Map.Entry interface as other + * implementations have done, so the API is available across + * all java versions. + * @param key + * @param value + */ + private static final class Tuple implements Map.Entry { + + private final K key; + + private final V value; + + private Tuple(final K key, final V value) { + this.key = key; + this.value = value; + } + + @Override + public K getKey() { + return key; + } + + @Override + public V getValue() { + return value; + } + + @Override + public V setValue(final V value) { + throw new UnsupportedOperationException("Tuple is immutable"); + } + + @Override + public String toString() { + return "(" + key + ", " + value + ')'; + } + + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index d6bcf9b918d28..fbaa8c8f4cfa2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -81,7 +81,6 @@ import software.amazon.awssdk.services.s3.model.ObjectIdentifier; import software.amazon.awssdk.services.s3.model.PutObjectRequest; import software.amazon.awssdk.services.s3.model.PutObjectResponse; -import software.amazon.awssdk.services.s3.model.S3Error; import software.amazon.awssdk.services.s3.model.S3Object; import software.amazon.awssdk.services.s3.model.StorageClass; import software.amazon.awssdk.services.s3.model.UploadPartRequest; @@ -123,7 +122,7 @@ import org.apache.hadoop.fs.s3a.impl.AWSCannedACL; import org.apache.hadoop.fs.s3a.impl.AWSHeaders; import org.apache.hadoop.fs.s3a.impl.BulkDeleteOperation; -import org.apache.hadoop.fs.s3a.impl.BulkDeleteRetryHandler; +import org.apache.hadoop.fs.s3a.impl.BulkDeleteOperationCallbacksImpl; import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.ConfigurationHelper; import org.apache.hadoop.fs.s3a.impl.ContextAccessors; @@ -144,9 +143,11 @@ import org.apache.hadoop.fs.s3a.impl.RenameOperation; import org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl; import org.apache.hadoop.fs.s3a.impl.S3AMultipartUploaderBuilder; +import org.apache.hadoop.fs.s3a.impl.S3AStoreBuilder; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder; +import org.apache.hadoop.fs.s3a.impl.StoreContextFactory; import org.apache.hadoop.fs.s3a.prefetch.S3APrefetchingInputStream; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations; import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl; @@ -247,7 +248,6 @@ import static org.apache.hadoop.fs.s3a.impl.InternalConstants.ARN_BUCKET_OPTION; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.CSE_PADDING_LENGTH; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT; -import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DELETE_CONSIDERED_IDEMPOTENT; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_403_FORBIDDEN; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404_NOT_FOUND; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.UPLOAD_PART_COUNT_LIMIT; @@ -261,11 +261,11 @@ import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.pairedTrackerFactory; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration; -import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; import static org.apache.hadoop.util.Preconditions.checkArgument; +import static org.apache.hadoop.util.RateLimitingFactory.unlimitedRate; import static org.apache.hadoop.util.functional.RemoteIterators.foreach; import static org.apache.hadoop.util.functional.RemoteIterators.typeCastingRemoteIterator; @@ -287,7 +287,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, AWSPolicyProvider, DelegationTokenProvider, IOStatisticsSource, AuditSpanSource, ActiveThreadSpanSource, - BulkDeleteSource { + BulkDeleteSource, StoreContextFactory { /** * Default blocksize as used in blocksize and FS status queries. @@ -300,6 +300,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private String username; + /** + * Store back end. + */ + private S3AStore store; + private S3Client s3Client; /** Async client is used for transfer manager. */ @@ -684,9 +689,6 @@ public void initialize(URI name, Configuration originalConf) // the encryption algorithms) bindAWSClient(name, delegationTokensEnabled); - // This initiates a probe against S3 for the bucket existing. - doBucketProbing(); - inputPolicy = S3AInputPolicy.getPolicy( conf.getTrimmed(INPUT_FADVISE, Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_DEFAULT), @@ -733,9 +735,6 @@ public void initialize(URI name, Configuration originalConf) directoryPolicy = DirectoryPolicyImpl.getDirectoryPolicy(conf, this::allowAuthoritative); LOG.debug("Directory marker retention policy is {}", directoryPolicy); - - initMultipartUploads(conf); - pageSize = intOption(getConf(), BULK_DELETE_PAGE_SIZE, BULK_DELETE_PAGE_SIZE_DEFAULT, 0); checkArgument(pageSize <= InternalConstants.MAX_ENTRIES_TO_DELETE, @@ -760,6 +759,25 @@ public void initialize(URI name, Configuration originalConf) OPTIMIZED_COPY_FROM_LOCAL_DEFAULT); LOG.debug("Using optimized copyFromLocal implementation: {}", optimizedCopyFromLocal); s3AccessGrantsEnabled = conf.getBoolean(AWS_S3_ACCESS_GRANTS_ENABLED, false); + + // now create the store + store = new S3AStoreBuilder() + .withS3Client(s3Client) + .withDurationTrackerFactory(getDurationTrackerFactory()) + .withStoreContextFactory(this) + .withAuditSpanSource(getAuditManager()) + .withInstrumentation(getInstrumentation()) + .withStatisticsContext(statisticsContext) + .withStorageStatistics(getStorageStatistics()) + .withReadRateLimiter(unlimitedRate()) + .withWriteRateLimiter(unlimitedRate()) + .build(); + + // The filesystem is now ready to perform operations against + // S3 + // This initiates a probe against S3 for the bucket existing. + doBucketProbing(); + initMultipartUploads(conf); } catch (SdkException e) { // amazon client exception: stop all services then throw the translation cleanupWithLogger(LOG, span); @@ -1421,6 +1439,11 @@ public S3Client getAmazonS3Client(String reason) { return s3Client; } + @Override + public S3AStore getStore() { + return store; + } + /** * S3AInternals method. * {@inheritDoc}. @@ -3068,29 +3091,10 @@ public void incrementWriteOperations() { @Retries.RetryRaw protected void deleteObject(String key) throws SdkException, IOException { - blockRootDelete(key); incrementWriteOperations(); - try (DurationInfo ignored = - new DurationInfo(LOG, false, - "deleting %s", key)) { - invoker.retryUntranslated(String.format("Delete %s:/%s", bucket, key), - DELETE_CONSIDERED_IDEMPOTENT, - () -> { - incrementStatistic(OBJECT_DELETE_OBJECTS); - trackDurationOfInvocation(getDurationTrackerFactory(), - OBJECT_DELETE_REQUEST.getSymbol(), - () -> s3Client.deleteObject(getRequestFactory() - .newDeleteObjectRequestBuilder(key) - .build())); - return null; - }); - } catch (AwsServiceException ase) { - // 404 errors get swallowed; this can be raised by - // third party stores (GCS). - if (!isObjectNotFound(ase)) { - throw ase; - } - } + store.deleteObject(getRequestFactory() + .newDeleteObjectRequestBuilder(key) + .build()); } /** @@ -3116,19 +3120,6 @@ void deleteObjectAtPath(Path f, deleteObject(key); } - /** - * Reject any request to delete an object where the key is root. - * @param key key to validate - * @throws InvalidRequestException if the request was rejected due to - * a mistaken attempt to delete the root directory. - */ - private void blockRootDelete(String key) throws InvalidRequestException { - if (key.isEmpty() || "/".equals(key)) { - throw new InvalidRequestException("Bucket "+ bucket - +" cannot be deleted"); - } - } - /** * Perform a bulk object delete operation against S3. * Increments the {@code OBJECT_DELETE_REQUESTS} and write @@ -3155,38 +3146,11 @@ private void blockRootDelete(String key) throws InvalidRequestException { private DeleteObjectsResponse deleteObjects(DeleteObjectsRequest deleteRequest) throws MultiObjectDeleteException, SdkException, IOException { incrementWriteOperations(); - BulkDeleteRetryHandler retryHandler = - new BulkDeleteRetryHandler(createStoreContext()); - int keyCount = deleteRequest.delete().objects().size(); - try (DurationInfo ignored = - new DurationInfo(LOG, false, "DELETE %d keys", - keyCount)) { - DeleteObjectsResponse response = - invoker.retryUntranslated("delete", DELETE_CONSIDERED_IDEMPOTENT, - (text, e, r, i) -> { - // handle the failure - retryHandler.bulkDeleteRetried(deleteRequest, e); - }, - // duration is tracked in the bulk delete counters - trackDurationOfOperation(getDurationTrackerFactory(), - OBJECT_BULK_DELETE_REQUEST.getSymbol(), () -> { - incrementStatistic(OBJECT_DELETE_OBJECTS, keyCount); - return s3Client.deleteObjects(deleteRequest); - })); - - if (!response.errors().isEmpty()) { - // one or more of the keys could not be deleted. - // log and then throw - List errors = response.errors(); - LOG.debug("Partial failure of delete, {} errors", errors.size()); - for (S3Error error : errors) { - LOG.debug("{}: \"{}\" - {}", error.key(), error.code(), error.message()); - } - throw new MultiObjectDeleteException(errors); - } - - return response; + DeleteObjectsResponse response = store.deleteObjects(deleteRequest).getValue(); + if (!response.errors().isEmpty()) { + throw new MultiObjectDeleteException(response.errors()); } + return response; } /** @@ -3395,14 +3359,6 @@ private void removeKeysS3( List keysToDelete, boolean deleteFakeDir) throws MultiObjectDeleteException, AwsServiceException, IOException { - if (LOG.isDebugEnabled()) { - LOG.debug("Initiating delete operation for {} objects", - keysToDelete.size()); - for (ObjectIdentifier objectIdentifier : keysToDelete) { - LOG.debug(" \"{}\" {}", objectIdentifier.key(), - objectIdentifier.versionId() != null ? objectIdentifier.versionId() : ""); - } - } if (keysToDelete.isEmpty()) { // exit fast if there are no keys to delete return; @@ -3414,9 +3370,6 @@ private void removeKeysS3( noteDeleted(1, deleteFakeDir); return; } - for (ObjectIdentifier objectIdentifier : keysToDelete) { - blockRootDelete(objectIdentifier.key()); - } try { if (enableMultiObjectsDelete) { if (keysToDelete.size() <= pageSize) { @@ -5682,6 +5635,7 @@ public S3AMultipartUploaderBuilder createMultipartUploader( * new store context instances should be created as appropriate. * @return the store context of this FS. */ + @Override @InterfaceAudience.Private public StoreContext createStoreContext() { return new StoreContextBuilder().setFsURI(getUri()) @@ -5789,44 +5743,23 @@ public BulkDelete createBulkDelete(final Path path) final Path p = makeQualified(path); final AuditSpanS3A span = createSpan("bulkdelete", p.toString(), null); + final int size = enableMultiObjectsDelete ? pageSize : 1; return new BulkDeleteOperation( createStoreContext(), - createBulkDeleteCallbacks(span), + createBulkDeleteCallbacks(p, size, span), p, - enableMultiObjectsDelete ? pageSize : 1, + size, span); } /** - * Override point for mocking. + * Create the callbacks for the bulk delete operation. * @param span span for operations. - * @return an instance of the Bulk Delette callbacks. + * @return an instance of the Bulk Delete callbacks. */ protected BulkDeleteOperation.BulkDeleteOperationCallbacks createBulkDeleteCallbacks( - final AuditSpanS3A span) { - return new BulkDeleteOperationCallbacksImpl(span); + Path path, int pageSize, AuditSpanS3A span) { + return new BulkDeleteOperationCallbacksImpl(store, pathToKey(path), pageSize, span); } - /** - * Callbacks for the bulk delete operation. - */ - protected class BulkDeleteOperationCallbacksImpl implements - BulkDeleteOperation.BulkDeleteOperationCallbacks { - - /** span for operations. */ - private final AuditSpan span; - - protected BulkDeleteOperationCallbacksImpl(AuditSpan span) { - this.span = span; - } - - @Override - @Retries.RetryTranslated - public void bulkDelete(final List keys) - throws MultiObjectDeleteException, IOException, IllegalArgumentException { - span.activate(); - once("bulkDelete", "", () -> - S3AFileSystem.this.removeKeys(keys, false)); - } - } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInternals.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInternals.java index b4116068565c2..3f3178c7e6e28 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInternals.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInternals.java @@ -33,6 +33,9 @@ /** * This is an unstable interface for access to S3A Internal state, S3 operations * and the S3 client connector itself. + *

+ * Note for maintainers: this is documented in {@code aws_sdk_upgrade.md}; update + * on changes. */ @InterfaceStability.Unstable @InterfaceAudience.LimitedPrivate("testing/diagnostics") @@ -52,13 +55,19 @@ public interface S3AInternals { * set to false. *

* Mocking note: this is the same S3Client as is used by the owning - * filesystem; changes to this client will be reflected by changes + * filesystem and S3AStore; changes to this client will be reflected by changes * in the behavior of that filesystem. * @param reason a justification for requesting access. * @return S3Client */ S3Client getAmazonS3Client(String reason); + /** + * Get the store for low-level operations. + * @return the store the S3A FS is working through. + */ + S3AStore getStore(); + /** * Get the region of a bucket. * Invoked from StoreContext; consider an entry point. @@ -131,4 +140,5 @@ public interface S3AInternals { @AuditEntryPoint @Retries.RetryTranslated long abortMultipartUploads(Path path) throws IOException; + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java new file mode 100644 index 0000000000000..c029364743cce --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java @@ -0,0 +1,119 @@ +/* + * 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.hadoop.fs.s3a; + +import java.io.IOException; +import java.time.Duration; +import java.util.Map; +import java.util.Optional; + +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; +import software.amazon.awssdk.services.s3.model.DeleteObjectResponse; +import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; +import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteException; +import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +/** + * Interface for the S3A Store; + * S3 client interactions should be via this; mocking + * is possible for unit tests. + */ +@InterfaceAudience.LimitedPrivate("Extensions") +@InterfaceStability.Unstable +public interface S3AStore extends IOStatisticsSource { + + /** + * Acquire write capacity for operations. + * This should be done within retry loops. + * @param capacity capacity to acquire. + * @return time spent waiting for output. + */ + Duration acquireWriteCapacity(int capacity); + + /** + * Acquire read capacity for operations. + * This should be done within retry loops. + * @param capacity capacity to acquire. + * @return time spent waiting for output. + */ + Duration acquireReadCapacity(int capacity); + + StoreContext getStoreContext(); + + DurationTrackerFactory getDurationTrackerFactory(); + + S3AStatisticsContext getStatisticsContext(); + + RequestFactory getRequestFactory(); + + /** + * Perform a bulk object delete operation against S3. + * Increments the {@code OBJECT_DELETE_REQUESTS} and write + * operation statistics + *

+ * {@code OBJECT_DELETE_OBJECTS} is updated with the actual number + * of objects deleted in the request. + *

+ * Retry policy: retry untranslated; delete considered idempotent. + * If the request is throttled, this is logged in the throttle statistics, + * with the counter set to the number of keys, rather than the number + * of invocations of the delete operation. + * This is because S3 considers each key as one mutating operation on + * the store when updating its load counters on a specific partition + * of an S3 bucket. + * If only the request was measured, this operation would under-report. + * @param deleteRequest keys to delete on the s3-backend + * @return the AWS response + * @throws MultiObjectDeleteException one or more of the keys could not + * be deleted. + * @throws SdkException amazon-layer failure. + */ + @Retries.RetryRaw + Map.Entry deleteObjects(DeleteObjectsRequest deleteRequest) + throws MultiObjectDeleteException, SdkException, IOException; + + /** + * Delete an object. + * Increments the {@code OBJECT_DELETE_REQUESTS} statistics. + *

+ * Retry policy: retry untranslated; delete considered idempotent. + * 404 errors other than bucket not found are swallowed; + * this can be raised by third party stores (GCS). + * If an exception is caught and swallowed, the response will be empty; + * otherwise it will be the response from the delete operation. + * @param request request to make + * @return the total duration and response. + * @throws SdkException problems working with S3 + * @throws IllegalArgumentException if the request was rejected due to + * a mistaken attempt to delete the root directory. + */ + @Retries.RetryRaw + Map.Entry> deleteObject( + DeleteObjectRequest request) throws SdkException; + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java index f4d0fea16e7b9..61e666cacbfef 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java @@ -79,14 +79,14 @@ public BulkDeleteOutcome bulkDelete(final List paths) final String k = getStoreContext().pathToKey(p); return ObjectIdentifier.builder().key(k).build(); }).collect(Collectors.toList()); - try { - callbacks.bulkDelete(objects); - } catch (MultiObjectDeleteException e) { - final List outcomeElements = e.errors() + + final List errors = callbacks.bulkDelete(objects); + if (!errors.isEmpty()) { + + final List outcomeElements = errors .stream() .map(error -> new BulkDeleteOutcomeElement( - getStoreContext().keyToPath(error.key()), - MultiObjectDeleteException.errorToString(error), + getStoreContext().keyToPath(error), error, null)) .collect(Collectors.toList()); return new BulkDeleteOutcome(outcomeElements); @@ -104,6 +104,7 @@ public interface BulkDeleteOperationCallbacks { /** * Attempt a bulk delete operation. * @param keys key list + * @return * @throws MultiObjectDeleteException one or more of the keys could not * be deleted in a multiple object delete operation. * @throws AwsServiceException amazon-layer failure. @@ -111,7 +112,7 @@ public interface BulkDeleteOperationCallbacks { * @throws IllegalArgumentException illegal arguments */ @Retries.RetryTranslated - void bulkDelete(final List keys) + List bulkDelete(final List keys) throws MultiObjectDeleteException, IOException, IllegalArgumentException; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperationCallbacksImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperationCallbacksImpl.java new file mode 100644 index 0000000000000..4c4c1dfe44aff --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperationCallbacksImpl.java @@ -0,0 +1,94 @@ +/* + * 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.hadoop.fs.s3a.impl; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; +import software.amazon.awssdk.services.s3.model.S3Error; + +import org.apache.hadoop.fs.s3a.Retries; +import org.apache.hadoop.fs.s3a.S3AStore; +import org.apache.hadoop.fs.store.audit.AuditSpan; + +import static org.apache.hadoop.fs.s3a.Invoker.once; +import static org.apache.hadoop.util.Preconditions.checkArgument; + +/** + * Callbacks for the bulk delete operation. + */ +public class BulkDeleteOperationCallbacksImpl implements + BulkDeleteOperation.BulkDeleteOperationCallbacks { + + private final String path; + + private final int pageSize; + + /** span for operations. */ + private final AuditSpan span; + + private final S3AStore store; + + + public BulkDeleteOperationCallbacksImpl(final S3AStore store, + String path, int pageSize, AuditSpan span) { + this.span = span; + this.pageSize = pageSize; + this.path = path; + this.store = store; + } + + @Override + @Retries.RetryTranslated + public List bulkDelete(final List keysToDelete) + throws MultiObjectDeleteException, IOException, IllegalArgumentException { + span.activate(); + final int size = keysToDelete.size(); + checkArgument(size <= pageSize, + "Too many paths to delete in one operation: %s", size); + if (size == 0) { + return Collections.emptyList(); + } + if (size == 1) { + store.deleteObject(store.getRequestFactory() + .newDeleteObjectRequestBuilder(keysToDelete.get(0).key()) + .build()); + // no failures, so return an empty list + return Collections.emptyList(); + } + + final DeleteObjectsResponse response = once("bulkDelete", path, () -> + store.deleteObjects(store.getRequestFactory() + .newBulkDeleteRequestBuilder(keysToDelete) + .build())).getValue(); + final List errors = response.errors(); + if (errors.isEmpty()) { + // all good. + return Collections.emptyList(); + } else { + return errors.stream() + .map(S3Error::key) + .collect(Collectors.toList()); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java new file mode 100644 index 0000000000000..40f649a7378b6 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java @@ -0,0 +1,104 @@ +/* + * 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.hadoop.fs.s3a.impl; + +import software.amazon.awssdk.services.s3.S3Client; + +import org.apache.hadoop.fs.s3a.S3AInstrumentation; +import org.apache.hadoop.fs.s3a.S3AStorageStatistics; +import org.apache.hadoop.fs.s3a.S3AStore; +import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A; +import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; +import org.apache.hadoop.fs.store.audit.AuditSpanSource; +import org.apache.hadoop.util.RateLimiting; + +/** + * Builder for the S3AStore. + */ +public class S3AStoreBuilder { + + private StoreContextFactory storeContextFactory; + + private S3Client s3Client; + + private DurationTrackerFactory durationTrackerFactory; + + private S3AInstrumentation instrumentation; + + private S3AStatisticsContext statisticsContext; + + private S3AStorageStatistics storageStatistics; + + private RateLimiting readRateLimiter; + + private RateLimiting writeRateLimiter; + + private AuditSpanSource auditSpanSource; + + public S3AStoreBuilder withStoreContextFactory(final StoreContextFactory storeContextFactory) { + this.storeContextFactory = storeContextFactory; + return this; + } + + public S3AStoreBuilder withS3Client(final S3Client s3Client) { + this.s3Client = s3Client; + return this; + } + + public S3AStoreBuilder withDurationTrackerFactory(final DurationTrackerFactory durationTrackerFactory) { + this.durationTrackerFactory = durationTrackerFactory; + return this; + } + + public S3AStoreBuilder withInstrumentation(final S3AInstrumentation instrumentation) { + this.instrumentation = instrumentation; + return this; + } + + public S3AStoreBuilder withStatisticsContext(final S3AStatisticsContext statisticsContext) { + this.statisticsContext = statisticsContext; + return this; + } + + public S3AStoreBuilder withStorageStatistics(final S3AStorageStatistics storageStatistics) { + this.storageStatistics = storageStatistics; + return this; + } + + public S3AStoreBuilder withReadRateLimiter(final RateLimiting readRateLimiter) { + this.readRateLimiter = readRateLimiter; + return this; + } + + public S3AStoreBuilder withWriteRateLimiter(final RateLimiting writeRateLimiter) { + this.writeRateLimiter = writeRateLimiter; + return this; + } + + public S3AStoreBuilder withAuditSpanSource(final AuditSpanSource auditSpanSource) { + this.auditSpanSource = auditSpanSource; + return this; + } + + public S3AStore build() { + return new S3AStoreImpl(storeContextFactory, s3Client, durationTrackerFactory, instrumentation, + statisticsContext, storageStatistics, readRateLimiter, writeRateLimiter, auditSpanSource); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java new file mode 100644 index 0000000000000..6d60b71a95cb2 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java @@ -0,0 +1,412 @@ +/* + * 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.hadoop.fs.s3a.impl; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import javax.annotation.Nullable; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; +import software.amazon.awssdk.services.s3.model.DeleteObjectResponse; +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.S3Error; + +import org.apache.hadoop.fs.s3a.Invoker; +import org.apache.hadoop.fs.s3a.Retries; +import org.apache.hadoop.fs.s3a.S3AInstrumentation; +import org.apache.hadoop.fs.s3a.S3AStorageStatistics; +import org.apache.hadoop.fs.s3a.S3AStore; +import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A; +import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; +import org.apache.hadoop.fs.statistics.DurationTrackerFactory; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.store.audit.AuditSpanSource; +import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.RateLimiting; +import org.apache.hadoop.util.functional.Tuples; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException; +import static org.apache.hadoop.fs.s3a.Statistic.IGNORED_ERRORS; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_BULK_DELETE_REQUEST; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_OBJECTS; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUEST; +import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_RETRY; +import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLED; +import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLE_RATE; +import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DELETE_CONSIDERED_IDEMPOTENT; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation; +import static org.apache.hadoop.util.Preconditions.checkArgument; + +/** + * Store Layer. + * This is where lower level storage operations are intended + * to move. + */ +public class S3AStoreImpl implements S3AStore { + + private static final Logger LOG = LoggerFactory.getLogger(S3AStoreImpl.class); + + private final StoreContextFactory storeContextFactory; + + private final S3Client s3Client; + + private final String bucket; + + private final RequestFactory requestFactory; + + /** Async client is used for transfer manager. */ + private S3AsyncClient s3AsyncClient; + + private final DurationTrackerFactory durationTrackerFactory; + + /** The core instrumentation. */ + private final S3AInstrumentation instrumentation; + + /** Accessors to statistics for this FS. */ + private final S3AStatisticsContext statisticsContext; + + /** Storage Statistics Bonded to the instrumentation. */ + private final S3AStorageStatistics storageStatistics; + + private final RateLimiting readRateLimiter; + + private final RateLimiting writeRateLimiter; + + private final StoreContext storeContext; + + private final Invoker invoker; + + private final AuditSpanSource auditSpanSource; + + S3AStoreImpl(StoreContextFactory storeContextFactory, + S3Client s3Client, + DurationTrackerFactory durationTrackerFactory, + S3AInstrumentation instrumentation, + S3AStatisticsContext statisticsContext, + S3AStorageStatistics storageStatistics, + RateLimiting readRateLimiter, + RateLimiting writeRateLimiter, + AuditSpanSource auditSpanSource) { + this.storeContextFactory = requireNonNull(storeContextFactory); + this.s3Client = requireNonNull(s3Client); + this.durationTrackerFactory = requireNonNull(durationTrackerFactory); + this.instrumentation = requireNonNull(instrumentation); + this.statisticsContext = requireNonNull(statisticsContext); + this.storageStatistics = requireNonNull(storageStatistics); + this.readRateLimiter = requireNonNull(readRateLimiter); + this.writeRateLimiter = requireNonNull(writeRateLimiter); + this.auditSpanSource = requireNonNull(auditSpanSource); + this.storeContext = requireNonNull(storeContextFactory.createStoreContext()); + this.invoker = storeContext.getInvoker(); + this.bucket = storeContext.getBucket(); + this.requestFactory = storeContext.getRequestFactory(); + } + + @Override + public Duration acquireWriteCapacity(final int capacity) { + return writeRateLimiter.acquire(capacity); + } + + @Override + public Duration acquireReadCapacity(final int capacity) { + return readRateLimiter.acquire(capacity); + + } + + /** + * Create the store context. + * @return a new store context. + */ + private StoreContext createStoreContext() { + return storeContextFactory.createStoreContext(); + } + + @Override + public StoreContext getStoreContext() { + return storeContext; + } + + private S3Client getS3Client() { + return s3Client; + } + + @Override + public DurationTrackerFactory getDurationTrackerFactory() { + return durationTrackerFactory; + } + + private S3AInstrumentation getInstrumentation() { + return instrumentation; + } + + @Override + public S3AStatisticsContext getStatisticsContext() { + return statisticsContext; + } + + private S3AStorageStatistics getStorageStatistics() { + return storageStatistics; + } + + @Override + public RequestFactory getRequestFactory() { + return requestFactory; + } + + /** + * Increment a statistic by 1. + * This increments both the instrumentation and storage statistics. + * @param statistic The operation to increment + */ + protected void incrementStatistic(Statistic statistic) { + incrementStatistic(statistic, 1); + } + + /** + * Increment a statistic by a specific value. + * This increments both the instrumentation and storage statistics. + * @param statistic The operation to increment + * @param count the count to increment + */ + protected void incrementStatistic(Statistic statistic, long count) { + statisticsContext.incrementCounter(statistic, count); + } + + /** + * Decrement a gauge by a specific value. + * @param statistic The operation to decrement + * @param count the count to decrement + */ + protected void decrementGauge(Statistic statistic, long count) { + statisticsContext.decrementGauge(statistic, count); + } + + /** + * Increment a gauge by a specific value. + * @param statistic The operation to increment + * @param count the count to increment + */ + protected void incrementGauge(Statistic statistic, long count) { + statisticsContext.incrementGauge(statistic, count); + } + + /** + * Callback when an operation was retried. + * Increments the statistics of ignored errors or throttled requests, + * depending up on the exception class. + * @param ex exception. + */ + public void operationRetried(Exception ex) { + if (isThrottleException(ex)) { + LOG.debug("Request throttled"); + incrementStatistic(STORE_IO_THROTTLED); + statisticsContext.addValueToQuantiles(STORE_IO_THROTTLE_RATE, 1); + } else { + incrementStatistic(STORE_IO_RETRY); + incrementStatistic(IGNORED_ERRORS); + } + } + + /** + * Callback from {@link Invoker} when an operation is retried. + * @param text text of the operation + * @param ex exception + * @param retries number of retries + * @param idempotent is the method idempotent + */ + public void operationRetried(String text, Exception ex, int retries, boolean idempotent) { + operationRetried(ex); + } + + /** + * Get the instrumentation's IOStatistics. + * @return statistics + */ + @Override + public IOStatistics getIOStatistics() { + return instrumentation.getIOStatistics(); + } + + /** + * Start an operation; this informs the audit service of the event + * and then sets it as the active span. + * @param operation operation name. + * @param path1 first path of operation + * @param path2 second path of operation + * @return a span for the audit + * @throws IOException failure + */ + public AuditSpanS3A createSpan(String operation, @Nullable String path1, @Nullable String path2) + throws IOException { + + return auditSpanSource.createSpan(operation, path1, path2); + } + + /** + * Reject any request to delete an object where the key is root. + * @param key key to validate + * @throws IllegalArgumentException if the request was rejected due to + * a mistaken attempt to delete the root directory. + */ + private void blockRootDelete(String key) throws IllegalArgumentException { + checkArgument(!key.isEmpty() && !"/".equals(key), "Bucket %s cannot be deleted", bucket); + } + + /** + * Perform a bulk object delete operation against S3. + * Increments the {@code OBJECT_DELETE_REQUESTS} and write + * operation statistics + *

+ * {@code OBJECT_DELETE_OBJECTS} is updated with the actual number + * of objects deleted in the request. + *

+ * Retry policy: retry untranslated; delete considered idempotent. + * If the request is throttled, this is logged in the throttle statistics, + * with the counter set to the number of keys, rather than the number + * of invocations of the delete operation. + * This is because S3 considers each key as one mutating operation on + * the store when updating its load counters on a specific partition + * of an S3 bucket. + * If only the request was measured, this operation would under-report. + * @param deleteRequest keys to delete on the s3-backend + * @return the AWS response + * @throws IllegalArgumentException if the request was rejected due to + * a mistaken attempt to delete the root directory + * @throws SdkException amazon-layer failure. + */ + @Override + @Retries.RetryRaw + public Map.Entry deleteObjects(final DeleteObjectsRequest deleteRequest) + throws SdkException { + + DeleteObjectsResponse response; + BulkDeleteRetryHandler retryHandler = new BulkDeleteRetryHandler(createStoreContext()); + + final List keysToDelete = deleteRequest.delete().objects(); + int keyCount = keysToDelete.size(); + if (LOG.isDebugEnabled()) { + LOG.debug("Initiating delete operation for {} objects", keysToDelete.size()); + keysToDelete.stream().forEach(objectIdentifier -> { + LOG.debug(" \"{}\" {}", objectIdentifier.key(), + objectIdentifier.versionId() != null ? objectIdentifier.versionId() : ""); + }); + } + // block root calls + keysToDelete.stream().map(ObjectIdentifier::key).forEach(this::blockRootDelete); + + try (DurationInfo d = new DurationInfo(LOG, false, "DELETE %d keys", keyCount)) { + response = + invoker.retryUntranslated("delete", DELETE_CONSIDERED_IDEMPOTENT, (text, e, r, i) -> { + // handle the failure + retryHandler.bulkDeleteRetried(deleteRequest, e); + }, + // duration is tracked in the bulk delete counters + trackDurationOfOperation(getDurationTrackerFactory(), + OBJECT_BULK_DELETE_REQUEST.getSymbol(), () -> { + acquireWriteCapacity(keyCount); + incrementStatistic(OBJECT_DELETE_OBJECTS, keyCount); + return s3Client.deleteObjects(deleteRequest); + })); + if (!response.errors().isEmpty()) { + // one or more of the keys could not be deleted. + // log and then throw + List errors = response.errors(); + if (LOG.isDebugEnabled()) { + LOG.debug("Partial failure of delete, {} errors", errors.size()); + for (S3Error error : errors) { + LOG.debug("{}: \"{}\" - {}", error.key(), error.code(), error.message()); + } + } + } + d.close(); + return Tuples.pair(d.asDuration(), response); + + } catch (IOException e) { + // this is part of the retry signature, nothing else. + // convert to unchecked. + throw new UncheckedIOException(e); + } + } + + /** + * Delete an object. + * Increments the {@code OBJECT_DELETE_REQUESTS} statistics. + *

+ * Retry policy: retry untranslated; delete considered idempotent. + * 404 errors other than bucket not found are swallowed; + * this can be raised by third party stores (GCS). + * If an exception is caught and swallowed, the response will be empty; + * otherwise it will be the response from the delete operation. + * @param request request to make + * @return the total duration and response. + * @throws SdkException problems working with S3 + * @throws IllegalArgumentException if the request was rejected due to + * a mistaken attempt to delete the root directory. + */ + @Override + @Retries.RetryRaw + public Map.Entry> deleteObject(final DeleteObjectRequest request) + throws SdkException { + + String key = request.key(); + blockRootDelete(key); + DurationInfo d = new DurationInfo(LOG, false, "deleting %s", key); + try { + DeleteObjectResponse response = + invoker.retryUntranslated(String.format("Delete %s:/%s", bucket, key), + DELETE_CONSIDERED_IDEMPOTENT, trackDurationOfOperation(getDurationTrackerFactory(), + OBJECT_DELETE_REQUEST.getSymbol(), () -> { + incrementStatistic(OBJECT_DELETE_OBJECTS); + acquireWriteCapacity(1); + return s3Client.deleteObject(request); + })); + d.close(); + return Tuples.pair(d.asDuration(), Optional.of(response)); + } catch (AwsServiceException ase) { + // 404 errors get swallowed; this can be raised by + // third party stores (GCS). + if (!isObjectNotFound(ase)) { + throw ase; + } + d.close(); + return Tuples.pair(d.asDuration(), Optional.empty()); + } catch (IOException e) { + // this is part of the retry signature, nothing else. + // convert to unchecked. + throw new UncheckedIOException(e); + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextFactory.java new file mode 100644 index 0000000000000..355288619d30a --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextFactory.java @@ -0,0 +1,32 @@ +/* + * 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.hadoop.fs.s3a.impl; + +import org.apache.hadoop.classification.InterfaceAudience; + +@InterfaceAudience.Private +public interface StoreContextFactory { + + /** + * Build an immutable store context, including picking + * up the current audit span. + * @return the store context. + */ + StoreContext createStoreContext(); +} diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/aws_sdk_upgrade.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/aws_sdk_upgrade.md index e2c095e5317a4..abd58bffc6201 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/aws_sdk_upgrade.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/aws_sdk_upgrade.md @@ -324,6 +324,7 @@ They have also been updated to return V2 SDK classes. public interface S3AInternals { S3Client getAmazonS3V2Client(String reason); + S3AStore getStore(); @Retries.RetryTranslated @AuditEntryPoint String getBucketLocation() throws IOException; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java index 734bcfd9c5d30..f43710cf25eb0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java @@ -35,8 +35,7 @@ /** - * Abstract base class for S3A unit tests using a mock S3 client and a null - * metadata store. + * Abstract base class for S3A unit tests using a mock S3 client. */ public abstract class AbstractS3AMockTest { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ADeleteOnExit.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ADeleteOnExit.java index a4162f212179b..28a443f04cda9 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ADeleteOnExit.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ADeleteOnExit.java @@ -61,9 +61,8 @@ public boolean deleteOnExit(Path f) throws IOException { // processDeleteOnExit. @Override protected boolean deleteWithoutCloseCheck(Path f, boolean recursive) throws IOException { - boolean result = super.deleteWithoutCloseCheck(f, recursive); deleteOnDnExitCount--; - return result; + return true; } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsContractBulkDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsContractBulkDelete.java new file mode 100644 index 0000000000000..85038755d2958 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsContractBulkDelete.java @@ -0,0 +1,32 @@ +//package org.apache.hadoop.fs.azurebfs.contract; +// +//import org.apache.hadoop.conf.Configuration; +//import org.apache.hadoop.fs.contract.AbstractContractBulkDeleteTest; +//import org.apache.hadoop.fs.contract.AbstractFSContract; +// +//public class ITestAbfsContractBulkDelete extends AbstractContractBulkDeleteTest { +// +// private final boolean isSecure; +// private final ABFSContractTestBinding binding; +// +// public ITestAbfsContractBulkDelete() throws Exception { +// binding = new ABFSContractTestBinding(); +// this.isSecure = binding.isSecureMode(); +// } +// +// @Override +// public void setup() throws Exception { +// binding.setup(); +// super.setup(); +// } +// +// @Override +// protected Configuration createConfiguration() { +// return binding.getRawConfiguration(); +// } +// +// @Override +// protected AbstractFSContract createContract(Configuration conf) { +// return new AbfsFileSystemContract(conf, isSecure); +// } +//} From ec5625c4987fa8a943848048bf6aefcbd189ce12 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 14 Mar 2024 16:00:16 +0000 Subject: [PATCH 05/19] HADOOP-18679. Bulk Delete getting the API simpler Changing results of method calls, using Tuples.pair() to return Map.Entry() instances as immutable tuples. Change-Id: Ibdd5a5b11fe0a57b293b9cb623272e272c8bab69 --- .../java/org/apache/hadoop/fs/BulkDelete.java | 61 +--- .../apache/hadoop/fs/BulkDeleteSource.java | 7 +- .../java/org/apache/hadoop/fs/FileUtil.java | 12 +- .../site/markdown/filesystem/bulkdelete.md | 284 ++++++++++++++++++ .../org/apache/hadoop/fs/s3a/S3AStore.java | 1 + .../fs/s3a/impl/BulkDeleteOperation.java | 50 +-- .../BulkDeleteOperationCallbacksImpl.java | 51 +++- 7 files changed, 360 insertions(+), 106 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java index af454bc49a8a8..9e8b052a40a41 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java @@ -21,6 +21,7 @@ import java.io.Closeable; import java.io.IOException; import java.util.List; +import java.util.Map; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -81,65 +82,7 @@ public interface BulkDelete extends IOStatisticsSource, Closeable { * @throws IOException IO problems including networking, authentication and more. * @throws IllegalArgumentException if a path argument is invalid. */ - BulkDeleteOutcome bulkDelete(List paths) + List> bulkDelete(List paths) throws IOException, IllegalArgumentException; - /** - * The outcome: a list of paths which failed to delete. - * An empty list means all files were successfully deleted. - * There are no guarantees about the ordering of the list. - * Reasons for failures are not provided. - * File Not Found is not a failure. - */ - class BulkDeleteOutcome { - - /** - * List of paths which failed to delete. - */ - private final List failures; - - /** - * Constructor. - * @param failures list of failures. This must be non-null. - */ - public BulkDeleteOutcome(final List failures) { - this.failures = requireNonNull(failures); - } - - /** - * Get the list of failures. - * @return a possibly empty list of failures. - */ - public List getFailures() { - return failures; - } - } - - class BulkDeleteOutcomeElement { - private final Path path; - private final String error; - private final Exception exception; - - public BulkDeleteOutcomeElement( - final Path path, - final String error, - final Exception exception) { - this.path = path; - this.error = error; - this.exception = exception; - } - - public Path getPath() { - return path; - } - - public String getError() { - return error; - } - - public Exception getException() { - return exception; - } - } - } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java index d4cffb39bdf7c..1cc5de653ded8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java @@ -36,11 +36,6 @@ @InterfaceStability.Unstable public interface BulkDeleteSource { - /** - * Exception message for the case where bulk delete is not supported. - */ - String BULK_DELETE_NOT_SUPPORTED = "Bulk delete not supported"; - /** * Create a bulk delete operation. * There is no network IO at this point, simply the creation of @@ -54,7 +49,7 @@ public interface BulkDeleteSource { */ default BulkDelete createBulkDelete(Path path) throws UnsupportedOperationException, IllegalArgumentException, IOException { - throw new UnsupportedOperationException(BULK_DELETE_NOT_SUPPORTED); + throw new UnsupportedOperationException("Bulk delete not supported"); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java index 049a76da2da83..fabe589e24cf5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java @@ -55,7 +55,6 @@ import java.util.jar.Attributes; import java.util.jar.JarOutputStream; import java.util.jar.Manifest; -import java.util.stream.Collectors; import java.util.zip.GZIPInputStream; import org.apache.commons.collections.map.CaseInsensitiveMap; @@ -2133,7 +2132,7 @@ public static int bulkDeletePageSize(FileSystem fs, Path path) throws IOExceptio */ private static BulkDeleteSource toBulkDeleteSource(final FileSystem fs) { if (!(fs instanceof BulkDeleteSource)) { - throw new UnsupportedOperationException(BulkDeleteSource.BULK_DELETE_NOT_SUPPORTED); + throw new UnsupportedOperationException("Bulk delete not supported"); } return (BulkDeleteSource) fs; } @@ -2155,18 +2154,15 @@ private static BulkDeleteSource toBulkDeleteSource(final FileSystem fs) { * @param fs filesystem * @param base path to delete under. * @param paths list of paths which must be absolute and under the base path. - * @return a list of all the paths which couldn't be deleted for a reason other than "not found". + * @return a list of all the paths which couldn't be deleted for a reason other than "not found" and any associated error message. * @throws UnsupportedOperationException bulk delete under that path is not supported. * @throws IOException IO problems including networking, authentication and more. * @throws IllegalArgumentException if a path argument is invalid. */ - public static List bulkDelete(FileSystem fs, Path base, List paths) + public static List> bulkDelete(FileSystem fs, Path base, List paths) throws IOException { try (BulkDelete bulk = toBulkDeleteSource(fs).createBulkDelete(base)) { - final BulkDelete.BulkDeleteOutcome outcome = bulk.bulkDelete(paths); - return outcome.getFailures().stream() - .map(BulkDelete.BulkDeleteOutcomeElement::getPath) - .collect(Collectors.toList()); + return bulk.bulkDelete(paths); } } diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md new file mode 100644 index 0000000000000..2f98417c061d0 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md @@ -0,0 +1,284 @@ + + +# interface `BulkDelete` + + + +The `BulkDelete` interface provides an API to perform bulk delete of files/objects +in an object store or filesystem. + +## Key Features + +* An API for submitting a list of paths to delete. +* This list must be no larger than the "page size" supported by the client; This size is also exposed as a method. +* Triggers a request to delete files at the specific paths. +* Returns a list of which paths were reported as delete failures by the store. +* Does not consider a nonexistent file to be a failure. +* Does not offer any atomicity guarantees. +* Idempotency guarantees are weak: retries may delete files newly created by other clients. +* Provides no guarantees as to the outcome if a path references a directory. +* Provides no guarantees that parent directories will exist after the call. + + +The API is designed to match the semantics of the AWS S3 [Bulk Delete](https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html) REST API call, but it is not +exclusively restricted to this store. This is why the "provides no guarantees" +restrictions do not state what the outcome will be when executed on other stores. + +### Interface `org.apache.hadoop.fs.BulkDeleteSource` + +The interface `BulkDeleteSource` is offered by a FileSystem/FileContext class if +it supports the API. + +```java +@InterfaceAudience.Public +@InterfaceStability.Unstable +public interface BulkDeleteSource { + + /** + * Create a bulk delete operation. + * There is no network IO at this point, simply the creation of + * a bulk delete object. + * A path must be supplied to assist in link resolution. + * @param path path to delete under. + * @return the bulk delete. + * @throws UnsupportedOperationException bulk delete under that path is not supported. + * @throws IllegalArgumentException path not valid. + * @throws IOException problems resolving paths + */ + default BulkDelete createBulkDelete(Path path) + throws UnsupportedOperationException, IllegalArgumentException, IOException; + +} + +``` + +### Interface `org.apache.hadoop.fs.BulkDelete` + +This is the bulk delete implementation returned by the `createBulkDelete()` call. + +```java +/** + * API for bulk deletion of objects/files, + * but not directories. + * After use, call {@code close()} to release any resources and + * to guarantee store IOStatistics are updated. + *

+ * Callers MUST have no expectation that parent directories will exist after the + * operation completes; if an object store needs to explicitly look for and create + * directory markers, that step will be omitted. + *

+ * Be aware that on some stores (AWS S3) each object listed in a bulk delete counts + * against the write IOPS limit; large page sizes are counterproductive here, as + * are attempts at parallel submissions across multiple threads. + * @see HADOOP-16823. + * Large DeleteObject requests are their own Thundering Herd + *

+ */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public interface BulkDelete extends IOStatisticsSource, Closeable { + + /** + * The maximum number of objects/files to delete in a single request. + * @return a number greater than or equal to zero. + */ + int pageSize(); + + /** + * Base path of a bulk delete operation. + * All paths submitted in {@link #bulkDelete(List)} must be under this path. + */ + Path basePath(); + + /** + * Delete a list of files/objects. + *

    + *
  • Files must be under the path provided in {@link #basePath()}.
  • + *
  • The size of the list must be equal to or less than the page size + * declared in {@link #pageSize()}.
  • + *
  • Directories are not supported; the outcome of attempting to delete + * directories is undefined (ignored; undetected, listed as failures...).
  • + *
  • The operation is not atomic.
  • + *
  • The operation is treated as idempotent: network failures may + * trigger resubmission of the request -any new objects created under a + * path in the list may then be deleted.
  • + *
  • There is no guarantee that any parent directories exist after this call. + *
  • + *
+ * @param paths list of paths which must be absolute and under the base path. + * provided in {@link #basePath()}. + * @throws IOException IO problems including networking, authentication and more. + * @throws IllegalArgumentException if a path argument is invalid. + */ + List> bulkDelete(List paths) + throws IOException, IllegalArgumentException; + +} + +``` + +### `bulkDelete(paths)` + +#### Preconditions + +```python +if length(paths) > pageSize: throw IllegalArgumentException +``` + +#### Postconditions + +All paths which refer to files are removed from the set of files. +```python +FS'Files = FS.Files - [paths] +``` + +No other restrictions are placed upon the outcome. + + +### Availability + +The `BulkDeleteSource` interface is exported by `FileSystem` and `FileContext` storage clients +which MAY support the API; it may still be unsupported by the +specific instance. + +Use the `PathCapabilities` probe `fs.capability.bulk.delete`. + +```java +store.hasPathCapability(path, "fs.capability.bulk.delete") +``` + +### Invocation through Reflection. + +The need for many Libraries to compile against very old versions of Hadoop +means that most of the cloud-first Filesystem API calls cannot be used except +through reflection -And the more complicated The API and its data types are, +The harder that reflection is to implement. + +To assist this, the class `org.apache.hadoop.fs.FileUtil` has two methods +which are intended to provide simple access to the API, especially +through reflection. + +```java + /** + * Get the maximum number of objects/files to delete in a single request. + * @param fs filesystem + * @param path path to delete under. + * @return a number greater than or equal to zero. + * @throws UnsupportedOperationException bulk delete under that path is not supported. + * @throws IllegalArgumentException path not valid. + * @throws IOException problems resolving paths + */ + public static int bulkDeletePageSize(FileSystem fs, Path path) throws IOException; + + /** + * Delete a list of files/objects. + *
    + *
  • Files must be under the path provided in {@code base}.
  • + *
  • The size of the list must be equal to or less than the page size.
  • + *
  • Directories are not supported; the outcome of attempting to delete + * directories is undefined (ignored; undetected, listed as failures...).
  • + *
  • The operation is not atomic.
  • + *
  • The operation is treated as idempotent: network failures may + * trigger resubmission of the request -any new objects created under a + * path in the list may then be deleted.
  • + *
  • There is no guarantee that any parent directories exist after this call. + *
  • + *
+ * @param fs filesystem + * @param base path to delete under. + * @param paths list of paths which must be absolute and under the base path. + * @return a list of all the paths which couldn't be deleted for a reason other than "not found" and any associated error message. + * @throws UnsupportedOperationException bulk delete under that path is not supported. + * @throws IOException IO problems including networking, authentication and more. + * @throws IllegalArgumentException if a path argument is invalid. + */ + public static List> bulkDelete(FileSystem fs, Path base, List paths) +``` + +## S3A Implementation + +The S3A client exports this API. + +If multi-object delete is enabled (`fs.s3a.multiobjectdelete.enable` = true), as +it is by default, then the page size is limited to that defined in +`fs.s3a.bulk.delete.page.size`, which MUST be less than or equal to1000. +* The entire list of paths to delete is aggregated into a single bulk delete request, +issued to the store. +* Provided the caller has the correct permissions, every entry in the list + will, if the path references an object, cause that object to be deleted. +* If the path does not reference an object: the path will not be deleted + "This is for deleting objects, not directories" +* No probes for the existence of parent directories will take place; no + parent directory markers will be created. + "If you need parent directories, call mkdir() yourself" +* The list of failed keys listed in the `DeleteObjectsResponse` response + are converted into paths and returned along with their error messages. +* Network and other IO errors are raised as exceptions. + +If multi-object delete is disabled (or the list of size 1) +* A single `DELETE` call is issued +* Any `AccessDeniedException` raised is converted to a result in the error list. +* Any 404 response from a (non-AWS) store will be ignored. +* Network and other IO errors are raised as exceptions. + +Because there are no probes to ensure the call does not overwrite a directory, +or to see if a parentDirectory marker needs to be created, +this API is still faster than issuing a normal `FileSystem.delete(path)` call. + +That is: all the overhead normally undertaken to preserve the Posix System model are omitted. + + +### S3 Scalability and Performance + +Every entry in a bulk delete request counts as one write operation +against AWS S3 storage. +With the default write rate under a prefix on AWS S3 Standard storage +restricted to 3,500 writes/second, it is very easy to overload +the store by issuing a few bulk delete requests simultaneously. + +* If throttling is triggered then all clients interacting with + the store may observe performance issues. +* The write quota applies even for paths which do not exist. +* The S3A client *may* perform rate throttling as well as page size limiting. + +What does that mean? it means that attempting to issue multiple +bulk delete calls in parallel can be counterproductive. + +When overloaded, the S3 store returns a 403 throttle response. +This will trigger it back off and retry of posting the request. +However, the repeated request will still include the same number of objects and +*so generate the same load*. + +This can lead to a pathological situation where the repeated requests will +never be satisfied because the request itself is sufficient to overload the store. +See [HADOOP-16823.Large DeleteObject requests are their own Thundering Herd](https://issues.apache.org/jira/browse/HADOOP-16823) +for an example of where this did actually surface in production. + +This is why the default page size of S3A clients is 250 paths, not the store limit of 1000 entries. +It is also why the S3A delete/rename Operations do not attempt to do massive parallel deletions, +Instead bulk delete requests are queued for a single blocking thread to issue. +Consider a similar design. + + +When working with versioned S3 buckets, every path deleted will add a tombstone marker +to the store at that location, even if there was no object at that path. +While this has no negative performance impact on the bulk delete call, +it will slow down list requests subsequently made against that path. +That is: bulk delete requests of paths which do not exist will hurt future queries. + +Avoid this. Note also that TPC-DS Benchmark do not create the right load to make the +performance problems observable -but they can surface in production. +* Configure buckets to have a limited number of days for tombstones to be preserved. +* Do not delete which you know do not contain objects. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java index c029364743cce..da4f52a8f11a0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java @@ -104,6 +104,7 @@ Map.Entry deleteObjects(DeleteObjectsRequest de * Retry policy: retry untranslated; delete considered idempotent. * 404 errors other than bucket not found are swallowed; * this can be raised by third party stores (GCS). + *

* If an exception is caught and swallowed, the response will be empty; * otherwise it will be the response from the delete operation. * @param request request to make diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java index 61e666cacbfef..7cae4c81f198a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java @@ -19,19 +19,20 @@ package org.apache.hadoop.fs.s3a.impl; import java.io.IOException; -import java.util.Collections; import java.util.List; -import java.util.stream.Collectors; +import java.util.Map; -import software.amazon.awssdk.awscore.exception.AwsServiceException; import software.amazon.awssdk.services.s3.model.ObjectIdentifier; import org.apache.hadoop.fs.BulkDelete; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.Retries; import org.apache.hadoop.fs.store.audit.AuditSpan; +import org.apache.hadoop.util.functional.Tuples; +import static java.util.Collections.emptyList; import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.toList; import static org.apache.hadoop.util.Preconditions.checkArgument; /** @@ -45,7 +46,8 @@ public class BulkDeleteOperation extends AbstractStoreOperation implements BulkD private final int pageSize; - public BulkDeleteOperation(final StoreContext storeContext, + public BulkDeleteOperation( + final StoreContext storeContext, final BulkDeleteOperationCallbacks callbacks, final Path basePath, final int pageSize, @@ -68,30 +70,32 @@ public Path basePath() { } @Override - public BulkDeleteOutcome bulkDelete(final List paths) + public List> bulkDelete(final List paths) throws IOException, IllegalArgumentException { requireNonNull(paths); checkArgument(paths.size() <= pageSize, "Number of paths (%d) is larger than the page size (%d)", paths.size(), pageSize); + final StoreContext context = getStoreContext(); final List objects = paths.stream().map(p -> { checkArgument(p.isAbsolute(), "Path %s is not absolute", p); - final String k = getStoreContext().pathToKey(p); + final String k = context.pathToKey(p); return ObjectIdentifier.builder().key(k).build(); - }).collect(Collectors.toList()); + }).collect(toList()); - final List errors = callbacks.bulkDelete(objects); + final List> errors = callbacks.bulkDelete(objects); if (!errors.isEmpty()) { - final List outcomeElements = errors + final List> outcomeElements = errors .stream() - .map(error -> new BulkDeleteOutcomeElement( - getStoreContext().keyToPath(error), error, - null)) - .collect(Collectors.toList()); - return new BulkDeleteOutcome(outcomeElements); + .map(error -> Tuples.pair( + context.keyToPath(error.getKey()), + error.getValue() + )) + .collect(toList()); + return outcomeElements; } - return new BulkDeleteOutcome(Collections.emptyList()); + return emptyList(); } @Override @@ -99,20 +103,20 @@ public void close() throws IOException { } + /** + * Callbacks for the bulk delete operation. + */ public interface BulkDeleteOperationCallbacks { /** - * Attempt a bulk delete operation. + * Perform a bulk delete operation. * @param keys key list - * @return - * @throws MultiObjectDeleteException one or more of the keys could not - * be deleted in a multiple object delete operation. - * @throws AwsServiceException amazon-layer failure. - * @throws IOException other IO Exception. + * @return paths which failed to delete (if any). + * @throws IOException IO Exception. * @throws IllegalArgumentException illegal arguments */ @Retries.RetryTranslated - List bulkDelete(final List keys) - throws MultiObjectDeleteException, IOException, IllegalArgumentException; + List> bulkDelete(final List keys) + throws IOException, IllegalArgumentException; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperationCallbacksImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperationCallbacksImpl.java index 4c4c1dfe44aff..8cc02dca19848 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperationCallbacksImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperationCallbacksImpl.java @@ -18,9 +18,12 @@ package org.apache.hadoop.fs.s3a.impl; +import java.io.FileNotFoundException; import java.io.IOException; +import java.nio.file.AccessDeniedException; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse; @@ -30,9 +33,13 @@ import org.apache.hadoop.fs.s3a.Retries; import org.apache.hadoop.fs.s3a.S3AStore; import org.apache.hadoop.fs.store.audit.AuditSpan; +import org.apache.hadoop.util.functional.Tuples; +import static java.util.Collections.emptyList; +import static java.util.Collections.singletonList; import static org.apache.hadoop.fs.s3a.Invoker.once; import static org.apache.hadoop.util.Preconditions.checkArgument; +import static org.apache.hadoop.util.functional.Tuples.pair; /** * Callbacks for the bulk delete operation. @@ -40,13 +47,20 @@ public class BulkDeleteOperationCallbacksImpl implements BulkDeleteOperation.BulkDeleteOperationCallbacks { + /** + * Path for logging. + */ private final String path; + /** Page size for bulk delete. */ private final int pageSize; /** span for operations. */ private final AuditSpan span; + /** + * Store. + */ private final S3AStore store; @@ -60,21 +74,18 @@ public BulkDeleteOperationCallbacksImpl(final S3AStore store, @Override @Retries.RetryTranslated - public List bulkDelete(final List keysToDelete) - throws MultiObjectDeleteException, IOException, IllegalArgumentException { + public List> bulkDelete(final List keysToDelete) + throws IOException, IllegalArgumentException { span.activate(); final int size = keysToDelete.size(); checkArgument(size <= pageSize, "Too many paths to delete in one operation: %s", size); if (size == 0) { - return Collections.emptyList(); + return emptyList(); } + if (size == 1) { - store.deleteObject(store.getRequestFactory() - .newDeleteObjectRequestBuilder(keysToDelete.get(0).key()) - .build()); - // no failures, so return an empty list - return Collections.emptyList(); + return deleteSingleObject(keysToDelete.get(0).key()); } final DeleteObjectsResponse response = once("bulkDelete", path, () -> @@ -84,11 +95,31 @@ public List bulkDelete(final List keysToDelete) final List errors = response.errors(); if (errors.isEmpty()) { // all good. - return Collections.emptyList(); + return emptyList(); } else { return errors.stream() - .map(S3Error::key) + .map(e -> pair(e.key(), e.message())) .collect(Collectors.toList()); } } + + /** + * Delete a single object. + * @param key key to delete + * @return list of keys which failed to delete: length 0 or 1. + * @throws IOException IO problem other than AccessDeniedException + */ + @Retries.RetryTranslated + private List> deleteSingleObject(final String key) throws IOException { + try { + once("bulkDelete", path, () -> + store.deleteObject(store.getRequestFactory() + .newDeleteObjectRequestBuilder(key) + .build())); + } catch (AccessDeniedException e) { + return singletonList(pair(key, e.toString())); + } + return emptyList(); + + } } From a61f139ca5007326a1eeae78f150ec03f107d50f Mon Sep 17 00:00:00 2001 From: Mukund Thakur Date: Mon, 8 Apr 2024 18:14:09 -0500 Subject: [PATCH 06/19] Adding some contract tests. This will evolve. and some minor prod changes. --- hadoop-common-project/hadoop-common/pom.xml | 6 + .../java/org/apache/hadoop/fs/BulkDelete.java | 2 +- .../fs/statistics/StoreStatisticNames.java | 3 + .../AbstractContractBulkDeleteTest.java | 222 ++++++++++++++++++ .../org/apache/hadoop/fs/s3a/Constants.java | 3 + .../apache/hadoop/fs/s3a/S3AFileSystem.java | 12 +- .../org/apache/hadoop/fs/s3a/Statistic.java | 4 + .../fs/s3a/impl/BulkDeleteOperation.java | 13 +- .../BulkDeleteOperationCallbacksImpl.java | 2 +- .../hadoop/fs/s3a/impl/S3AStoreImpl.java | 18 +- .../s3a/ITestS3AContractBulkDelete.java | 126 ++++++++++ .../hadoop/fs/s3a/auth/ITestAssumeRole.java | 58 ++++- 12 files changed, 444 insertions(+), 25 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml index a7dcbb24a9b40..3a9189c4d914d 100644 --- a/hadoop-common-project/hadoop-common/pom.xml +++ b/hadoop-common-project/hadoop-common/pom.xml @@ -429,6 +429,12 @@ lz4-java provided + + org.testng + testng + RELEASE + test + diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java index 9e8b052a40a41..b5bdf14bf9895 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java @@ -52,7 +52,7 @@ public interface BulkDelete extends IOStatisticsSource, Closeable { /** * The maximum number of objects/files to delete in a single request. - * @return a number greater than or equal to zero. + * @return a number greater than zero. */ int pageSize(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java index d7d4bc45dba08..a513cffd849b6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java @@ -197,6 +197,9 @@ public final class StoreStatisticNames { public static final String STORE_IO_RETRY = "store_io_retry"; + public static final String STORE_IO_RATE_LIMITED_DURATION + = "store_io_rate_limited_duration"; + /** * A store's equivalent of a paged LIST request was initiated: {@value}. */ diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java new file mode 100644 index 0000000000000..615253c1187b1 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java @@ -0,0 +1,222 @@ +/* + * 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.hadoop.fs.contract; + +import org.apache.hadoop.fs.*; +import org.assertj.core.api.Assertions; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +public abstract class AbstractContractBulkDeleteTest extends AbstractFSContractTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(AbstractContractBulkDeleteTest.class); + + protected int pageSize; + + protected Path basePath; + + protected FileSystem fs; + + @Before + public void setUp() throws Exception { + fs = getFileSystem(); + basePath = path(getClass().getName()); + pageSize = FileUtil.bulkDeletePageSize(getFileSystem(), basePath); + fs.mkdirs(basePath); + } + + public Path getBasePath() { + return basePath; + } + + /** + * Validate the page size for bulk delete operation. Different stores can have different + * implementations for bulk delete operation thus different page size. + */ + @Test + public void validatePageSize() throws Exception { + Assertions.assertThat(pageSize) + .describedAs("Page size should be 1 by default for all stores") + .isEqualTo(1); + } + + @Test + public void testPathsSizeEqualsPageSizePrecondition() throws Exception { + List listOfPaths = createListOfPaths(pageSize, basePath); + // Bulk delete call should pass with no exception. + FileUtil.bulkDelete(getFileSystem(), basePath, listOfPaths); + } + + @Test + public void testPathsSizeGreaterThanPageSizePrecondition() throws Exception { + List listOfPaths = createListOfPaths(pageSize + 1, basePath); + intercept(IllegalArgumentException.class, + () -> FileUtil.bulkDelete(getFileSystem(), basePath, listOfPaths)); + } + + @Test + public void testPathsSizeLessThanPageSizePrecondition() throws Exception { + List listOfPaths = createListOfPaths(pageSize - 1, basePath); + // Bulk delete call should pass with no exception. + FileUtil.bulkDelete(getFileSystem(), basePath, listOfPaths); + } + + @Test + public void testBulkDeleteSuccessful() throws Exception { + List listOfPaths = createListOfPaths(pageSize, basePath); + for (Path path : listOfPaths) { + touch(fs, path); + } + FileStatus[] fileStatuses = fs.listStatus(basePath); + Assertions.assertThat(fileStatuses) + .describedAs("File count after create") + .hasSize(pageSize); + assertSuccessfulBulkDelete(FileUtil.bulkDelete(getFileSystem(), basePath, listOfPaths)); + FileStatus[] fileStatusesAfterDelete = fs.listStatus(basePath); + Assertions.assertThat(fileStatusesAfterDelete) + .describedAs("File statuses should be empty after delete") + .isEmpty(); + } + + @Test + public void validatePathCapabilityDeclared() throws Exception { + Assertions.assertThat(fs.hasPathCapability(basePath, CommonPathCapabilities.BULK_DELETE)) + .describedAs("Path capability BULK_DELETE should be declared") + .isTrue(); + } + + @Test + public void testDeletePathsNotUnderBase() throws Exception { + List paths = new ArrayList<>(); + Path pathNotUnderBase = path("not-under-base"); + paths.add(pathNotUnderBase); + // Should fail as path is not under the base path. + intercept(IllegalArgumentException.class, + () -> FileUtil.bulkDelete(getFileSystem(), basePath, paths)); + } + + @Test + public void testDeletePathsNotExists() throws Exception { + List paths = new ArrayList<>(); + Path pathNotExists = new Path(basePath, "not-exists"); + paths.add(pathNotExists); + // bulk delete call doesn't verify if a path exist or not before deleting. + assertSuccessfulBulkDelete(FileUtil.bulkDelete(getFileSystem(), basePath, paths)); + } + + @Test + public void testDeletePathsDirectory() throws Exception { + List paths = new ArrayList<>(); + Path dirPath = new Path(basePath, "dir"); + fs.mkdirs(dirPath); + paths.add(dirPath); + Path filePath = new Path(dirPath, "file"); + touch(fs, filePath); + paths.add(filePath); + // Outcome is undefined. But call shouldn't fail. In case of S3 directories will still be present. + assertSuccessfulBulkDelete(FileUtil.bulkDelete(getFileSystem(), basePath, paths)); + } + + @Test + public void testDeleteEmptyDirectory() throws Exception { + List paths = new ArrayList<>(); + Path emptyDirPath = new Path(basePath, "empty-dir"); + fs.mkdirs(emptyDirPath); + paths.add(emptyDirPath); + // Should pass as empty directory. + assertSuccessfulBulkDelete(FileUtil.bulkDelete(getFileSystem(), basePath, paths)); + } + + @Test + public void testDeleteEmptyList() throws Exception { + List paths = new ArrayList<>(); + // Empty list should pass. + assertSuccessfulBulkDelete(FileUtil.bulkDelete(getFileSystem(), basePath, paths)); + } + + @Test + public void testDeleteSamePathsMoreThanOnce() throws Exception { + List paths = new ArrayList<>(); + Path path = new Path(basePath, "file"); + touch(fs, path); + paths.add(path); + paths.add(path); + Path another = new Path(basePath, "another-file"); + touch(fs, another); + paths.add(another); + assertSuccessfulBulkDelete(FileUtil.bulkDelete(getFileSystem(), basePath, paths)); + } + + /** + * This test validates that files to be deleted don't have + * to be direct children of the base path. + */ + @Test + public void testDeepDirectoryFilesDelete() throws Exception { + List paths = new ArrayList<>(); + Path dir1 = new Path(basePath, "dir1"); + Path dir2 = new Path(dir1, "dir2"); + Path dir3 = new Path(dir2, "dir3"); + fs.mkdirs(dir3); + Path file1 = new Path(dir3, "file1"); + touch(fs, file1); + paths.add(file1); + assertSuccessfulBulkDelete(FileUtil.bulkDelete(getFileSystem(), basePath, paths)); + } + + + @Test + public void testChildPaths() throws Exception { + List paths = new ArrayList<>(); + Path dirPath = new Path(basePath, "dir"); + fs.mkdirs(dirPath); + paths.add(dirPath); + Path filePath = new Path(dirPath, "file"); + touch(fs, filePath); + paths.add(filePath); + // Should pass as both paths are under the base path. + assertSuccessfulBulkDelete(FileUtil.bulkDelete(getFileSystem(), basePath, paths)); + } + + + public static void assertSuccessfulBulkDelete(List> entries) { + Assertions.assertThat(entries) + .describedAs("return entries should be empty after successful delete") + .isEmpty(); + } + + private List createListOfPaths(int count, Path basePath) { + List paths = new ArrayList<>(); + for (int i=0; i < count; i++) { + Path path = new Path(basePath, "file-" + i); + paths.add(path); + } + return paths; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index 67df37e5ebfcd..55607be78f534 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -1641,4 +1641,7 @@ private Constants() { */ public static final String AWS_S3_ACCESS_GRANTS_FALLBACK_TO_IAM_ENABLED = "fs.s3a.access.grants.fallback.to.iam"; + public static final int DEFAULT_S3A_IO_RATE_LIMIT = 0; + + public static final String S3A_IO_RATE_LIMIT = "fs.s3a.io.rate.limit"; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index fbaa8c8f4cfa2..76cab15e3986f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -52,6 +52,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import javax.annotation.Nullable; +import org.apache.hadoop.util.*; import software.amazon.awssdk.core.ResponseInputStream; import software.amazon.awssdk.core.exception.SdkException; import software.amazon.awssdk.services.s3.S3AsyncClient; @@ -166,10 +167,6 @@ import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.token.DelegationTokenIssuer; import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.hadoop.util.DurationInfo; -import org.apache.hadoop.util.LambdaUtils; -import org.apache.hadoop.util.Lists; -import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -207,12 +204,8 @@ import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.fs.store.EtagChecksum; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.util.BlockingThreadPoolExecutorService; import org.apache.hadoop.security.ProviderUtils; import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.util.Progressable; -import org.apache.hadoop.util.ReflectionUtils; -import org.apache.hadoop.util.SemaphoredDelegatingExecutor; import org.apache.hadoop.util.concurrent.HadoopExecutors; import org.apache.hadoop.util.functional.CallableRaisingIOE; @@ -760,6 +753,7 @@ public void initialize(URI name, Configuration originalConf) LOG.debug("Using optimized copyFromLocal implementation: {}", optimizedCopyFromLocal); s3AccessGrantsEnabled = conf.getBoolean(AWS_S3_ACCESS_GRANTS_ENABLED, false); + int rateLimitCapacity = intOption(conf, S3A_IO_RATE_LIMIT, DEFAULT_S3A_IO_RATE_LIMIT, 0); // now create the store store = new S3AStoreBuilder() .withS3Client(s3Client) @@ -770,7 +764,7 @@ public void initialize(URI name, Configuration originalConf) .withStatisticsContext(statisticsContext) .withStorageStatistics(getStorageStatistics()) .withReadRateLimiter(unlimitedRate()) - .withWriteRateLimiter(unlimitedRate()) + .withWriteRateLimiter(RateLimitingFactory.create(rateLimitCapacity)) .build(); // The filesystem is now ready to perform operations against diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java index 7102623996e95..7c4883c3d9967 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java @@ -543,6 +543,10 @@ public enum Statistic { "retried requests made of the remote store", TYPE_COUNTER), + STORE_IO_RATE_LIMITED(StoreStatisticNames.STORE_IO_RATE_LIMITED_DURATION, + "Duration of rate limited operations", + TYPE_DURATION), + STORE_IO_THROTTLED( StoreStatisticNames.STORE_IO_THROTTLED, "Requests throttled and retried", diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java index 7cae4c81f198a..2c232c231708d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java @@ -75,10 +75,11 @@ public List> bulkDelete(final List paths) requireNonNull(paths); checkArgument(paths.size() <= pageSize, "Number of paths (%d) is larger than the page size (%d)", paths.size(), pageSize); - final StoreContext context = getStoreContext(); final List objects = paths.stream().map(p -> { checkArgument(p.isAbsolute(), "Path %s is not absolute", p); + checkArgument(validatePathIsUnderParent(p), + "Path %s is not under the base path %s", p, basePath); final String k = context.pathToKey(p); return ObjectIdentifier.builder().key(k).build(); }).collect(toList()); @@ -98,6 +99,16 @@ public List> bulkDelete(final List paths) return emptyList(); } + private boolean validatePathIsUnderParent(Path p) { + while (p.getParent() != null) { + if (p.getParent().equals(basePath)) { + return true; + } + p = p.getParent(); + } + return false; + } + @Override public void close() throws IOException { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperationCallbacksImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperationCallbacksImpl.java index 8cc02dca19848..2edcc3c7bbd3a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperationCallbacksImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperationCallbacksImpl.java @@ -98,7 +98,7 @@ public List> bulkDelete(final List k return emptyList(); } else { return errors.stream() - .map(e -> pair(e.key(), e.message())) + .map(e -> pair(e.key(), e.toString())) .collect(Collectors.toList()); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java index 6d60b71a95cb2..c808ddcb5719c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java @@ -57,15 +57,10 @@ import static java.util.Objects.requireNonNull; import static org.apache.hadoop.fs.s3a.S3AUtils.isThrottleException; -import static org.apache.hadoop.fs.s3a.Statistic.IGNORED_ERRORS; -import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_BULK_DELETE_REQUEST; -import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_OBJECTS; -import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUEST; -import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_RETRY; -import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLED; -import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_THROTTLE_RATE; +import static org.apache.hadoop.fs.s3a.Statistic.*; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DELETE_CONSIDERED_IDEMPOTENT; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.STORE_IO_RATE_LIMITED_DURATION; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation; import static org.apache.hadoop.util.Preconditions.checkArgument; @@ -194,6 +189,10 @@ protected void incrementStatistic(Statistic statistic) { incrementStatistic(statistic, 1); } + protected void incrementDurationStatistic(Statistic statistic, Duration duration) { + statisticsContext.addValueToQuantiles(statistic, duration.toMillis()); + } + /** * Increment a statistic by a specific value. * This increments both the instrumentation and storage statistics. @@ -335,8 +334,9 @@ public Map.Entry deleteObjects(final DeleteObje // duration is tracked in the bulk delete counters trackDurationOfOperation(getDurationTrackerFactory(), OBJECT_BULK_DELETE_REQUEST.getSymbol(), () -> { - acquireWriteCapacity(keyCount); - incrementStatistic(OBJECT_DELETE_OBJECTS, keyCount); + Duration durationToAcquireWriteCapacity = acquireWriteCapacity(keyCount); + instrumentation.recordDuration(STORE_IO_RATE_LIMITED, true, durationToAcquireWriteCapacity); + incrementStatistic(OBJECT_DELETE_OBJECTS, keyCount); return s3Client.deleteObjects(deleteRequest); })); if (!response.errors().isEmpty()) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java new file mode 100644 index 0000000000000..6e3a575f625fc --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java @@ -0,0 +1,126 @@ +/* + * 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.hadoop.fs.contract.s3a; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.BulkDelete; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.AbstractContractBulkDeleteTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.s3a.Constants; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.S3ATestUtils; +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.List; + +import static java.util.stream.Collectors.toList; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.createFiles; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.STORE_IO_RATE_LIMITED_DURATION; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +public class ITestS3AContractBulkDelete extends AbstractContractBulkDeleteTest { + + private static final Logger LOG = LoggerFactory.getLogger(ITestS3AContractBulkDelete.class); + + /** + * Delete Page size: {@value}. + * This is the default page size for bulk delete operation for this contract test. + * All the tests in this class should pass number of paths equal to or less than + * this page size during the bulk delete operation. + */ + private static final int DELETE_PAGE_SIZE = 20; + + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + S3ATestUtils.disableFilesystemCaching(conf); + S3ATestUtils.removeBaseAndBucketOverrides(conf, + Constants.BULK_DELETE_PAGE_SIZE); + conf.setInt(Constants.BULK_DELETE_PAGE_SIZE, DELETE_PAGE_SIZE); + return conf; + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new S3AContract(createConfiguration()); + } + + @Override + public void validatePageSize() throws Exception { + Assertions.assertThat(pageSize) + .describedAs("Page size should match the configured page size") + .isEqualTo(DELETE_PAGE_SIZE); + } + + @Test + public void testBulkDeleteZeroPageSizePrecondition() throws Exception { + Configuration conf = getContract().getConf(); + conf.setInt(Constants.BULK_DELETE_PAGE_SIZE, 0); + Path testPath = path(getMethodName()); + try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) { + intercept(IllegalArgumentException.class, + () -> fs.createBulkDelete(testPath)); + } + } + + @Test + public void testPageSizeWhenMultiObjectsDisabled() throws Exception { + Configuration conf = getContract().getConf(); + conf.setBoolean(Constants.ENABLE_MULTI_DELETE, false); + Path testPath = path(getMethodName()); + try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) { + BulkDelete bulkDelete = fs.createBulkDelete(testPath); + Assertions.assertThat(bulkDelete.pageSize()) + .describedAs("Page size should be 1 when multi-object delete is disabled") + .isEqualTo(1); + } + } + + @Test + public void testRateLimiting() throws Exception { + Configuration conf = getContract().getConf(); + conf.setInt(Constants.S3A_IO_RATE_LIMIT, 5); + Path basePath = path(getMethodName()); + try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) { + createFiles(fs, basePath, 1, 20, 0); + FileStatus[] fileStatuses = fs.listStatus(basePath); + List paths = Arrays.stream(fileStatuses) + .map(FileStatus::getPath) + .collect(toList()); + BulkDelete bulkDelete = fs.createBulkDelete(basePath); + bulkDelete.bulkDelete(paths); + String mean = STORE_IO_RATE_LIMITED_DURATION + ".mean"; + Assertions.assertThat(fs.getIOStatistics().meanStatistics().get(mean).mean()) + .describedAs("Rate limiting should not have happened during first delete call") + .isEqualTo(0.0); + bulkDelete.bulkDelete(paths); + bulkDelete.bulkDelete(paths); + bulkDelete.bulkDelete(paths); + Assertions.assertThat(fs.getIOStatistics().meanStatistics().get(mean).mean()) + .describedAs("Rate limiting should have happened during multiple delete calls") + .isGreaterThan(0.0); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java index a7ccc92e133c8..cfbcb91509d71 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java @@ -23,9 +23,14 @@ import java.io.IOException; import java.net.URI; import java.nio.file.AccessDeniedException; +import java.util.ArrayList; +import java.util.Arrays; import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; import java.util.stream.IntStream; +import org.apache.hadoop.fs.*; import software.amazon.awssdk.auth.credentials.AwsCredentials; import software.amazon.awssdk.services.s3.model.MultipartUpload; import software.amazon.awssdk.services.sts.model.StsException; @@ -38,10 +43,6 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocatedFileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.AWSBadRequestException; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; @@ -56,6 +57,7 @@ import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; +import static org.apache.hadoop.fs.contract.AbstractContractBulkDeleteTest.assertSuccessfulBulkDelete; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.Constants.S3EXPRESS_CREATE_SESSION; @@ -702,6 +704,54 @@ public void testPartialDeleteSingleDelete() throws Throwable { executePartialDelete(createAssumedRoleConfig(), true); } + @Test + public void testBulkDelete() throws Throwable { + describe("Bulk delete with part of the child tree read only"); + executeBulkDelete(createAssumedRoleConfig()); + } + + private void executeBulkDelete(Configuration assumedRoleConfig) throws Exception { + Path destDir = methodPath(); + Path readOnlyDir = new Path(destDir, "readonlyDir"); + + // the full FS + S3AFileSystem fs = getFileSystem(); + FileUtil.bulkDelete(fs, destDir, new ArrayList<>()); + + bindRolePolicyStatements(assumedRoleConfig, STATEMENT_ALLOW_KMS_RW, + statement(true, S3_ALL_BUCKETS, S3_ALL_OPERATIONS), + new Statement(Effects.Deny) + .addActions(S3_PATH_WRITE_OPERATIONS) + .addResources(directory(readOnlyDir)) + ); + roleFS = (S3AFileSystem) destDir.getFileSystem(assumedRoleConfig); + + int range = 10; + touchFiles(fs, readOnlyDir, range); + touchFiles(roleFS, destDir, range); + FileStatus[] fileStatuses = roleFS.listStatus(readOnlyDir); + List pathsToDelete = Arrays.stream(fileStatuses) + .map(FileStatus::getPath) + .collect(Collectors.toList()); + // bulk delete in the read only FS should fail. + BulkDelete bulkDelete = roleFS.createBulkDelete(readOnlyDir); + assertAccessDeniedForEachPath(bulkDelete.bulkDelete(pathsToDelete)); + BulkDelete bulkDelete2 = roleFS.createBulkDelete(destDir); + assertAccessDeniedForEachPath(bulkDelete2.bulkDelete(pathsToDelete)); + // delete the files in the original FS should succeed. + BulkDelete bulkDelete3 = fs.createBulkDelete(readOnlyDir); + assertSuccessfulBulkDelete(bulkDelete3.bulkDelete(pathsToDelete)); + BulkDelete bulkDelete4 = fs.createBulkDelete(destDir); + assertSuccessfulBulkDelete(bulkDelete4.bulkDelete(pathsToDelete)); + // we can write a test for some successful and some failure as well. + } + + private void assertAccessDeniedForEachPath(List> entries) { + for (Map.Entry entry : entries) { + Assertions.assertThat(entry.getValue()).contains("AccessDenied"); + } + } + /** * Have a directory with full R/W permissions, but then remove * write access underneath, and try to delete it. From 8ef09f7350e6d8abb6caaf5a258339374efcf5a0 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 16 Apr 2024 14:22:47 +0100 Subject: [PATCH 07/19] HADOOP-18679. My changes based on iceberg PoC We are going to need a default FS impl which just invokes delete(path, false) and maps any IOE to a failure. Change-Id: If56bca7cb8529ccbfbb1dfa29cedc8287ec980d4 --- .../java/org/apache/hadoop/fs/BulkDelete.java | 5 +- .../apache/hadoop/fs/BulkDeleteSource.java | 9 ++ .../java/org/apache/hadoop/fs/FileUtil.java | 58 --------- .../apache/hadoop/io/wrappedio/WrappedIO.java | 116 ++++++++++++++++++ .../AbstractContractBulkDeleteTest.java | 41 ++++--- .../fs/s3a/impl/BulkDeleteOperation.java | 3 +- .../hadoop/fs/s3a/auth/ITestAssumeRole.java | 3 +- 7 files changed, 156 insertions(+), 79 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java index b5bdf14bf9895..c768ef93062ba 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java @@ -20,6 +20,7 @@ import java.io.Closeable; import java.io.IOException; +import java.util.Collection; import java.util.List; import java.util.Map; @@ -58,7 +59,7 @@ public interface BulkDelete extends IOStatisticsSource, Closeable { /** * Base path of a bulk delete operation. - * All paths submitted in {@link #bulkDelete(List)} must be under this path. + * All paths submitted in {@link #bulkDelete(Collection)} must be under this path. */ Path basePath(); @@ -82,7 +83,7 @@ public interface BulkDelete extends IOStatisticsSource, Closeable { * @throws IOException IO problems including networking, authentication and more. * @throws IllegalArgumentException if a path argument is invalid. */ - List> bulkDelete(List paths) + List> bulkDelete(Collection paths) throws IOException, IllegalArgumentException; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java index 1cc5de653ded8..53e45408638ac 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java @@ -52,4 +52,13 @@ default BulkDelete createBulkDelete(Path path) throw new UnsupportedOperationException("Bulk delete not supported"); } + /** + * Is bulk delete available on this path for this source? + * @param path path to delete under. + * @return true if bulk delete is available. + */ + default boolean isBulkDeleteAvailable(Path path) { + return true; + } + } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java index fabe589e24cf5..fa87bb48aaa69 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java @@ -2108,62 +2108,4 @@ public static void maybeIgnoreMissingDirectory(FileSystem fs, LOG.info("Ignoring missing directory {}", path); LOG.debug("Directory missing", e); } - - /** - * Get the maximum number of objects/files to delete in a single request. - * @param fs filesystem - * @param path path to delete under. - * @return a number greater than or equal to zero. - * @throws UnsupportedOperationException bulk delete under that path is not supported. - * @throws IllegalArgumentException path not valid. - * @throws IOException problems resolving paths - */ - public static int bulkDeletePageSize(FileSystem fs, Path path) throws IOException { - try (BulkDelete bulk = toBulkDeleteSource(fs).createBulkDelete(path)) { - return bulk.pageSize(); - } - } - - /** - * Convert a filesystem to a bulk delete source. - * @param fs filesystem - * @return cast fs. - * @throws UnsupportedOperationException FS doesn't implement the interface. - */ - private static BulkDeleteSource toBulkDeleteSource(final FileSystem fs) { - if (!(fs instanceof BulkDeleteSource)) { - throw new UnsupportedOperationException("Bulk delete not supported"); - } - return (BulkDeleteSource) fs; - } - - /** - * Delete a list of files/objects. - *

    - *
  • Files must be under the path provided in {@code base}.
  • - *
  • The size of the list must be equal to or less than the page size.
  • - *
  • Directories are not supported; the outcome of attempting to delete - * directories is undefined (ignored; undetected, listed as failures...).
  • - *
  • The operation is not atomic.
  • - *
  • The operation is treated as idempotent: network failures may - * trigger resubmission of the request -any new objects created under a - * path in the list may then be deleted.
  • - *
  • There is no guarantee that any parent directories exist after this call. - *
  • - *
- * @param fs filesystem - * @param base path to delete under. - * @param paths list of paths which must be absolute and under the base path. - * @return a list of all the paths which couldn't be deleted for a reason other than "not found" and any associated error message. - * @throws UnsupportedOperationException bulk delete under that path is not supported. - * @throws IOException IO problems including networking, authentication and more. - * @throws IllegalArgumentException if a path argument is invalid. - */ - public static List> bulkDelete(FileSystem fs, Path base, List paths) - throws IOException { - try (BulkDelete bulk = toBulkDeleteSource(fs).createBulkDelete(base)) { - return bulk.bulkDelete(paths); - } - } - } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java new file mode 100644 index 0000000000000..af5249195ca8d --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java @@ -0,0 +1,116 @@ +/* + * 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.hadoop.io.wrappedio; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.BulkDelete; +import org.apache.hadoop.fs.BulkDeleteSource; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +/** + * Reflection-friendly access to APIs which are not available in + * some of the older Hadoop versions which libraries still + * compile against. + *

+ * The intent is to avoid the need for complex reflection operations + * including wrapping of parameter classes, direct instatiation of + * new classes etc. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class WrappedIO { + + private WrappedIO() { + } + + /** + * Is bulk delete available on a filesystem? + * @param fs filesystem + * @param path path to delete under. + * @return true if bulk delete is available. + */ + public static boolean isBulkDeleteAvailable(FileSystem fs, Path path) { + return (fs instanceof BulkDeleteSource) + && ((BulkDeleteSource) fs).isBulkDeleteAvailable(path); + } + + /** + * Get the maximum number of objects/files to delete in a single request. + * @param fs filesystem + * @param path path to delete under. + * @return a number greater than or equal to zero. + * @throws UnsupportedOperationException bulk delete under that path is not supported. + * @throws IllegalArgumentException path not valid. + * @throws IOException problems resolving paths + */ + public static int bulkDeletePageSize(FileSystem fs, Path path) throws IOException { + try (BulkDelete bulk = toBulkDeleteSource(fs).createBulkDelete(path)) { + return bulk.pageSize(); + } + } + + /** + * Convert a filesystem to a bulk delete source. + * @param fs filesystem + * @return cast fs. + * @throws UnsupportedOperationException FS doesn't implement the interface. + */ + private static BulkDeleteSource toBulkDeleteSource(final FileSystem fs) { + if (!(fs instanceof BulkDeleteSource)) { + throw new UnsupportedOperationException("Bulk delete not supported"); + } + return (BulkDeleteSource) fs; + } + + /** + * Delete a list of files/objects. + *

    + *
  • Files must be under the path provided in {@code base}.
  • + *
  • The size of the list must be equal to or less than the page size.
  • + *
  • Directories are not supported; the outcome of attempting to delete + * directories is undefined (ignored; undetected, listed as failures...).
  • + *
  • The operation is not atomic.
  • + *
  • The operation is treated as idempotent: network failures may + * trigger resubmission of the request -any new objects created under a + * path in the list may then be deleted.
  • + *
  • There is no guarantee that any parent directories exist after this call. + *
  • + *
+ * @param fs filesystem + * @param base path to delete under. + * @param paths list of paths which must be absolute and under the base path. + * @return a list of all the paths which couldn't be deleted for a reason other than "not found" and any associated error message. + * @throws UnsupportedOperationException bulk delete under that path is not supported. + * @throws IOException IO problems including networking, authentication and more. + * @throws IllegalArgumentException if a path argument is invalid. + */ + public static List> bulkDelete(FileSystem fs, Path base, Collection paths) + throws IOException { + try (BulkDelete bulk = toBulkDeleteSource(fs).createBulkDelete(base)) { + return bulk.bulkDelete(paths); + } + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java index 615253c1187b1..09a219efe1ad5 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java @@ -18,18 +18,24 @@ package org.apache.hadoop.fs.contract; -import org.apache.hadoop.fs.*; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + import org.assertj.core.api.Assertions; import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; +import org.apache.hadoop.fs.CommonPathCapabilities; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.wrappedio.WrappedIO; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.io.wrappedio.WrappedIO.bulkDelete; import static org.apache.hadoop.test.LambdaTestUtils.intercept; public abstract class AbstractContractBulkDeleteTest extends AbstractFSContractTestBase { @@ -47,7 +53,7 @@ public abstract class AbstractContractBulkDeleteTest extends AbstractFSContractT public void setUp() throws Exception { fs = getFileSystem(); basePath = path(getClass().getName()); - pageSize = FileUtil.bulkDeletePageSize(getFileSystem(), basePath); + pageSize = WrappedIO.bulkDeletePageSize(getFileSystem(), basePath); fs.mkdirs(basePath); } @@ -70,21 +76,21 @@ public void validatePageSize() throws Exception { public void testPathsSizeEqualsPageSizePrecondition() throws Exception { List listOfPaths = createListOfPaths(pageSize, basePath); // Bulk delete call should pass with no exception. - FileUtil.bulkDelete(getFileSystem(), basePath, listOfPaths); + bulkDelete(getFileSystem(), basePath, listOfPaths); } @Test public void testPathsSizeGreaterThanPageSizePrecondition() throws Exception { List listOfPaths = createListOfPaths(pageSize + 1, basePath); intercept(IllegalArgumentException.class, - () -> FileUtil.bulkDelete(getFileSystem(), basePath, listOfPaths)); + () -> bulkDelete(getFileSystem(), basePath, listOfPaths)); } @Test public void testPathsSizeLessThanPageSizePrecondition() throws Exception { List listOfPaths = createListOfPaths(pageSize - 1, basePath); // Bulk delete call should pass with no exception. - FileUtil.bulkDelete(getFileSystem(), basePath, listOfPaths); + bulkDelete(getFileSystem(), basePath, listOfPaths); } @Test @@ -97,7 +103,8 @@ public void testBulkDeleteSuccessful() throws Exception { Assertions.assertThat(fileStatuses) .describedAs("File count after create") .hasSize(pageSize); - assertSuccessfulBulkDelete(FileUtil.bulkDelete(getFileSystem(), basePath, listOfPaths)); + assertSuccessfulBulkDelete( + bulkDelete(getFileSystem(), basePath, listOfPaths)); FileStatus[] fileStatusesAfterDelete = fs.listStatus(basePath); Assertions.assertThat(fileStatusesAfterDelete) .describedAs("File statuses should be empty after delete") @@ -118,7 +125,7 @@ public void testDeletePathsNotUnderBase() throws Exception { paths.add(pathNotUnderBase); // Should fail as path is not under the base path. intercept(IllegalArgumentException.class, - () -> FileUtil.bulkDelete(getFileSystem(), basePath, paths)); + () -> bulkDelete(getFileSystem(), basePath, paths)); } @Test @@ -127,7 +134,7 @@ public void testDeletePathsNotExists() throws Exception { Path pathNotExists = new Path(basePath, "not-exists"); paths.add(pathNotExists); // bulk delete call doesn't verify if a path exist or not before deleting. - assertSuccessfulBulkDelete(FileUtil.bulkDelete(getFileSystem(), basePath, paths)); + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); } @Test @@ -140,7 +147,7 @@ public void testDeletePathsDirectory() throws Exception { touch(fs, filePath); paths.add(filePath); // Outcome is undefined. But call shouldn't fail. In case of S3 directories will still be present. - assertSuccessfulBulkDelete(FileUtil.bulkDelete(getFileSystem(), basePath, paths)); + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); } @Test @@ -150,14 +157,14 @@ public void testDeleteEmptyDirectory() throws Exception { fs.mkdirs(emptyDirPath); paths.add(emptyDirPath); // Should pass as empty directory. - assertSuccessfulBulkDelete(FileUtil.bulkDelete(getFileSystem(), basePath, paths)); + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); } @Test public void testDeleteEmptyList() throws Exception { List paths = new ArrayList<>(); // Empty list should pass. - assertSuccessfulBulkDelete(FileUtil.bulkDelete(getFileSystem(), basePath, paths)); + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); } @Test @@ -170,7 +177,7 @@ public void testDeleteSamePathsMoreThanOnce() throws Exception { Path another = new Path(basePath, "another-file"); touch(fs, another); paths.add(another); - assertSuccessfulBulkDelete(FileUtil.bulkDelete(getFileSystem(), basePath, paths)); + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); } /** @@ -187,7 +194,7 @@ public void testDeepDirectoryFilesDelete() throws Exception { Path file1 = new Path(dir3, "file1"); touch(fs, file1); paths.add(file1); - assertSuccessfulBulkDelete(FileUtil.bulkDelete(getFileSystem(), basePath, paths)); + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); } @@ -201,7 +208,7 @@ public void testChildPaths() throws Exception { touch(fs, filePath); paths.add(filePath); // Should pass as both paths are under the base path. - assertSuccessfulBulkDelete(FileUtil.bulkDelete(getFileSystem(), basePath, paths)); + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java index 2c232c231708d..d64165beea9fa 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.s3a.impl; import java.io.IOException; +import java.util.Collection; import java.util.List; import java.util.Map; @@ -70,7 +71,7 @@ public Path basePath() { } @Override - public List> bulkDelete(final List paths) + public List> bulkDelete(final Collection paths) throws IOException, IllegalArgumentException { requireNonNull(paths); checkArgument(paths.size() <= pageSize, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java index cfbcb91509d71..9f4b906e3a5a4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java @@ -56,6 +56,7 @@ import org.apache.hadoop.fs.s3a.impl.InstantiationIOException; import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; +import org.apache.hadoop.io.wrappedio.WrappedIO; import static org.apache.hadoop.fs.contract.AbstractContractBulkDeleteTest.assertSuccessfulBulkDelete; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; @@ -716,7 +717,7 @@ private void executeBulkDelete(Configuration assumedRoleConfig) throws Exception // the full FS S3AFileSystem fs = getFileSystem(); - FileUtil.bulkDelete(fs, destDir, new ArrayList<>()); + WrappedIO.bulkDelete(fs, destDir, new ArrayList<>()); bindRolePolicyStatements(assumedRoleConfig, STATEMENT_ALLOW_KMS_RW, statement(true, S3_ALL_BUCKETS, S3_ALL_OPERATIONS), From 9ad220b7d19251fddbc03eeab8fbce24030c35a0 Mon Sep 17 00:00:00 2001 From: Mukund Thakur Date: Wed, 17 Apr 2024 15:02:02 -0500 Subject: [PATCH 08/19] Refactoring and review comments and adding more tests --- .../site/markdown/filesystem/bulkdelete.md | 180 +----------------- .../src/site/markdown/filesystem/index.md | 3 +- .../AbstractContractBulkDeleteTest.java | 16 ++ .../org/apache/hadoop/fs/s3a/Constants.java | 9 + .../apache/hadoop/fs/s3a/S3AFileSystem.java | 10 +- .../fs/s3a/impl/BulkDeleteOperation.java | 5 + .../fs/s3a/impl/StoreContextFactory.java | 3 + .../markdown/tools/hadoop-aws/performance.md | 79 ++++++++ .../s3a/ITestS3AContractBulkDelete.java | 81 ++++++-- .../hadoop/fs/s3a/auth/ITestAssumeRole.java | 102 ++++++++-- 10 files changed, 285 insertions(+), 203 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md index 2f98417c061d0..bf553d30d373f 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md @@ -45,18 +45,7 @@ it supports the API. @InterfaceAudience.Public @InterfaceStability.Unstable public interface BulkDeleteSource { - - /** - * Create a bulk delete operation. - * There is no network IO at this point, simply the creation of - * a bulk delete object. - * A path must be supplied to assist in link resolution. - * @param path path to delete under. - * @return the bulk delete. - * @throws UnsupportedOperationException bulk delete under that path is not supported. - * @throws IllegalArgumentException path not valid. - * @throws IOException problems resolving paths - */ + default BulkDelete createBulkDelete(Path path) throws UnsupportedOperationException, IllegalArgumentException, IOException; @@ -69,59 +58,14 @@ public interface BulkDeleteSource { This is the bulk delete implementation returned by the `createBulkDelete()` call. ```java -/** - * API for bulk deletion of objects/files, - * but not directories. - * After use, call {@code close()} to release any resources and - * to guarantee store IOStatistics are updated. - *

- * Callers MUST have no expectation that parent directories will exist after the - * operation completes; if an object store needs to explicitly look for and create - * directory markers, that step will be omitted. - *

- * Be aware that on some stores (AWS S3) each object listed in a bulk delete counts - * against the write IOPS limit; large page sizes are counterproductive here, as - * are attempts at parallel submissions across multiple threads. - * @see HADOOP-16823. - * Large DeleteObject requests are their own Thundering Herd - *

- */ @InterfaceAudience.Public @InterfaceStability.Unstable public interface BulkDelete extends IOStatisticsSource, Closeable { - - /** - * The maximum number of objects/files to delete in a single request. - * @return a number greater than or equal to zero. - */ + int pageSize(); - - /** - * Base path of a bulk delete operation. - * All paths submitted in {@link #bulkDelete(List)} must be under this path. - */ + Path basePath(); - - /** - * Delete a list of files/objects. - *

    - *
  • Files must be under the path provided in {@link #basePath()}.
  • - *
  • The size of the list must be equal to or less than the page size - * declared in {@link #pageSize()}.
  • - *
  • Directories are not supported; the outcome of attempting to delete - * directories is undefined (ignored; undetected, listed as failures...).
  • - *
  • The operation is not atomic.
  • - *
  • The operation is treated as idempotent: network failures may - * trigger resubmission of the request -any new objects created under a - * path in the list may then be deleted.
  • - *
  • There is no guarantee that any parent directories exist after this call. - *
  • - *
- * @param paths list of paths which must be absolute and under the base path. - * provided in {@link #basePath()}. - * @throws IOException IO problems including networking, authentication and more. - * @throws IllegalArgumentException if a path argument is invalid. - */ + List> bulkDelete(List paths) throws IOException, IllegalArgumentException; @@ -161,124 +105,20 @@ store.hasPathCapability(path, "fs.capability.bulk.delete") ### Invocation through Reflection. -The need for many Libraries to compile against very old versions of Hadoop +The need for many libraries to compile against very old versions of Hadoop means that most of the cloud-first Filesystem API calls cannot be used except through reflection -And the more complicated The API and its data types are, The harder that reflection is to implement. -To assist this, the class `org.apache.hadoop.fs.FileUtil` has two methods +To assist this, the class `org.apache.hadoop.io.wrappedio.WrappedIO` has few methods which are intended to provide simple access to the API, especially through reflection. ```java - /** - * Get the maximum number of objects/files to delete in a single request. - * @param fs filesystem - * @param path path to delete under. - * @return a number greater than or equal to zero. - * @throws UnsupportedOperationException bulk delete under that path is not supported. - * @throws IllegalArgumentException path not valid. - * @throws IOException problems resolving paths - */ + public static int bulkDeletePageSize(FileSystem fs, Path path) throws IOException; - /** - * Delete a list of files/objects. - *
    - *
  • Files must be under the path provided in {@code base}.
  • - *
  • The size of the list must be equal to or less than the page size.
  • - *
  • Directories are not supported; the outcome of attempting to delete - * directories is undefined (ignored; undetected, listed as failures...).
  • - *
  • The operation is not atomic.
  • - *
  • The operation is treated as idempotent: network failures may - * trigger resubmission of the request -any new objects created under a - * path in the list may then be deleted.
  • - *
  • There is no guarantee that any parent directories exist after this call. - *
  • - *
- * @param fs filesystem - * @param base path to delete under. - * @param paths list of paths which must be absolute and under the base path. - * @return a list of all the paths which couldn't be deleted for a reason other than "not found" and any associated error message. - * @throws UnsupportedOperationException bulk delete under that path is not supported. - * @throws IOException IO problems including networking, authentication and more. - * @throws IllegalArgumentException if a path argument is invalid. - */ - public static List> bulkDelete(FileSystem fs, Path base, List paths) -``` + public static int bulkDeletePageSize(FileSystem fs, Path path) throws IOException; -## S3A Implementation - -The S3A client exports this API. - -If multi-object delete is enabled (`fs.s3a.multiobjectdelete.enable` = true), as -it is by default, then the page size is limited to that defined in -`fs.s3a.bulk.delete.page.size`, which MUST be less than or equal to1000. -* The entire list of paths to delete is aggregated into a single bulk delete request, -issued to the store. -* Provided the caller has the correct permissions, every entry in the list - will, if the path references an object, cause that object to be deleted. -* If the path does not reference an object: the path will not be deleted - "This is for deleting objects, not directories" -* No probes for the existence of parent directories will take place; no - parent directory markers will be created. - "If you need parent directories, call mkdir() yourself" -* The list of failed keys listed in the `DeleteObjectsResponse` response - are converted into paths and returned along with their error messages. -* Network and other IO errors are raised as exceptions. - -If multi-object delete is disabled (or the list of size 1) -* A single `DELETE` call is issued -* Any `AccessDeniedException` raised is converted to a result in the error list. -* Any 404 response from a (non-AWS) store will be ignored. -* Network and other IO errors are raised as exceptions. - -Because there are no probes to ensure the call does not overwrite a directory, -or to see if a parentDirectory marker needs to be created, -this API is still faster than issuing a normal `FileSystem.delete(path)` call. - -That is: all the overhead normally undertaken to preserve the Posix System model are omitted. - - -### S3 Scalability and Performance - -Every entry in a bulk delete request counts as one write operation -against AWS S3 storage. -With the default write rate under a prefix on AWS S3 Standard storage -restricted to 3,500 writes/second, it is very easy to overload -the store by issuing a few bulk delete requests simultaneously. - -* If throttling is triggered then all clients interacting with - the store may observe performance issues. -* The write quota applies even for paths which do not exist. -* The S3A client *may* perform rate throttling as well as page size limiting. - -What does that mean? it means that attempting to issue multiple -bulk delete calls in parallel can be counterproductive. - -When overloaded, the S3 store returns a 403 throttle response. -This will trigger it back off and retry of posting the request. -However, the repeated request will still include the same number of objects and -*so generate the same load*. - -This can lead to a pathological situation where the repeated requests will -never be satisfied because the request itself is sufficient to overload the store. -See [HADOOP-16823.Large DeleteObject requests are their own Thundering Herd](https://issues.apache.org/jira/browse/HADOOP-16823) -for an example of where this did actually surface in production. - -This is why the default page size of S3A clients is 250 paths, not the store limit of 1000 entries. -It is also why the S3A delete/rename Operations do not attempt to do massive parallel deletions, -Instead bulk delete requests are queued for a single blocking thread to issue. -Consider a similar design. - - -When working with versioned S3 buckets, every path deleted will add a tombstone marker -to the store at that location, even if there was no object at that path. -While this has no negative performance impact on the bulk delete call, -it will slow down list requests subsequently made against that path. -That is: bulk delete requests of paths which do not exist will hurt future queries. - -Avoid this. Note also that TPC-DS Benchmark do not create the right load to make the -performance problems observable -but they can surface in production. -* Configure buckets to have a limited number of days for tombstones to be preserved. -* Do not delete which you know do not contain objects. + public static List> bulkDelete(FileSystem fs, Path base, Collection paths); +``` diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md index df39839e831c8..be72f35789aad 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md @@ -43,4 +43,5 @@ HDFS as these are commonly expected by Hadoop client applications. 1. [IOStatistics](iostatistics.html) 1. [openFile()](openfile.html) 1. [SafeMode](safemode.html) -1. [LeaseRecoverable](leaserecoverable.html) \ No newline at end of file +1. [LeaseRecoverable](leaserecoverable.html) +1. [BulkDelete](bulkdelete.html) \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java index 09a219efe1ad5..c4502ac5deedf 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java @@ -34,10 +34,14 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.wrappedio.WrappedIO; +import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.io.wrappedio.WrappedIO.bulkDelete; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +/** + * Contract tests for bulk delete operation. + */ public abstract class AbstractContractBulkDeleteTest extends AbstractFSContractTestBase { private static final Logger LOG = @@ -146,6 +150,7 @@ public void testDeletePathsDirectory() throws Exception { Path filePath = new Path(dirPath, "file"); touch(fs, filePath); paths.add(filePath); + pageSizePreconditionForTest(paths.size()); // Outcome is undefined. But call shouldn't fail. In case of S3 directories will still be present. assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); } @@ -177,9 +182,19 @@ public void testDeleteSamePathsMoreThanOnce() throws Exception { Path another = new Path(basePath, "another-file"); touch(fs, another); paths.add(another); + pageSizePreconditionForTest(paths.size()); assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); } + /** + * Skip test if paths size is greater than page size. + */ + protected void pageSizePreconditionForTest(int size) { + if (size > pageSize) { + skip("Test requires paths size less than or equal to page size"); + } + } + /** * This test validates that files to be deleted don't have * to be direct children of the base path. @@ -207,6 +222,7 @@ public void testChildPaths() throws Exception { Path filePath = new Path(dirPath, "file"); touch(fs, filePath); paths.add(filePath); + pageSizePreconditionForTest(paths.size()); // Should pass as both paths are under the base path. assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index 55607be78f534..185389739cbad 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -1641,7 +1641,16 @@ private Constants() { */ public static final String AWS_S3_ACCESS_GRANTS_FALLBACK_TO_IAM_ENABLED = "fs.s3a.access.grants.fallback.to.iam"; + /** + * Default value for {@link #S3A_IO_RATE_LIMIT}. + * Value: {@value}. + * 0 means no rate limiting. + */ public static final int DEFAULT_S3A_IO_RATE_LIMIT = 0; + /** + * Config to set the rate limit for S3A IO operations. + * Value: {@value}. + */ public static final String S3A_IO_RATE_LIMIT = "fs.s3a.io.rate.limit"; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 76cab15e3986f..7830cd36c49a5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -52,7 +52,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import javax.annotation.Nullable; -import org.apache.hadoop.util.*; import software.amazon.awssdk.core.ResponseInputStream; import software.amazon.awssdk.core.exception.SdkException; import software.amazon.awssdk.services.s3.S3AsyncClient; @@ -206,6 +205,15 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.ProviderUtils; import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.util.BlockingThreadPoolExecutorService; +import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.LambdaUtils; +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.util.Preconditions; +import org.apache.hadoop.util.Progressable; +import org.apache.hadoop.util.RateLimitingFactory; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hadoop.util.SemaphoredDelegatingExecutor; import org.apache.hadoop.util.concurrent.HadoopExecutors; import org.apache.hadoop.util.functional.CallableRaisingIOE; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java index d64165beea9fa..6e6ec5957887e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java @@ -100,6 +100,11 @@ public List> bulkDelete(final Collection paths) return emptyList(); } + /** + * Validate that the path is under the base path. + * @param p path to validate. + * @return true if the path is under the base path else false. + */ private boolean validatePathIsUnderParent(Path p) { while (p.getParent() != null) { if (p.getParent().equals(basePath)) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextFactory.java index 355288619d30a..9d8d708b2bcc7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/StoreContextFactory.java @@ -20,6 +20,9 @@ import org.apache.hadoop.classification.InterfaceAudience; +/** + * Factory for creating store contexts. + */ @InterfaceAudience.Private public interface StoreContextFactory { diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md index 4bb824356e9d4..2b11c7e5d6817 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md @@ -94,6 +94,85 @@ on the client requirements. ``` +## Improving delete performance through bulkdelete API. + +For bulk delete API spec refer to [BulkDelete](bulkdelete.html). + +The S3A client exports this API. + +### S3A Implementation of Bulk Delete. +If multi-object delete is enabled (`fs.s3a.multiobjectdelete.enable` = true), as +it is by default, then the page size is limited to that defined in +`fs.s3a.bulk.delete.page.size`, which MUST be less than or equal to 1000. +* The entire list of paths to delete is aggregated into a single bulk delete request, + issued to the store. +* Provided the caller has the correct permissions, every entry in the list + will, if the path references an object, cause that object to be deleted. +* If the path does not reference an object: the path will not be deleted + "This is for deleting objects, not directories" +* No probes for the existence of parent directories will take place; no + parent directory markers will be created. + "If you need parent directories, call mkdir() yourself" +* The list of failed keys listed in the `DeleteObjectsResponse` response + are converted into paths and returned along with their error messages. +* Network and other IO errors are raised as exceptions. + +If multi-object delete is disabled (or the list of size 1) +* A single `DELETE` call is issued +* Any `AccessDeniedException` raised is converted to a result in the error list. +* Any 404 response from a (non-AWS) store will be ignored. +* Network and other IO errors are raised as exceptions. + +Because there are no probes to ensure the call does not overwrite a directory, +or to see if a parentDirectory marker needs to be created, +this API is still faster than issuing a normal `FileSystem.delete(path)` call. + +That is: all the overhead normally undertaken to preserve the Posix System model are omitted. + + +### S3 Scalability and Performance + +Every entry in a bulk delete request counts as one write operation +against AWS S3 storage. +With the default write rate under a prefix on AWS S3 Standard storage +restricted to 3,500 writes/second, it is very easy to overload +the store by issuing a few bulk delete requests simultaneously. + +* If throttling is triggered then all clients interacting with + the store may observe performance issues. +* The write quota applies even for paths which do not exist. +* The S3A client *may* perform rate throttling as well as page size limiting. + +What does that mean? it means that attempting to issue multiple +bulk delete calls in parallel can be counterproductive. + +When overloaded, the S3 store returns a 403 throttle response. +This will trigger it back off and retry of posting the request. +However, the repeated request will still include the same number of objects and +*so generate the same load*. + +This can lead to a pathological situation where the repeated requests will +never be satisfied because the request itself is sufficient to overload the store. +See [HADOOP-16823.Large DeleteObject requests are their own Thundering Herd](https://issues.apache.org/jira/browse/HADOOP-16823) +for an example of where this did actually surface in production. + +This is why the default page size of S3A clients is 250 paths, not the store limit of 1000 entries. +It is also why the S3A delete/rename Operations do not attempt to do massive parallel deletions, +Instead bulk delete requests are queued for a single blocking thread to issue. +Consider a similar design. + + +When working with versioned S3 buckets, every path deleted will add a tombstone marker +to the store at that location, even if there was no object at that path. +While this has no negative performance impact on the bulk delete call, +it will slow down list requests subsequently made against that path. +That is: bulk delete requests of paths which do not exist will hurt future queries. + +Avoid this. Note also that TPC-DS Benchmark do not create the right load to make the +performance problems observable -but they can surface in production. +* Configure buckets to have a limited number of days for tombstones to be preserved. +* Do not delete which you know do not contain objects. + ## Improving data input performance through fadvise The S3A Filesystem client supports the notion of input policies, similar diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java index 6e3a575f625fc..24025b5180fc3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java @@ -18,6 +18,17 @@ package org.apache.hadoop.fs.contract.s3a; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BulkDelete; import org.apache.hadoop.fs.FileStatus; @@ -27,19 +38,23 @@ import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; -import org.assertj.core.api.Assertions; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Arrays; -import java.util.List; +import org.apache.hadoop.fs.statistics.MeanStatistic; +import org.apache.hadoop.io.wrappedio.WrappedIO; import static java.util.stream.Collectors.toList; +import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; +import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.S3ATestUtils.createFiles; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupMeanStatistic; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.STORE_IO_RATE_LIMITED_DURATION; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MEAN; +import static org.apache.hadoop.io.wrappedio.WrappedIO.bulkDelete; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +/** + * Contract tests for bulk delete operation for S3A Implementation. + */ +@RunWith(Parameterized.class) public class ITestS3AContractBulkDelete extends AbstractContractBulkDeleteTest { private static final Logger LOG = LoggerFactory.getLogger(ITestS3AContractBulkDelete.class); @@ -52,6 +67,20 @@ public class ITestS3AContractBulkDelete extends AbstractContractBulkDeleteTest { */ private static final int DELETE_PAGE_SIZE = 20; + private final boolean enableMultiObjectDelete; + + @Parameterized.Parameters(name = "enableMultiObjectDelete = {0}") + public static Iterable enableMultiObjectDelete() { + return Arrays.asList(new Object[][] { + {true}, + {false} + }); + } + + public ITestS3AContractBulkDelete(boolean enableMultiObjectDelete) { + this.enableMultiObjectDelete = enableMultiObjectDelete; + } + @Override protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); @@ -59,6 +88,7 @@ protected Configuration createConfiguration() { S3ATestUtils.removeBaseAndBucketOverrides(conf, Constants.BULK_DELETE_PAGE_SIZE); conf.setInt(Constants.BULK_DELETE_PAGE_SIZE, DELETE_PAGE_SIZE); + conf.setBoolean(Constants.ENABLE_MULTI_DELETE, enableMultiObjectDelete); return conf; } @@ -69,13 +99,23 @@ protected AbstractFSContract createContract(Configuration conf) { @Override public void validatePageSize() throws Exception { + int targetPageSize = DELETE_PAGE_SIZE; + if (!enableMultiObjectDelete) { + // if multi-object delete is disabled, page size should be 1. + targetPageSize = 1; + } Assertions.assertThat(pageSize) .describedAs("Page size should match the configured page size") - .isEqualTo(DELETE_PAGE_SIZE); + .isEqualTo(targetPageSize); } @Test public void testBulkDeleteZeroPageSizePrecondition() throws Exception { + if(!enableMultiObjectDelete) { + // if multi-object delete is disabled, skip this test as + // page size is always 1. + skip("Multi-object delete is disabled"); + } Configuration conf = getContract().getConf(); conf.setInt(Constants.BULK_DELETE_PAGE_SIZE, 0); Path testPath = path(getMethodName()); @@ -98,6 +138,21 @@ public void testPageSizeWhenMultiObjectsDisabled() throws Exception { } } + @Override + public void testDeletePathsDirectory() throws Exception { + List paths = new ArrayList<>(); + Path dirPath = new Path(basePath, "dir"); + fs.mkdirs(dirPath); + paths.add(dirPath); + Path filePath = new Path(dirPath, "file"); + touch(fs, filePath); + paths.add(filePath); + pageSizePreconditionForTest(paths.size()); + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + // During the bulk delete operation, the directories are not deleted in S3A. + assertIsDirectory(dirPath); + } + @Test public void testRateLimiting() throws Exception { Configuration conf = getContract().getConf(); @@ -109,16 +164,20 @@ public void testRateLimiting() throws Exception { List paths = Arrays.stream(fileStatuses) .map(FileStatus::getPath) .collect(toList()); + pageSizePreconditionForTest(paths.size()); BulkDelete bulkDelete = fs.createBulkDelete(basePath); bulkDelete.bulkDelete(paths); - String mean = STORE_IO_RATE_LIMITED_DURATION + ".mean"; - Assertions.assertThat(fs.getIOStatistics().meanStatistics().get(mean).mean()) + MeanStatistic meanStatisticBefore = lookupMeanStatistic(fs.getIOStatistics(), + STORE_IO_RATE_LIMITED_DURATION + SUFFIX_MEAN); + Assertions.assertThat(meanStatisticBefore.mean()) .describedAs("Rate limiting should not have happened during first delete call") .isEqualTo(0.0); bulkDelete.bulkDelete(paths); bulkDelete.bulkDelete(paths); bulkDelete.bulkDelete(paths); - Assertions.assertThat(fs.getIOStatistics().meanStatistics().get(mean).mean()) + MeanStatistic meanStatisticAfter = lookupMeanStatistic(fs.getIOStatistics(), + STORE_IO_RATE_LIMITED_DURATION + SUFFIX_MEAN); + Assertions.assertThat(meanStatisticAfter.mean()) .describedAs("Rate limiting should have happened during multiple delete calls") .isGreaterThan(0.0); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java index 9f4b906e3a5a4..8d6a3065854aa 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java @@ -30,7 +30,6 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -import org.apache.hadoop.fs.*; import software.amazon.awssdk.auth.credentials.AwsCredentials; import software.amazon.awssdk.services.s3.model.MultipartUpload; import software.amazon.awssdk.services.sts.model.StsException; @@ -40,9 +39,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.BulkDelete; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.AWSBadRequestException; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; @@ -706,12 +710,21 @@ public void testPartialDeleteSingleDelete() throws Throwable { } @Test - public void testBulkDelete() throws Throwable { + public void testBulkDeleteOnReadOnlyAccess() throws Throwable { describe("Bulk delete with part of the child tree read only"); - executeBulkDelete(createAssumedRoleConfig()); + executeBulkDeleteOnReadOnlyFiles(createAssumedRoleConfig()); + } + + @Test + public void testBulkDeleteWithReadWriteAccess() throws Throwable { + describe("Bulk delete with read write access"); + executeBulkDeleteOnSomeReadOnlyFiles(createAssumedRoleConfig()); } - private void executeBulkDelete(Configuration assumedRoleConfig) throws Exception { + /** + * Execute bulk delete on read only files and some read write files. + */ + private void executeBulkDeleteOnReadOnlyFiles(Configuration assumedRoleConfig) throws Exception { Path destDir = methodPath(); Path readOnlyDir = new Path(destDir, "readonlyDir"); @@ -719,12 +732,7 @@ private void executeBulkDelete(Configuration assumedRoleConfig) throws Exception S3AFileSystem fs = getFileSystem(); WrappedIO.bulkDelete(fs, destDir, new ArrayList<>()); - bindRolePolicyStatements(assumedRoleConfig, STATEMENT_ALLOW_KMS_RW, - statement(true, S3_ALL_BUCKETS, S3_ALL_OPERATIONS), - new Statement(Effects.Deny) - .addActions(S3_PATH_WRITE_OPERATIONS) - .addResources(directory(readOnlyDir)) - ); + bindReadOnlyRolePolicy(assumedRoleConfig, readOnlyDir); roleFS = (S3AFileSystem) destDir.getFileSystem(assumedRoleConfig); int range = 10; @@ -742,14 +750,73 @@ private void executeBulkDelete(Configuration assumedRoleConfig) throws Exception // delete the files in the original FS should succeed. BulkDelete bulkDelete3 = fs.createBulkDelete(readOnlyDir); assertSuccessfulBulkDelete(bulkDelete3.bulkDelete(pathsToDelete)); + FileStatus[] fileStatusesUnderDestDir = roleFS.listStatus(destDir); + List pathsToDeleteUnderDestDir = Arrays.stream(fileStatusesUnderDestDir) + .map(FileStatus::getPath) + .collect(Collectors.toList()); BulkDelete bulkDelete4 = fs.createBulkDelete(destDir); - assertSuccessfulBulkDelete(bulkDelete4.bulkDelete(pathsToDelete)); - // we can write a test for some successful and some failure as well. + assertSuccessfulBulkDelete(bulkDelete4.bulkDelete(pathsToDeleteUnderDestDir)); } + /** + * Execute bulk delete on some read only files and some read write files. + */ + private void executeBulkDeleteOnSomeReadOnlyFiles(Configuration assumedRoleConfig) + throws IOException { + Path destDir = methodPath(); + Path readOnlyDir = new Path(destDir, "readonlyDir"); + bindReadOnlyRolePolicy(assumedRoleConfig, readOnlyDir); + roleFS = (S3AFileSystem) destDir.getFileSystem(assumedRoleConfig); + S3AFileSystem fs = getFileSystem(); + WrappedIO.bulkDelete(fs, destDir, new ArrayList<>()); + // creating 5 files in the read only dir. + int readOnlyRange = 5; + int readWriteRange = 3; + touchFiles(fs, readOnlyDir, readOnlyRange); + // creating 3 files in the base destination dir. + touchFiles(roleFS, destDir, readWriteRange); + RemoteIterator locatedFileStatusRemoteIterator = roleFS.listFiles(destDir, true); + List pathsToDelete2 = new ArrayList<>(); + while (locatedFileStatusRemoteIterator.hasNext()) { + pathsToDelete2.add(locatedFileStatusRemoteIterator.next().getPath()); + } + Assertions.assertThat(pathsToDelete2.size()) + .describedAs("Number of paths to delete in base destination dir") + .isEqualTo(readOnlyRange + readWriteRange); + BulkDelete bulkDelete5 = roleFS.createBulkDelete(destDir); + List> entries = bulkDelete5.bulkDelete(pathsToDelete2); + Assertions.assertThat(entries.size()) + .describedAs("Number of error entries in bulk delete result") + .isEqualTo(readOnlyRange); + assertAccessDeniedForEachPath(entries); + // delete the files in the original FS should succeed. + BulkDelete bulkDelete6 = fs.createBulkDelete(destDir); + assertSuccessfulBulkDelete(bulkDelete6.bulkDelete(pathsToDelete2)); + } + + /** + * Bind a read only role policy to a directory to the FS conf. + */ + private static void bindReadOnlyRolePolicy(Configuration assumedRoleConfig, + Path readOnlyDir) + throws JsonProcessingException { + bindRolePolicyStatements(assumedRoleConfig, STATEMENT_ALLOW_KMS_RW, + statement(true, S3_ALL_BUCKETS, S3_ALL_OPERATIONS), + new Statement(Effects.Deny) + .addActions(S3_PATH_WRITE_OPERATIONS) + .addResources(directory(readOnlyDir)) + ); + } + + /** + * Validate delete results for each path in the list + * has access denied error. + */ private void assertAccessDeniedForEachPath(List> entries) { for (Map.Entry entry : entries) { - Assertions.assertThat(entry.getValue()).contains("AccessDenied"); + Assertions.assertThat(entry.getValue()) + .describedAs("Error message for path %s is %s", entry.getKey(), entry.getValue()) + .contains("AccessDenied"); } } @@ -770,12 +837,7 @@ public void executePartialDelete(final Configuration conf, S3AFileSystem fs = getFileSystem(); fs.delete(destDir, true); - bindRolePolicyStatements(conf, STATEMENT_ALLOW_KMS_RW, - statement(true, S3_ALL_BUCKETS, S3_ALL_OPERATIONS), - new Statement(Effects.Deny) - .addActions(S3_PATH_WRITE_OPERATIONS) - .addResources(directory(readOnlyDir)) - ); + bindReadOnlyRolePolicy(conf, readOnlyDir); roleFS = (S3AFileSystem) destDir.getFileSystem(conf); int range = 10; From 741542703607b954851f005514b12af61a98afb6 Mon Sep 17 00:00:00 2001 From: Mukund Thakur Date: Thu, 18 Apr 2024 16:41:45 -0500 Subject: [PATCH 09/19] Adding default impl for bulk delete for other file systems --- .../org/apache/hadoop/fs/BulkDeleteUtils.java | 54 ++++++++++++ .../hadoop/fs/DefalutBulkDeleteSource.java | 38 +++++++++ .../hadoop/fs/DefaultBulkDeleteOperation.java | 84 +++++++++++++++++++ .../apache/hadoop/io/wrappedio/WrappedIO.java | 8 +- .../AbstractContractBulkDeleteTest.java | 10 +++ .../fs/s3a/impl/BulkDeleteOperation.java | 18 +--- .../contract/ITestAbfsContractBulkDelete.java | 64 +++++++------- 7 files changed, 225 insertions(+), 51 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteUtils.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefalutBulkDeleteSource.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteUtils.java new file mode 100644 index 0000000000000..fa7af088608a4 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteUtils.java @@ -0,0 +1,54 @@ +/** + * 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.hadoop.fs; + +import java.util.Collection; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.util.Preconditions.checkArgument; + +/** + * Utility class for bulk delete operations. + */ +public final class BulkDeleteUtils { + + private BulkDeleteUtils() { + } + + public static void validateBulkDeletePaths(Collection paths, int pageSize, Path basePath) { + requireNonNull(paths); + checkArgument(paths.size() <= pageSize, + "Number of paths (%d) is larger than the page size (%d)", paths.size(), pageSize); + paths.forEach(p -> { + checkArgument(p.isAbsolute(), "Path %s is not absolute", p); + checkArgument(validatePathIsUnderParent(p, basePath), + "Path %s is not under the base path %s", p, basePath); + }); + } + + public static boolean validatePathIsUnderParent(Path p, Path basePath) { + while (p.getParent() != null) { + if (p.getParent().equals(basePath)) { + return true; + } + p = p.getParent(); + } + return false; + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefalutBulkDeleteSource.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefalutBulkDeleteSource.java new file mode 100644 index 0000000000000..d41e723a8c56c --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefalutBulkDeleteSource.java @@ -0,0 +1,38 @@ +/** + * 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.hadoop.fs; + +import java.io.IOException; + +/** + * Default implementation of {@link BulkDeleteSource}. + */ +public class DefalutBulkDeleteSource implements BulkDeleteSource { + + private final FileSystem fs; + + public DefalutBulkDeleteSource(FileSystem fs) { + this.fs = fs; + } + + @Override + public BulkDelete createBulkDelete(Path base) + throws UnsupportedOperationException, IllegalArgumentException, IOException { + return new DefaultBulkDeleteOperation(1, base, fs); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java new file mode 100644 index 0000000000000..ae7cbff4321f0 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java @@ -0,0 +1,84 @@ +/** + * 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.hadoop.fs; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.util.functional.Tuples; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.BulkDeleteUtils.validateBulkDeletePaths; +import static org.apache.hadoop.util.Preconditions.checkArgument; + +/** + * Default implementation of the {@link BulkDelete} interface. + */ +public class DefaultBulkDeleteOperation implements BulkDelete { + + private final int pageSize; + + private final Path basePath; + + private final FileSystem fs; + + public DefaultBulkDeleteOperation(int pageSize, + Path basePath, + FileSystem fs) { + checkArgument(pageSize == 1, "Page size must be equal to 1"); + this.pageSize = pageSize; + this.basePath = requireNonNull(basePath); + this.fs = fs; + } + + @Override + public int pageSize() { + return pageSize; + } + + @Override + public Path basePath() { + return basePath; + } + + @Override + public List> bulkDelete(Collection paths) + throws IOException, IllegalArgumentException { + validateBulkDeletePaths(paths, pageSize, basePath); + List> result = new ArrayList<>(); + // this for loop doesn't make sense as pageSize must be 1. + for (Path path : paths) { + try { + fs.delete(path, false); + // What to do if this return false? + // I think we should add the path to the result list with value "Not Deleted". + } catch (IOException e) { + result.add(Tuples.pair(path, e.toString())); + } + } + return result; + } + + @Override + public void close() throws IOException { + + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java index af5249195ca8d..f5c455f53798e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java @@ -27,6 +27,8 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.BulkDelete; import org.apache.hadoop.fs.BulkDeleteSource; +import org.apache.hadoop.fs.DefalutBulkDeleteSource; +import org.apache.hadoop.fs.DefaultBulkDeleteOperation; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -79,10 +81,10 @@ public static int bulkDeletePageSize(FileSystem fs, Path path) throws IOExceptio * @throws UnsupportedOperationException FS doesn't implement the interface. */ private static BulkDeleteSource toBulkDeleteSource(final FileSystem fs) { - if (!(fs instanceof BulkDeleteSource)) { - throw new UnsupportedOperationException("Bulk delete not supported"); + if (fs instanceof BulkDeleteSource) { + return (BulkDeleteSource) fs; } - return (BulkDeleteSource) fs; + return new DefalutBulkDeleteSource(fs); } /** diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java index c4502ac5deedf..cc883f8e564e4 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java @@ -132,6 +132,16 @@ public void testDeletePathsNotUnderBase() throws Exception { () -> bulkDelete(getFileSystem(), basePath, paths)); } + @Test + public void testDeletePathsNotAbsolute() throws Exception { + List paths = new ArrayList<>(); + Path pathNotAbsolute = new Path("not-absolute"); + paths.add(pathNotAbsolute); + // Should fail as path is not absolute. + intercept(IllegalArgumentException.class, + () -> bulkDelete(getFileSystem(), basePath, paths)); + } + @Test public void testDeletePathsNotExists() throws Exception { List paths = new ArrayList<>(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java index 6e6ec5957887e..d0e2e63e7c30f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java @@ -34,6 +34,7 @@ import static java.util.Collections.emptyList; import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.toList; +import static org.apache.hadoop.fs.BulkDeleteUtils.validatePathIsUnderParent; import static org.apache.hadoop.util.Preconditions.checkArgument; /** @@ -79,7 +80,7 @@ public List> bulkDelete(final Collection paths) final StoreContext context = getStoreContext(); final List objects = paths.stream().map(p -> { checkArgument(p.isAbsolute(), "Path %s is not absolute", p); - checkArgument(validatePathIsUnderParent(p), + checkArgument(validatePathIsUnderParent(p, basePath), "Path %s is not under the base path %s", p, basePath); final String k = context.pathToKey(p); return ObjectIdentifier.builder().key(k).build(); @@ -100,21 +101,6 @@ public List> bulkDelete(final Collection paths) return emptyList(); } - /** - * Validate that the path is under the base path. - * @param p path to validate. - * @return true if the path is under the base path else false. - */ - private boolean validatePathIsUnderParent(Path p) { - while (p.getParent() != null) { - if (p.getParent().equals(basePath)) { - return true; - } - p = p.getParent(); - } - return false; - } - @Override public void close() throws IOException { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsContractBulkDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsContractBulkDelete.java index 85038755d2958..ca115983d33d9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsContractBulkDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsContractBulkDelete.java @@ -1,32 +1,32 @@ -//package org.apache.hadoop.fs.azurebfs.contract; -// -//import org.apache.hadoop.conf.Configuration; -//import org.apache.hadoop.fs.contract.AbstractContractBulkDeleteTest; -//import org.apache.hadoop.fs.contract.AbstractFSContract; -// -//public class ITestAbfsContractBulkDelete extends AbstractContractBulkDeleteTest { -// -// private final boolean isSecure; -// private final ABFSContractTestBinding binding; -// -// public ITestAbfsContractBulkDelete() throws Exception { -// binding = new ABFSContractTestBinding(); -// this.isSecure = binding.isSecureMode(); -// } -// -// @Override -// public void setup() throws Exception { -// binding.setup(); -// super.setup(); -// } -// -// @Override -// protected Configuration createConfiguration() { -// return binding.getRawConfiguration(); -// } -// -// @Override -// protected AbstractFSContract createContract(Configuration conf) { -// return new AbfsFileSystemContract(conf, isSecure); -// } -//} +package org.apache.hadoop.fs.azurebfs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractBulkDeleteTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +public class ITestAbfsContractBulkDelete extends AbstractContractBulkDeleteTest { + + private final boolean isSecure; + private final ABFSContractTestBinding binding; + + public ITestAbfsContractBulkDelete() throws Exception { + binding = new ABFSContractTestBinding(); + this.isSecure = binding.isSecureMode(); + } + + @Override + public void setup() throws Exception { + binding.setup(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return binding.getRawConfiguration(); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new AbfsFileSystemContract(conf, isSecure); + } +} From 27b16f94d724e0630f8522e3b8c1ab317f933a5a Mon Sep 17 00:00:00 2001 From: Mukund Thakur Date: Tue, 23 Apr 2024 16:41:15 -0500 Subject: [PATCH 10/19] review comments --- .../java/org/apache/hadoop/fs/BulkDelete.java | 2 + .../hadoop/fs/DefalutBulkDeleteSource.java | 2 +- .../hadoop/fs/DefaultBulkDeleteOperation.java | 51 ++++++++++++----- .../site/markdown/filesystem/bulkdelete.md | 13 +++++ .../AbstractContractBulkDeleteTest.java | 18 +++++- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 5 ++ .../org/apache/hadoop/fs/s3a/S3AStore.java | 8 +++ .../fs/s3a/impl/BulkDeleteOperation.java | 3 + .../hadoop/fs/s3a/impl/S3AStoreImpl.java | 57 +++++++------------ .../markdown/tools/hadoop-aws/performance.md | 6 +- .../s3a/ITestS3AContractBulkDelete.java | 43 +++++++++++--- .../hadoop/fs/s3a/auth/ITestAssumeRole.java | 10 +++- .../contract/ITestAbfsContractBulkDelete.java | 18 ++++++ 13 files changed, 170 insertions(+), 66 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java index c768ef93062ba..a6d473c7c04e0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java @@ -80,9 +80,11 @@ public interface BulkDelete extends IOStatisticsSource, Closeable { * * @param paths list of paths which must be absolute and under the base path. * provided in {@link #basePath()}. + * @return a list of paths which failed to delete, with the exception message. * @throws IOException IO problems including networking, authentication and more. * @throws IllegalArgumentException if a path argument is invalid. */ + List> bulkDelete(Collection paths) throws IOException, IllegalArgumentException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefalutBulkDeleteSource.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefalutBulkDeleteSource.java index d41e723a8c56c..5b769b53494f6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefalutBulkDeleteSource.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefalutBulkDeleteSource.java @@ -33,6 +33,6 @@ public DefalutBulkDeleteSource(FileSystem fs) { @Override public BulkDelete createBulkDelete(Path base) throws UnsupportedOperationException, IllegalArgumentException, IOException { - return new DefaultBulkDeleteOperation(1, base, fs); + return new DefaultBulkDeleteOperation(base, fs); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java index ae7cbff4321f0..47895a1efad33 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java @@ -17,12 +17,16 @@ */ package org.apache.hadoop.fs; +import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.hadoop.util.functional.Tuples; import static java.util.Objects.requireNonNull; @@ -34,24 +38,23 @@ */ public class DefaultBulkDeleteOperation implements BulkDelete { - private final int pageSize; + private static Logger LOG = LoggerFactory.getLogger(DefaultBulkDeleteOperation.class); + + private static final int DEFAULT_PAGE_SIZE = 1; private final Path basePath; private final FileSystem fs; - public DefaultBulkDeleteOperation(int pageSize, - Path basePath, + public DefaultBulkDeleteOperation(Path basePath, FileSystem fs) { - checkArgument(pageSize == 1, "Page size must be equal to 1"); - this.pageSize = pageSize; this.basePath = requireNonNull(basePath); this.fs = fs; } @Override public int pageSize() { - return pageSize; + return DEFAULT_PAGE_SIZE; } @Override @@ -59,19 +62,39 @@ public Path basePath() { return basePath; } + /** + * {@inheritDoc} + */ @Override public List> bulkDelete(Collection paths) throws IOException, IllegalArgumentException { - validateBulkDeletePaths(paths, pageSize, basePath); + validateBulkDeletePaths(paths, DEFAULT_PAGE_SIZE, basePath); List> result = new ArrayList<>(); - // this for loop doesn't make sense as pageSize must be 1. - for (Path path : paths) { + if (!paths.isEmpty()) { + // As the page size is always 1, this should be the only one + // path in the collection. + Path pathToDelete = paths.iterator().next(); try { - fs.delete(path, false); - // What to do if this return false? - // I think we should add the path to the result list with value "Not Deleted". - } catch (IOException e) { - result.add(Tuples.pair(path, e.toString())); + boolean deleted = fs.delete(pathToDelete, false); + if (deleted) { + return result; + } else { + try { + FileStatus fileStatus = fs.getFileStatus(pathToDelete); + if (fileStatus.isDirectory()) { + result.add(Tuples.pair(pathToDelete, "Path is a directory")); + } + } catch (FileNotFoundException e) { + // Ignore FNFE and don't add to the result list. + LOG.debug("Couldn't delete {} - does not exist: {}", pathToDelete, e.toString()); + } catch (Exception e) { + LOG.debug("Couldn't delete {} - exception occurred: {}", pathToDelete, e.toString()); + result.add(Tuples.pair(pathToDelete, e.toString())); + } + } + } catch (Exception ex) { + LOG.debug("Couldn't delete {} - exception occurred: {}", pathToDelete, ex.toString()); + result.add(Tuples.pair(pathToDelete, ex.toString())); } } return result; diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md index bf553d30d373f..8216f74799003 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md @@ -122,3 +122,16 @@ through reflection. public static List> bulkDelete(FileSystem fs, Path base, Collection paths); ``` + +### Implementations + +#### Default Implementation + +The default implementation of the `BulkDelete` interface is `org.apache.hadoop.fs.DefaultBulkDeleteOperation` +which fixes the page size to be 1 and calls `FileSystem.delete(path, false)` on the single path in the list. + + +#### S3A Implementation +The S3A implementation is `org.apache.hadoop.fs.s3a.impl.BulkDeleteOperation` which implements the +multi object delete semantics of the AWS S3 API [Bulk Delete](https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html) +For more details please refer to the S3A Performance documentation. \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java index cc883f8e564e4..e912dbec03530 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java @@ -65,6 +65,10 @@ public Path getBasePath() { return basePath; } + protected int getExpectedPageSize() { + return 1; + } + /** * Validate the page size for bulk delete operation. Different stores can have different * implementations for bulk delete operation thus different page size. @@ -161,7 +165,19 @@ public void testDeletePathsDirectory() throws Exception { touch(fs, filePath); paths.add(filePath); pageSizePreconditionForTest(paths.size()); - // Outcome is undefined. But call shouldn't fail. In case of S3 directories will still be present. + // Outcome is undefined. But call shouldn't fail. + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + } + + @Test + public void testBulkDeleteParentDirectory() throws Exception { + List paths = new ArrayList<>(); + Path dirPath = new Path(basePath, "dir"); + fs.mkdirs(dirPath); + Path subDir = new Path(dirPath, "subdir"); + fs.mkdirs(subDir); + // adding parent directory to the list of paths. + paths.add(dirPath); assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 7830cd36c49a5..19ae0c4c86c4b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -5739,6 +5739,11 @@ public boolean isMultipartUploadEnabled() { return isMultipartUploadEnabled; } + /** + * S3A implementation to create a bulk delete operation using + * which actual bulk delete calls can be made. + * @return an implementation of the bulk delete. + */ @Override public BulkDelete createBulkDelete(final Path path) throws IllegalArgumentException, IOException { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java index da4f52a8f11a0..68abe40bd63bb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java @@ -87,6 +87,10 @@ public interface S3AStore extends IOStatisticsSource { * the store when updating its load counters on a specific partition * of an S3 bucket. * If only the request was measured, this operation would under-report. + * A write capacity will be requested proportional to the number of keys + * preset in the request and will be re-requested during retries such that + * retries throttle better. If the request is throttled, the time spent is + * recorded in a duration IOStat named {@code STORE_IO_RATE_LIMITED_DURATION}. * @param deleteRequest keys to delete on the s3-backend * @return the AWS response * @throws MultiObjectDeleteException one or more of the keys could not @@ -105,6 +109,10 @@ Map.Entry deleteObjects(DeleteObjectsRequest de * 404 errors other than bucket not found are swallowed; * this can be raised by third party stores (GCS). *

+ * A write capacity of 1 ( as it is signle object delete) will be requested before + * the delete call and will be re-requested during retries such that + * retries throttle better. If the request is throttled, the time spent is + * recorded in a duration IOStat named {@code STORE_IO_RATE_LIMITED_DURATION}. * If an exception is caught and swallowed, the response will be empty; * otherwise it will be the response from the delete operation. * @param request request to make diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java index d0e2e63e7c30f..64bebd880cd6c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/BulkDeleteOperation.java @@ -71,6 +71,9 @@ public Path basePath() { return basePath; } + /** + * {@inheritDoc} + */ @Override public List> bulkDelete(final Collection paths) throws IOException, IllegalArgumentException { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java index c808ddcb5719c..d7b7f4115063b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java @@ -73,17 +73,22 @@ public class S3AStoreImpl implements S3AStore { private static final Logger LOG = LoggerFactory.getLogger(S3AStoreImpl.class); + /** Factory to create store contexts. */ private final StoreContextFactory storeContextFactory; + /** The S3 client used to communicate with S3 bucket. */ private final S3Client s3Client; + /** The S3 bucket to communicate with. */ private final String bucket; + /** Request factory for creating requests. */ private final RequestFactory requestFactory; /** Async client is used for transfer manager. */ private S3AsyncClient s3AsyncClient; + /** Duration tracker factory. */ private final DurationTrackerFactory durationTrackerFactory; /** The core instrumentation. */ @@ -95,16 +100,22 @@ public class S3AStoreImpl implements S3AStore { /** Storage Statistics Bonded to the instrumentation. */ private final S3AStorageStatistics storageStatistics; + /** Rate limiter for read operations. */ private final RateLimiting readRateLimiter; + /** Rate limiter for write operations. */ private final RateLimiting writeRateLimiter; + /** Store context. */ private final StoreContext storeContext; + /** Invoker for retry operations. */ private final Invoker invoker; + /** Audit span source. */ private final AuditSpanSource auditSpanSource; + /** Constructor to create S3A store. */ S3AStoreImpl(StoreContextFactory storeContextFactory, S3Client s3Client, DurationTrackerFactory durationTrackerFactory, @@ -129,11 +140,13 @@ public class S3AStoreImpl implements S3AStore { this.requestFactory = storeContext.getRequestFactory(); } + /** Acquire write capacity for rate limiting {@inheritDoc}. */ @Override public Duration acquireWriteCapacity(final int capacity) { return writeRateLimiter.acquire(capacity); } + /** Acquire read capacity for rate limiting {@inheritDoc}. */ @Override public Duration acquireReadCapacity(final int capacity) { return readRateLimiter.acquire(capacity); @@ -189,10 +202,6 @@ protected void incrementStatistic(Statistic statistic) { incrementStatistic(statistic, 1); } - protected void incrementDurationStatistic(Statistic statistic, Duration duration) { - statisticsContext.addValueToQuantiles(statistic, duration.toMillis()); - } - /** * Increment a statistic by a specific value. * This increments both the instrumentation and storage statistics. @@ -284,26 +293,7 @@ private void blockRootDelete(String key) throws IllegalArgumentException { } /** - * Perform a bulk object delete operation against S3. - * Increments the {@code OBJECT_DELETE_REQUESTS} and write - * operation statistics - *

- * {@code OBJECT_DELETE_OBJECTS} is updated with the actual number - * of objects deleted in the request. - *

- * Retry policy: retry untranslated; delete considered idempotent. - * If the request is throttled, this is logged in the throttle statistics, - * with the counter set to the number of keys, rather than the number - * of invocations of the delete operation. - * This is because S3 considers each key as one mutating operation on - * the store when updating its load counters on a specific partition - * of an S3 bucket. - * If only the request was measured, this operation would under-report. - * @param deleteRequest keys to delete on the s3-backend - * @return the AWS response - * @throws IllegalArgumentException if the request was rejected due to - * a mistaken attempt to delete the root directory - * @throws SdkException amazon-layer failure. + * {@inheritDoc}. */ @Override @Retries.RetryRaw @@ -334,6 +324,7 @@ public Map.Entry deleteObjects(final DeleteObje // duration is tracked in the bulk delete counters trackDurationOfOperation(getDurationTrackerFactory(), OBJECT_BULK_DELETE_REQUEST.getSymbol(), () -> { + // acquire the write capacity for the number of keys to delete and record the duration. Duration durationToAcquireWriteCapacity = acquireWriteCapacity(keyCount); instrumentation.recordDuration(STORE_IO_RATE_LIMITED, true, durationToAcquireWriteCapacity); incrementStatistic(OBJECT_DELETE_OBJECTS, keyCount); @@ -361,19 +352,7 @@ public Map.Entry deleteObjects(final DeleteObje } /** - * Delete an object. - * Increments the {@code OBJECT_DELETE_REQUESTS} statistics. - *

- * Retry policy: retry untranslated; delete considered idempotent. - * 404 errors other than bucket not found are swallowed; - * this can be raised by third party stores (GCS). - * If an exception is caught and swallowed, the response will be empty; - * otherwise it will be the response from the delete operation. - * @param request request to make - * @return the total duration and response. - * @throws SdkException problems working with S3 - * @throws IllegalArgumentException if the request was rejected due to - * a mistaken attempt to delete the root directory. + * {@inheritDoc}. */ @Override @Retries.RetryRaw @@ -389,7 +368,9 @@ public Map.Entry> deleteObject(final De DELETE_CONSIDERED_IDEMPOTENT, trackDurationOfOperation(getDurationTrackerFactory(), OBJECT_DELETE_REQUEST.getSymbol(), () -> { incrementStatistic(OBJECT_DELETE_OBJECTS); - acquireWriteCapacity(1); + // We try to acquire write capacity just before delete call. + Duration durationToAcquireWriteCapacity = acquireWriteCapacity(1); + instrumentation.recordDuration(STORE_IO_RATE_LIMITED, true, durationToAcquireWriteCapacity); return s3Client.deleteObject(request); })); d.close(); diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md index 2b11c7e5d6817..45445f51b150b 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md @@ -59,7 +59,7 @@ To make most efficient use of S3, care is needed. The S3A FileSystem supports implementation of vectored read api using which a client can provide a list of file ranges to read returning a future read object associated with each range. For full api specification please see -[FSDataInputStream](../../hadoop-common-project/hadoop-common/filesystem/fsdatainputstream.html). +[FSDataInputStream](../../../../../../hadoop-common-project/hadoop-common/target/site/filesystem/fsdatainputstream.html). The following properties can be configured to optimise vectored reads based on the client requirements. @@ -96,7 +96,7 @@ on the client requirements. ## Improving delete performance through bulkdelete API. -For bulk delete API spec refer to [BulkDelete](bulkdelete.html). +For bulk delete API spec refer to File System specification. [BulkDelete](../../../../../../hadoop-common-project/hadoop-common/target/site/filesystem/bulkdelete.html) The S3A client exports this API. @@ -171,7 +171,7 @@ That is: bulk delete requests of paths which do not exist will hurt future queri Avoid this. Note also that TPC-DS Benchmark do not create the right load to make the performance problems observable -but they can surface in production. * Configure buckets to have a limited number of days for tombstones to be preserved. -* Do not delete which you know do not contain objects. +* Do not delete paths which you know reference nonexistent files or directories. ## Improving data input performance through fadvise diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java index 24025b5180fc3..ae52a28e5b362 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java @@ -23,6 +23,7 @@ import java.util.List; import org.assertj.core.api.Assertions; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -44,7 +45,8 @@ import static java.util.stream.Collectors.toList; import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.createFiles; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; +import static org.apache.hadoop.fs.s3a.S3AUtils.propagateBucketOptions; import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupMeanStatistic; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.STORE_IO_RATE_LIMITED_DURATION; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MEAN; @@ -85,6 +87,9 @@ public ITestS3AContractBulkDelete(boolean enableMultiObjectDelete) { protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); S3ATestUtils.disableFilesystemCaching(conf); + conf = propagateBucketOptions(conf, getTestBucketName(conf)); + skipIfNotEnabled(conf, Constants.ENABLE_MULTI_DELETE, + "Bulk delete is explicitly disabled for this bucket"); S3ATestUtils.removeBaseAndBucketOverrides(conf, Constants.BULK_DELETE_PAGE_SIZE); conf.setInt(Constants.BULK_DELETE_PAGE_SIZE, DELETE_PAGE_SIZE); @@ -98,15 +103,19 @@ protected AbstractFSContract createContract(Configuration conf) { } @Override - public void validatePageSize() throws Exception { - int targetPageSize = DELETE_PAGE_SIZE; + protected int getExpectedPageSize() { if (!enableMultiObjectDelete) { // if multi-object delete is disabled, page size should be 1. - targetPageSize = 1; + return 1; } + return DELETE_PAGE_SIZE; + } + + @Override + public void validatePageSize() throws Exception { Assertions.assertThat(pageSize) .describedAs("Page size should match the configured page size") - .isEqualTo(targetPageSize); + .isEqualTo(getExpectedPageSize()); } @Test @@ -146,15 +155,35 @@ public void testDeletePathsDirectory() throws Exception { paths.add(dirPath); Path filePath = new Path(dirPath, "file"); touch(fs, filePath); - paths.add(filePath); - pageSizePreconditionForTest(paths.size()); + if (enableMultiObjectDelete) { + // Adding more paths only if multi-object delete is enabled. + paths.add(filePath); + } + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + // During the bulk delete operation, the directories are not deleted in S3A. + assertIsDirectory(dirPath); + } + + @Test + public void testBulkDeleteParentDirectory() throws Exception { + List paths = new ArrayList<>(); + Path dirPath = new Path(basePath, "dir"); + fs.mkdirs(dirPath); + Path subDir = new Path(dirPath, "subdir"); + fs.mkdirs(subDir); + // adding parent directory to the list of paths. + paths.add(dirPath); assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); // During the bulk delete operation, the directories are not deleted in S3A. assertIsDirectory(dirPath); + assertIsDirectory(subDir); } @Test public void testRateLimiting() throws Exception { + if (!enableMultiObjectDelete) { + skip("Multi-object delete is disabled so hard to trigger rate limiting"); + } Configuration conf = getContract().getConf(); conf.setInt(Constants.S3A_IO_RATE_LIMIT, 5); Path basePath = path(getMethodName()); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java index 8d6a3065854aa..9acd9b5199f08 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java @@ -63,6 +63,7 @@ import org.apache.hadoop.io.wrappedio.WrappedIO; import static org.apache.hadoop.fs.contract.AbstractContractBulkDeleteTest.assertSuccessfulBulkDelete; +import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.Constants.S3EXPRESS_CREATE_SESSION; @@ -734,8 +735,8 @@ private void executeBulkDeleteOnReadOnlyFiles(Configuration assumedRoleConfig) t bindReadOnlyRolePolicy(assumedRoleConfig, readOnlyDir); roleFS = (S3AFileSystem) destDir.getFileSystem(assumedRoleConfig); - - int range = 10; + int bulkDeletePageSize = WrappedIO.bulkDeletePageSize(roleFS, destDir); + int range = bulkDeletePageSize == 1 ? bulkDeletePageSize : 10; touchFiles(fs, readOnlyDir, range); touchFiles(roleFS, destDir, range); FileStatus[] fileStatuses = roleFS.listStatus(readOnlyDir); @@ -768,6 +769,11 @@ private void executeBulkDeleteOnSomeReadOnlyFiles(Configuration assumedRoleConfi bindReadOnlyRolePolicy(assumedRoleConfig, readOnlyDir); roleFS = (S3AFileSystem) destDir.getFileSystem(assumedRoleConfig); S3AFileSystem fs = getFileSystem(); + if (WrappedIO.bulkDeletePageSize(fs, destDir) == 1) { + String msg = "Skipping as this requires more than one paths to be deleted in bulk"; + LOG.debug(msg); + skip(msg); + } WrappedIO.bulkDelete(fs, destDir, new ArrayList<>()); // creating 5 files in the read only dir. int readOnlyRange = 5; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsContractBulkDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsContractBulkDelete.java index ca115983d33d9..3609176786a9b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsContractBulkDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsContractBulkDelete.java @@ -1,3 +1,21 @@ +/* + * 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.hadoop.fs.azurebfs.contract; import org.apache.hadoop.conf.Configuration; From 86544f254b44b504144c803fb6c28441a6c95ede Mon Sep 17 00:00:00 2001 From: Mukund Thakur Date: Wed, 24 Apr 2024 13:55:57 -0500 Subject: [PATCH 11/19] add path capability Add bulk delete path capability true for all FS --- .../hadoop/fs/DefaultBulkDeleteOperation.java | 1 - .../java/org/apache/hadoop/fs/FileSystem.java | 16 ++++++++++------ .../contract/AbstractContractBulkDeleteTest.java | 3 ++- .../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 5 ----- .../src/test/resources/log4j.properties | 1 + 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java index 47895a1efad33..70513d743d081 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java @@ -31,7 +31,6 @@ import static java.util.Objects.requireNonNull; import static org.apache.hadoop.fs.BulkDeleteUtils.validateBulkDeletePaths; -import static org.apache.hadoop.util.Preconditions.checkArgument; /** * Default implementation of the {@link BulkDelete} interface. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java index 768fd5b5e1caa..86e7c1b01d5b3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java @@ -3485,12 +3485,16 @@ public Collection getTrashRoots(boolean allUsers) { public boolean hasPathCapability(final Path path, final String capability) throws IOException { switch (validatePathCapabilityArgs(makeQualified(path), capability)) { - case CommonPathCapabilities.FS_SYMLINKS: - // delegate to the existing supportsSymlinks() call. - return supportsSymlinks() && areSymlinksEnabled(); - default: - // the feature is not implemented. - return false; + case CommonPathCapabilities.BULK_DELETE: + // bulk delete has default implementation which + // can called on any FileSystem. + return true; + case CommonPathCapabilities.FS_SYMLINKS: + // delegate to the existing supportsSymlinks() call. + return supportsSymlinks() && areSymlinksEnabled(); + default: + // the feature is not implemented. + return false; } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java index e912dbec03530..7cc4ca8e63177 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java @@ -256,7 +256,8 @@ public void testChildPaths() throws Exception { public static void assertSuccessfulBulkDelete(List> entries) { Assertions.assertThat(entries) - .describedAs("return entries should be empty after successful delete") + .describedAs("Bulk delete failed, " + + "return entries should be empty after successful delete") .isEmpty(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 19ae0c4c86c4b..b7392089141e9 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -5447,11 +5447,6 @@ public boolean hasPathCapability(final Path path, final String capability) case STORE_CAPABILITY_DIRECTORY_MARKER_AWARE: return true; - // this is always true, even if multi object - // delete is disabled -the page size is simply reduced to 1. - case CommonPathCapabilities.BULK_DELETE: - return true; - case ENABLE_MULTI_DELETE: return enableMultiObjectsDelete; diff --git a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties index 9f72d03653306..14bca5a85a84e 100644 --- a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties @@ -26,6 +26,7 @@ log4j.logger.org.apache.hadoop.fs.azure.AzureFileSystemThreadPoolExecutor=DEBUG log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG log4j.logger.org.apache.hadoop.fs.azurebfs.contracts.services.TracingService=TRACE log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsClient=DEBUG +log4j.logger.org.apache.hadoop.fs.DefaultBulkDeleteOperation=DEBUG # after here: turn off log messages from other parts of the system # which only clutter test reports. From 88c862386d9d4576ea6e2756efd23082d57a6f9f Mon Sep 17 00:00:00 2001 From: Mukund Thakur Date: Wed, 24 Apr 2024 16:07:30 -0500 Subject: [PATCH 12/19] Some more tests and docs --- .../hadoop/fs/DefaultBulkDeleteOperation.java | 3 + .../apache/hadoop/io/wrappedio/WrappedIO.java | 3 +- .../site/markdown/filesystem/bulkdelete.md | 11 ++-- .../AbstractContractBulkDeleteTest.java | 59 ++++++++++++++++--- .../TestLocalFSContractBulkDelete.java | 13 ++++ .../TestRawLocalContractBulkDelete.java | 14 +++++ .../hdfs/TestHDFSContractBulkDelete.java | 28 +++++++++ .../s3a/ITestS3AContractBulkDelete.java | 18 +++++- 8 files changed, 134 insertions(+), 15 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractBulkDelete.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractBulkDelete.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractBulkDelete.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java index 70513d743d081..40d38998b5a81 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java @@ -39,10 +39,13 @@ public class DefaultBulkDeleteOperation implements BulkDelete { private static Logger LOG = LoggerFactory.getLogger(DefaultBulkDeleteOperation.class); + /** Default page size for bulk delete. */ private static final int DEFAULT_PAGE_SIZE = 1; + /** Base path for the bulk delete operation. */ private final Path basePath; + /** Delegate File system make actual delete calls. */ private final FileSystem fs; public DefaultBulkDeleteOperation(Path basePath, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java index f5c455f53798e..e75fecb903f3b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java @@ -77,8 +77,7 @@ public static int bulkDeletePageSize(FileSystem fs, Path path) throws IOExceptio /** * Convert a filesystem to a bulk delete source. * @param fs filesystem - * @return cast fs. - * @throws UnsupportedOperationException FS doesn't implement the interface. + * @return cast fs or a default implementation. */ private static BulkDeleteSource toBulkDeleteSource(final FileSystem fs) { if (fs instanceof BulkDeleteSource) { diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md index 8216f74799003..ec97fc9f18ad0 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md @@ -94,8 +94,10 @@ No other restrictions are placed upon the outcome. ### Availability The `BulkDeleteSource` interface is exported by `FileSystem` and `FileContext` storage clients -which MAY support the API; it may still be unsupported by the -specific instance. +which is available for all FS via `org.apache.hadoop.fs.DefalutBulkDeleteSource` +Some FS MAY still decide to not support the API by overwriting the `createBulkDelete()` method +with an UnsupportedOperationException. While doing so they must also declare the path +capability `fs.capability.bulk.delete` as false. Use the `PathCapabilities` probe `fs.capability.bulk.delete`. @@ -127,8 +129,9 @@ through reflection. #### Default Implementation -The default implementation of the `BulkDelete` interface is `org.apache.hadoop.fs.DefaultBulkDeleteOperation` -which fixes the page size to be 1 and calls `FileSystem.delete(path, false)` on the single path in the list. +The default implementation which will be used by all implementation of `FileSystem` of the +`BulkDelete` interface is `org.apache.hadoop.fs.DefaultBulkDeleteOperation` which fixes the page +size to be 1 and calls `FileSystem.delete(path, false)` on the single path in the list. #### S3A Implementation diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java index 7cc4ca8e63177..565c66794fac5 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java @@ -47,10 +47,21 @@ public abstract class AbstractContractBulkDeleteTest extends AbstractFSContractT private static final Logger LOG = LoggerFactory.getLogger(AbstractContractBulkDeleteTest.class); + /** + * Page size for bulk delete. This is calculated based + * on the store implementation. + */ protected int pageSize; + /** + * Base path for the bulk delete tests. + * All the paths to be deleted should be under this base path. + */ protected Path basePath; + /** + * Test file system. + */ protected FileSystem fs; @Before @@ -77,7 +88,7 @@ protected int getExpectedPageSize() { public void validatePageSize() throws Exception { Assertions.assertThat(pageSize) .describedAs("Page size should be 1 by default for all stores") - .isEqualTo(1); + .isEqualTo(getExpectedPageSize()); } @Test @@ -159,18 +170,18 @@ public void testDeletePathsNotExists() throws Exception { public void testDeletePathsDirectory() throws Exception { List paths = new ArrayList<>(); Path dirPath = new Path(basePath, "dir"); - fs.mkdirs(dirPath); paths.add(dirPath); Path filePath = new Path(dirPath, "file"); - touch(fs, filePath); paths.add(filePath); pageSizePreconditionForTest(paths.size()); + fs.mkdirs(dirPath); + touch(fs, filePath); // Outcome is undefined. But call shouldn't fail. assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); } @Test - public void testBulkDeleteParentDirectory() throws Exception { + public void testBulkDeleteParentDirectoryWithDirectories() throws Exception { List paths = new ArrayList<>(); Path dirPath = new Path(basePath, "dir"); fs.mkdirs(dirPath); @@ -178,9 +189,33 @@ public void testBulkDeleteParentDirectory() throws Exception { fs.mkdirs(subDir); // adding parent directory to the list of paths. paths.add(dirPath); - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + List> entries = bulkDelete(getFileSystem(), basePath, paths); + Assertions.assertThat(entries) + .describedAs("Parent non empty directory should not be deleted") + .hasSize(1); + // During the bulk delete operation, the non-empty directories are not deleted in default implementation. + assertIsDirectory(dirPath); } + @Test + public void testBulkDeleteParentDirectoryWithFiles() throws Exception { + List paths = new ArrayList<>(); + Path dirPath = new Path(basePath, "dir"); + fs.mkdirs(dirPath); + Path file = new Path(dirPath, "file"); + touch(fs, file); + // adding parent directory to the list of paths. + paths.add(dirPath); + List> entries = bulkDelete(getFileSystem(), basePath, paths); + Assertions.assertThat(entries) + .describedAs("Parent non empty directory should not be deleted") + .hasSize(1); + // During the bulk delete operation, the non-empty directories are not deleted in default implementation. + assertIsDirectory(dirPath); + } + + + @Test public void testDeleteEmptyDirectory() throws Exception { List paths = new ArrayList<>(); @@ -202,13 +237,13 @@ public void testDeleteEmptyList() throws Exception { public void testDeleteSamePathsMoreThanOnce() throws Exception { List paths = new ArrayList<>(); Path path = new Path(basePath, "file"); - touch(fs, path); paths.add(path); paths.add(path); Path another = new Path(basePath, "another-file"); - touch(fs, another); paths.add(another); pageSizePreconditionForTest(paths.size()); + touch(fs, path); + touch(fs, another); assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); } @@ -217,7 +252,7 @@ public void testDeleteSamePathsMoreThanOnce() throws Exception { */ protected void pageSizePreconditionForTest(int size) { if (size > pageSize) { - skip("Test requires paths size less than or equal to page size"); + skip("Test requires paths size less than or equal to page size: " + pageSize); } } @@ -254,6 +289,10 @@ public void testChildPaths() throws Exception { } + /** + * Assert on returned entries after bulk delete operation. + * Entries should be empty after successful delete. + */ public static void assertSuccessfulBulkDelete(List> entries) { Assertions.assertThat(entries) .describedAs("Bulk delete failed, " + @@ -261,6 +300,10 @@ public static void assertSuccessfulBulkDelete(List> entr .isEmpty(); } + /** + * Create a list of paths with the given count + * under the given base path. + */ private List createListOfPaths(int count, Path basePath) { List paths = new ArrayList<>(); for (int i=0; i < count; i++) { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractBulkDelete.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractBulkDelete.java new file mode 100644 index 0000000000000..656fe1a28047c --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractBulkDelete.java @@ -0,0 +1,13 @@ +package org.apache.hadoop.fs.contract.localfs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractBulkDeleteTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +public class TestLocalFSContractBulkDelete extends AbstractContractBulkDeleteTest { + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new LocalFSContract(conf); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractBulkDelete.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractBulkDelete.java new file mode 100644 index 0000000000000..1981b6bc77450 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractBulkDelete.java @@ -0,0 +1,14 @@ +package org.apache.hadoop.fs.contract.rawlocal; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractBulkDeleteTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +public class TestRawLocalContractBulkDelete extends AbstractContractBulkDeleteTest { + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new RawlocalFSContract(conf); + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractBulkDelete.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractBulkDelete.java new file mode 100644 index 0000000000000..7666a75303643 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractBulkDelete.java @@ -0,0 +1,28 @@ +package org.apache.hadoop.fs.contract.hdfs; + +import java.io.IOException; + +import org.junit.AfterClass; +import org.junit.BeforeClass; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractBulkDeleteTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +public class TestHDFSContractBulkDelete extends AbstractContractBulkDeleteTest { + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new HDFSContract(conf); + } + + @BeforeClass + public static void createCluster() throws IOException { + HDFSContract.createCluster(); + } + + @AfterClass + public static void teardownCluster() throws IOException { + HDFSContract.destroyCluster(); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java index ae52a28e5b362..27a70601ac102 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import org.assertj.core.api.Assertions; import org.junit.Before; @@ -165,7 +166,7 @@ public void testDeletePathsDirectory() throws Exception { } @Test - public void testBulkDeleteParentDirectory() throws Exception { + public void testBulkDeleteParentDirectoryWithDirectories() throws Exception { List paths = new ArrayList<>(); Path dirPath = new Path(basePath, "dir"); fs.mkdirs(dirPath); @@ -179,6 +180,21 @@ public void testBulkDeleteParentDirectory() throws Exception { assertIsDirectory(subDir); } + public void testBulkDeleteParentDirectoryWithFiles() throws Exception { + List paths = new ArrayList<>(); + Path dirPath = new Path(basePath, "dir"); + fs.mkdirs(dirPath); + Path file = new Path(dirPath, "file"); + touch(fs, file); + // adding parent directory to the list of paths. + paths.add(dirPath); + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + // During the bulk delete operation, + // the directories are not deleted in S3A. + assertIsDirectory(dirPath); + } + + @Test public void testRateLimiting() throws Exception { if (!enableMultiObjectDelete) { From 0339eeb5bd4f0a90e5530abb8df9530f582d99b3 Mon Sep 17 00:00:00 2001 From: Mukund Thakur Date: Thu, 25 Apr 2024 17:04:39 -0500 Subject: [PATCH 13/19] self review comments --- .../org/apache/hadoop/fs/BulkDeleteUtils.java | 12 +++++++++++ .../TestLocalFSContractBulkDelete.java | 21 +++++++++++++++++++ .../TestRawLocalContractBulkDelete.java | 21 +++++++++++++++++++ .../hdfs/TestHDFSContractBulkDelete.java | 21 +++++++++++++++++++ .../hadoop/fs/s3a/auth/ITestAssumeRole.java | 2 +- 5 files changed, 76 insertions(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteUtils.java index fa7af088608a4..46ae78b30d99b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteUtils.java @@ -30,6 +30,12 @@ public final class BulkDeleteUtils { private BulkDeleteUtils() { } + /** + * Preconditions for bulk delete paths. + * @param paths paths to delete. + * @param pageSize maximum number of paths to delete in a single operation. + * @param basePath base path for the delete operation. + */ public static void validateBulkDeletePaths(Collection paths, int pageSize, Path basePath) { requireNonNull(paths); checkArgument(paths.size() <= pageSize, @@ -41,6 +47,12 @@ public static void validateBulkDeletePaths(Collection paths, int pageSize, }); } + /** + * Check if a path is under a base path. + * @param p path to check. + * @param basePath base path. + * @return true if the path is under the base path. + */ public static boolean validatePathIsUnderParent(Path p, Path basePath) { while (p.getParent() != null) { if (p.getParent().equals(basePath)) { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractBulkDelete.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractBulkDelete.java index 656fe1a28047c..9d838b6e93ef3 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractBulkDelete.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractBulkDelete.java @@ -1,9 +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.hadoop.fs.contract.localfs; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.contract.AbstractContractBulkDeleteTest; import org.apache.hadoop.fs.contract.AbstractFSContract; +/** + * Bulk delete contract tests for the local filesystem. + */ public class TestLocalFSContractBulkDelete extends AbstractContractBulkDeleteTest { @Override diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractBulkDelete.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractBulkDelete.java index 1981b6bc77450..7a78294747773 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractBulkDelete.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractBulkDelete.java @@ -1,9 +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.hadoop.fs.contract.rawlocal; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.contract.AbstractContractBulkDeleteTest; import org.apache.hadoop.fs.contract.AbstractFSContract; +/** + * Bulk delete contract tests for the raw local filesystem. + */ public class TestRawLocalContractBulkDelete extends AbstractContractBulkDeleteTest { @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractBulkDelete.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractBulkDelete.java index 7666a75303643..44ed8a9deabbf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractBulkDelete.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractBulkDelete.java @@ -1,3 +1,21 @@ +/* + * 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.hadoop.fs.contract.hdfs; import java.io.IOException; @@ -9,6 +27,9 @@ import org.apache.hadoop.fs.contract.AbstractContractBulkDeleteTest; import org.apache.hadoop.fs.contract.AbstractFSContract; +/** + * Bulk delete contract tests for the HDFS filesystem. + */ public class TestHDFSContractBulkDelete extends AbstractContractBulkDeleteTest { @Override diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java index 9acd9b5199f08..46f706ed8fab5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java @@ -770,7 +770,7 @@ private void executeBulkDeleteOnSomeReadOnlyFiles(Configuration assumedRoleConfi roleFS = (S3AFileSystem) destDir.getFileSystem(assumedRoleConfig); S3AFileSystem fs = getFileSystem(); if (WrappedIO.bulkDeletePageSize(fs, destDir) == 1) { - String msg = "Skipping as this requires more than one paths to be deleted in bulk"; + String msg = "Skipping as this test requires more than one paths to be deleted in bulk"; LOG.debug(msg); skip(msg); } From 0776025c413ce579109bf47b6c0403b717519c31 Mon Sep 17 00:00:00 2001 From: Mukund Thakur Date: Fri, 26 Apr 2024 16:55:58 -0500 Subject: [PATCH 14/19] checkstyle --- .../org/apache/hadoop/fs/BulkDeleteUtils.java | 70 +-- .../site/markdown/filesystem/bulkdelete.md | 12 +- .../AbstractContractBulkDeleteTest.java | 527 +++++++++--------- .../TestLocalFSContractBulkDelete.java | 8 +- .../TestRawLocalContractBulkDelete.java | 8 +- .../hdfs/TestHDFSContractBulkDelete.java | 28 +- .../hadoop/fs/s3a/impl/S3AStoreBuilder.java | 45 +- .../hadoop/fs/s3a/impl/S3AStoreImpl.java | 57 +- .../s3a/ITestS3AContractBulkDelete.java | 323 ++++++----- .../contract/ITestAbfsContractBulkDelete.java | 46 +- 10 files changed, 566 insertions(+), 558 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteUtils.java index 46ae78b30d99b..d9916429425e9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteUtils.java @@ -6,9 +6,9 @@ * 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 - * + *

+ * 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. @@ -27,40 +27,40 @@ */ public final class BulkDeleteUtils { - private BulkDeleteUtils() { - } + private BulkDeleteUtils() { + } - /** - * Preconditions for bulk delete paths. - * @param paths paths to delete. - * @param pageSize maximum number of paths to delete in a single operation. - * @param basePath base path for the delete operation. - */ - public static void validateBulkDeletePaths(Collection paths, int pageSize, Path basePath) { - requireNonNull(paths); - checkArgument(paths.size() <= pageSize, - "Number of paths (%d) is larger than the page size (%d)", paths.size(), pageSize); - paths.forEach(p -> { - checkArgument(p.isAbsolute(), "Path %s is not absolute", p); - checkArgument(validatePathIsUnderParent(p, basePath), - "Path %s is not under the base path %s", p, basePath); - }); - } + /** + * Preconditions for bulk delete paths. + * @param paths paths to delete. + * @param pageSize maximum number of paths to delete in a single operation. + * @param basePath base path for the delete operation. + */ + public static void validateBulkDeletePaths(Collection paths, int pageSize, Path basePath) { + requireNonNull(paths); + checkArgument(paths.size() <= pageSize, + "Number of paths (%d) is larger than the page size (%d)", paths.size(), pageSize); + paths.forEach(p -> { + checkArgument(p.isAbsolute(), "Path %s is not absolute", p); + checkArgument(validatePathIsUnderParent(p, basePath), + "Path %s is not under the base path %s", p, basePath); + }); + } - /** - * Check if a path is under a base path. - * @param p path to check. - * @param basePath base path. - * @return true if the path is under the base path. - */ - public static boolean validatePathIsUnderParent(Path p, Path basePath) { - while (p.getParent() != null) { - if (p.getParent().equals(basePath)) { - return true; - } - p = p.getParent(); - } - return false; + /** + * Check if a path is under a base path. + * @param p path to check. + * @param basePath base path. + * @return true if the path is under the base path. + */ + public static boolean validatePathIsUnderParent(Path p, Path basePath) { + while (p.getParent() != null) { + if (p.getParent().equals(basePath)) { + return true; + } + p = p.getParent(); } + return false; + } } diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md index ec97fc9f18ad0..f3913b9f7e9b0 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md @@ -45,7 +45,6 @@ it supports the API. @InterfaceAudience.Public @InterfaceStability.Unstable public interface BulkDeleteSource { - default BulkDelete createBulkDelete(Path path) throws UnsupportedOperationException, IllegalArgumentException, IOException; @@ -61,11 +60,8 @@ This is the bulk delete implementation returned by the `createBulkDelete()` call @InterfaceAudience.Public @InterfaceStability.Unstable public interface BulkDelete extends IOStatisticsSource, Closeable { - int pageSize(); - Path basePath(); - List> bulkDelete(List paths) throws IOException, IllegalArgumentException; @@ -94,9 +90,9 @@ No other restrictions are placed upon the outcome. ### Availability The `BulkDeleteSource` interface is exported by `FileSystem` and `FileContext` storage clients -which is available for all FS via `org.apache.hadoop.fs.DefalutBulkDeleteSource` +which is available for all FS via `org.apache.hadoop.fs.DefalutBulkDeleteSource` Some FS MAY still decide to not support the API by overwriting the `createBulkDelete()` method -with an UnsupportedOperationException. While doing so they must also declare the path +with an UnsupportedOperationException. While doing so they must also declare the path capability `fs.capability.bulk.delete` as false. Use the `PathCapabilities` probe `fs.capability.bulk.delete`. @@ -119,7 +115,7 @@ through reflection. ```java public static int bulkDeletePageSize(FileSystem fs, Path path) throws IOException; - + public static int bulkDeletePageSize(FileSystem fs, Path path) throws IOException; public static List> bulkDelete(FileSystem fs, Path base, Collection paths); @@ -135,6 +131,6 @@ size to be 1 and calls `FileSystem.delete(path, false)` on the single path in th #### S3A Implementation -The S3A implementation is `org.apache.hadoop.fs.s3a.impl.BulkDeleteOperation` which implements the +The S3A implementation is `org.apache.hadoop.fs.s3a.impl.BulkDeleteOperation` which implements the multi object delete semantics of the AWS S3 API [Bulk Delete](https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html) For more details please refer to the S3A Performance documentation. \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java index 565c66794fac5..a0927ecd2638e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java @@ -44,272 +44,271 @@ */ public abstract class AbstractContractBulkDeleteTest extends AbstractFSContractTestBase { - private static final Logger LOG = - LoggerFactory.getLogger(AbstractContractBulkDeleteTest.class); - - /** - * Page size for bulk delete. This is calculated based - * on the store implementation. - */ - protected int pageSize; - - /** - * Base path for the bulk delete tests. - * All the paths to be deleted should be under this base path. - */ - protected Path basePath; - - /** - * Test file system. - */ - protected FileSystem fs; - - @Before - public void setUp() throws Exception { - fs = getFileSystem(); - basePath = path(getClass().getName()); - pageSize = WrappedIO.bulkDeletePageSize(getFileSystem(), basePath); - fs.mkdirs(basePath); + private static final Logger LOG = + LoggerFactory.getLogger(AbstractContractBulkDeleteTest.class); + + /** + * Page size for bulk delete. This is calculated based + * on the store implementation. + */ + protected int pageSize; + + /** + * Base path for the bulk delete tests. + * All the paths to be deleted should be under this base path. + */ + protected Path basePath; + + /** + * Test file system. + */ + protected FileSystem fs; + + @Before + public void setUp() throws Exception { + fs = getFileSystem(); + basePath = path(getClass().getName()); + pageSize = WrappedIO.bulkDeletePageSize(getFileSystem(), basePath); + fs.mkdirs(basePath); + } + + public Path getBasePath() { + return basePath; + } + + protected int getExpectedPageSize() { + return 1; + } + + /** + * Validate the page size for bulk delete operation. Different stores can have different + * implementations for bulk delete operation thus different page size. + */ + @Test + public void validatePageSize() throws Exception { + Assertions.assertThat(pageSize) + .describedAs("Page size should be 1 by default for all stores") + .isEqualTo(getExpectedPageSize()); + } + + @Test + public void testPathsSizeEqualsPageSizePrecondition() throws Exception { + List listOfPaths = createListOfPaths(pageSize, basePath); + // Bulk delete call should pass with no exception. + bulkDelete(getFileSystem(), basePath, listOfPaths); + } + + @Test + public void testPathsSizeGreaterThanPageSizePrecondition() throws Exception { + List listOfPaths = createListOfPaths(pageSize + 1, basePath); + intercept(IllegalArgumentException.class, + () -> bulkDelete(getFileSystem(), basePath, listOfPaths)); + } + + @Test + public void testPathsSizeLessThanPageSizePrecondition() throws Exception { + List listOfPaths = createListOfPaths(pageSize - 1, basePath); + // Bulk delete call should pass with no exception. + bulkDelete(getFileSystem(), basePath, listOfPaths); + } + + @Test + public void testBulkDeleteSuccessful() throws Exception { + List listOfPaths = createListOfPaths(pageSize, basePath); + for (Path path : listOfPaths) { + touch(fs, path); } - - public Path getBasePath() { - return basePath; - } - - protected int getExpectedPageSize() { - return 1; - } - - /** - * Validate the page size for bulk delete operation. Different stores can have different - * implementations for bulk delete operation thus different page size. - */ - @Test - public void validatePageSize() throws Exception { - Assertions.assertThat(pageSize) - .describedAs("Page size should be 1 by default for all stores") - .isEqualTo(getExpectedPageSize()); - } - - @Test - public void testPathsSizeEqualsPageSizePrecondition() throws Exception { - List listOfPaths = createListOfPaths(pageSize, basePath); - // Bulk delete call should pass with no exception. - bulkDelete(getFileSystem(), basePath, listOfPaths); - } - - @Test - public void testPathsSizeGreaterThanPageSizePrecondition() throws Exception { - List listOfPaths = createListOfPaths(pageSize + 1, basePath); - intercept(IllegalArgumentException.class, - () -> bulkDelete(getFileSystem(), basePath, listOfPaths)); - } - - @Test - public void testPathsSizeLessThanPageSizePrecondition() throws Exception { - List listOfPaths = createListOfPaths(pageSize - 1, basePath); - // Bulk delete call should pass with no exception. - bulkDelete(getFileSystem(), basePath, listOfPaths); - } - - @Test - public void testBulkDeleteSuccessful() throws Exception { - List listOfPaths = createListOfPaths(pageSize, basePath); - for (Path path : listOfPaths) { - touch(fs, path); - } - FileStatus[] fileStatuses = fs.listStatus(basePath); - Assertions.assertThat(fileStatuses) - .describedAs("File count after create") - .hasSize(pageSize); - assertSuccessfulBulkDelete( + FileStatus[] fileStatuses = fs.listStatus(basePath); + Assertions.assertThat(fileStatuses) + .describedAs("File count after create") + .hasSize(pageSize); + assertSuccessfulBulkDelete( bulkDelete(getFileSystem(), basePath, listOfPaths)); - FileStatus[] fileStatusesAfterDelete = fs.listStatus(basePath); - Assertions.assertThat(fileStatusesAfterDelete) - .describedAs("File statuses should be empty after delete") - .isEmpty(); - } - - @Test - public void validatePathCapabilityDeclared() throws Exception { - Assertions.assertThat(fs.hasPathCapability(basePath, CommonPathCapabilities.BULK_DELETE)) - .describedAs("Path capability BULK_DELETE should be declared") - .isTrue(); - } - - @Test - public void testDeletePathsNotUnderBase() throws Exception { - List paths = new ArrayList<>(); - Path pathNotUnderBase = path("not-under-base"); - paths.add(pathNotUnderBase); - // Should fail as path is not under the base path. - intercept(IllegalArgumentException.class, - () -> bulkDelete(getFileSystem(), basePath, paths)); - } - - @Test - public void testDeletePathsNotAbsolute() throws Exception { - List paths = new ArrayList<>(); - Path pathNotAbsolute = new Path("not-absolute"); - paths.add(pathNotAbsolute); - // Should fail as path is not absolute. - intercept(IllegalArgumentException.class, - () -> bulkDelete(getFileSystem(), basePath, paths)); - } - - @Test - public void testDeletePathsNotExists() throws Exception { - List paths = new ArrayList<>(); - Path pathNotExists = new Path(basePath, "not-exists"); - paths.add(pathNotExists); - // bulk delete call doesn't verify if a path exist or not before deleting. - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); - } - - @Test - public void testDeletePathsDirectory() throws Exception { - List paths = new ArrayList<>(); - Path dirPath = new Path(basePath, "dir"); - paths.add(dirPath); - Path filePath = new Path(dirPath, "file"); - paths.add(filePath); - pageSizePreconditionForTest(paths.size()); - fs.mkdirs(dirPath); - touch(fs, filePath); - // Outcome is undefined. But call shouldn't fail. - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + FileStatus[] fileStatusesAfterDelete = fs.listStatus(basePath); + Assertions.assertThat(fileStatusesAfterDelete) + .describedAs("File statuses should be empty after delete") + .isEmpty(); + } + + @Test + public void validatePathCapabilityDeclared() throws Exception { + Assertions.assertThat(fs.hasPathCapability(basePath, CommonPathCapabilities.BULK_DELETE)) + .describedAs("Path capability BULK_DELETE should be declared") + .isTrue(); + } + + @Test + public void testDeletePathsNotUnderBase() throws Exception { + List paths = new ArrayList<>(); + Path pathNotUnderBase = path("not-under-base"); + paths.add(pathNotUnderBase); + // Should fail as path is not under the base path. + intercept(IllegalArgumentException.class, + () -> bulkDelete(getFileSystem(), basePath, paths)); + } + + @Test + public void testDeletePathsNotAbsolute() throws Exception { + List paths = new ArrayList<>(); + Path pathNotAbsolute = new Path("not-absolute"); + paths.add(pathNotAbsolute); + // Should fail as path is not absolute. + intercept(IllegalArgumentException.class, + () -> bulkDelete(getFileSystem(), basePath, paths)); + } + + @Test + public void testDeletePathsNotExists() throws Exception { + List paths = new ArrayList<>(); + Path pathNotExists = new Path(basePath, "not-exists"); + paths.add(pathNotExists); + // bulk delete call doesn't verify if a path exist or not before deleting. + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + } + + @Test + public void testDeletePathsDirectory() throws Exception { + List paths = new ArrayList<>(); + Path dirPath = new Path(basePath, "dir"); + paths.add(dirPath); + Path filePath = new Path(dirPath, "file"); + paths.add(filePath); + pageSizePreconditionForTest(paths.size()); + fs.mkdirs(dirPath); + touch(fs, filePath); + // Outcome is undefined. But call shouldn't fail. + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + } + + @Test + public void testBulkDeleteParentDirectoryWithDirectories() throws Exception { + List paths = new ArrayList<>(); + Path dirPath = new Path(basePath, "dir"); + fs.mkdirs(dirPath); + Path subDir = new Path(dirPath, "subdir"); + fs.mkdirs(subDir); + // adding parent directory to the list of paths. + paths.add(dirPath); + List> entries = bulkDelete(getFileSystem(), basePath, paths); + Assertions.assertThat(entries) + .describedAs("Parent non empty directory should not be deleted") + .hasSize(1); + // During the bulk delete operation, the non-empty directories are not deleted in default implementation. + assertIsDirectory(dirPath); + } + + @Test + public void testBulkDeleteParentDirectoryWithFiles() throws Exception { + List paths = new ArrayList<>(); + Path dirPath = new Path(basePath, "dir"); + fs.mkdirs(dirPath); + Path file = new Path(dirPath, "file"); + touch(fs, file); + // adding parent directory to the list of paths. + paths.add(dirPath); + List> entries = bulkDelete(getFileSystem(), basePath, paths); + Assertions.assertThat(entries) + .describedAs("Parent non empty directory should not be deleted") + .hasSize(1); + // During the bulk delete operation, the non-empty directories are not deleted in default implementation. + assertIsDirectory(dirPath); + } + + + @Test + public void testDeleteEmptyDirectory() throws Exception { + List paths = new ArrayList<>(); + Path emptyDirPath = new Path(basePath, "empty-dir"); + fs.mkdirs(emptyDirPath); + paths.add(emptyDirPath); + // Should pass as empty directory. + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + } + + @Test + public void testDeleteEmptyList() throws Exception { + List paths = new ArrayList<>(); + // Empty list should pass. + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + } + + @Test + public void testDeleteSamePathsMoreThanOnce() throws Exception { + List paths = new ArrayList<>(); + Path path = new Path(basePath, "file"); + paths.add(path); + paths.add(path); + Path another = new Path(basePath, "another-file"); + paths.add(another); + pageSizePreconditionForTest(paths.size()); + touch(fs, path); + touch(fs, another); + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + } + + /** + * Skip test if paths size is greater than page size. + */ + protected void pageSizePreconditionForTest(int size) { + if (size > pageSize) { + skip("Test requires paths size less than or equal to page size: " + pageSize); } - - @Test - public void testBulkDeleteParentDirectoryWithDirectories() throws Exception { - List paths = new ArrayList<>(); - Path dirPath = new Path(basePath, "dir"); - fs.mkdirs(dirPath); - Path subDir = new Path(dirPath, "subdir"); - fs.mkdirs(subDir); - // adding parent directory to the list of paths. - paths.add(dirPath); - List> entries = bulkDelete(getFileSystem(), basePath, paths); - Assertions.assertThat(entries) - .describedAs("Parent non empty directory should not be deleted") - .hasSize(1); - // During the bulk delete operation, the non-empty directories are not deleted in default implementation. - assertIsDirectory(dirPath); - } - - @Test - public void testBulkDeleteParentDirectoryWithFiles() throws Exception { - List paths = new ArrayList<>(); - Path dirPath = new Path(basePath, "dir"); - fs.mkdirs(dirPath); - Path file = new Path(dirPath, "file"); - touch(fs, file); - // adding parent directory to the list of paths. - paths.add(dirPath); - List> entries = bulkDelete(getFileSystem(), basePath, paths); - Assertions.assertThat(entries) - .describedAs("Parent non empty directory should not be deleted") - .hasSize(1); - // During the bulk delete operation, the non-empty directories are not deleted in default implementation. - assertIsDirectory(dirPath); - } - - - - @Test - public void testDeleteEmptyDirectory() throws Exception { - List paths = new ArrayList<>(); - Path emptyDirPath = new Path(basePath, "empty-dir"); - fs.mkdirs(emptyDirPath); - paths.add(emptyDirPath); - // Should pass as empty directory. - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); - } - - @Test - public void testDeleteEmptyList() throws Exception { - List paths = new ArrayList<>(); - // Empty list should pass. - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); - } - - @Test - public void testDeleteSamePathsMoreThanOnce() throws Exception { - List paths = new ArrayList<>(); - Path path = new Path(basePath, "file"); - paths.add(path); - paths.add(path); - Path another = new Path(basePath, "another-file"); - paths.add(another); - pageSizePreconditionForTest(paths.size()); - touch(fs, path); - touch(fs, another); - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); - } - - /** - * Skip test if paths size is greater than page size. - */ - protected void pageSizePreconditionForTest(int size) { - if (size > pageSize) { - skip("Test requires paths size less than or equal to page size: " + pageSize); - } - } - - /** - * This test validates that files to be deleted don't have - * to be direct children of the base path. - */ - @Test - public void testDeepDirectoryFilesDelete() throws Exception { - List paths = new ArrayList<>(); - Path dir1 = new Path(basePath, "dir1"); - Path dir2 = new Path(dir1, "dir2"); - Path dir3 = new Path(dir2, "dir3"); - fs.mkdirs(dir3); - Path file1 = new Path(dir3, "file1"); - touch(fs, file1); - paths.add(file1); - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); - } - - - @Test - public void testChildPaths() throws Exception { - List paths = new ArrayList<>(); - Path dirPath = new Path(basePath, "dir"); - fs.mkdirs(dirPath); - paths.add(dirPath); - Path filePath = new Path(dirPath, "file"); - touch(fs, filePath); - paths.add(filePath); - pageSizePreconditionForTest(paths.size()); - // Should pass as both paths are under the base path. - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); - } - - - /** - * Assert on returned entries after bulk delete operation. - * Entries should be empty after successful delete. - */ - public static void assertSuccessfulBulkDelete(List> entries) { - Assertions.assertThat(entries) - .describedAs("Bulk delete failed, " + - "return entries should be empty after successful delete") - .isEmpty(); - } - - /** - * Create a list of paths with the given count - * under the given base path. - */ - private List createListOfPaths(int count, Path basePath) { - List paths = new ArrayList<>(); - for (int i=0; i < count; i++) { - Path path = new Path(basePath, "file-" + i); - paths.add(path); - } - return paths; + } + + /** + * This test validates that files to be deleted don't have + * to be direct children of the base path. + */ + @Test + public void testDeepDirectoryFilesDelete() throws Exception { + List paths = new ArrayList<>(); + Path dir1 = new Path(basePath, "dir1"); + Path dir2 = new Path(dir1, "dir2"); + Path dir3 = new Path(dir2, "dir3"); + fs.mkdirs(dir3); + Path file1 = new Path(dir3, "file1"); + touch(fs, file1); + paths.add(file1); + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + } + + + @Test + public void testChildPaths() throws Exception { + List paths = new ArrayList<>(); + Path dirPath = new Path(basePath, "dir"); + fs.mkdirs(dirPath); + paths.add(dirPath); + Path filePath = new Path(dirPath, "file"); + touch(fs, filePath); + paths.add(filePath); + pageSizePreconditionForTest(paths.size()); + // Should pass as both paths are under the base path. + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + } + + + /** + * Assert on returned entries after bulk delete operation. + * Entries should be empty after successful delete. + */ + public static void assertSuccessfulBulkDelete(List> entries) { + Assertions.assertThat(entries) + .describedAs("Bulk delete failed, " + + "return entries should be empty after successful delete") + .isEmpty(); + } + + /** + * Create a list of paths with the given count + * under the given base path. + */ + private List createListOfPaths(int count, Path basePath) { + List paths = new ArrayList<>(); + for (int i = 0; i < count; i++) { + Path path = new Path(basePath, "file-" + i); + paths.add(path); } + return paths; + } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractBulkDelete.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractBulkDelete.java index 9d838b6e93ef3..f1bd641806f42 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractBulkDelete.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractBulkDelete.java @@ -27,8 +27,8 @@ */ public class TestLocalFSContractBulkDelete extends AbstractContractBulkDeleteTest { - @Override - protected AbstractFSContract createContract(Configuration conf) { - return new LocalFSContract(conf); - } + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new LocalFSContract(conf); + } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractBulkDelete.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractBulkDelete.java index 7a78294747773..46d98249ab327 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractBulkDelete.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractBulkDelete.java @@ -27,9 +27,9 @@ */ public class TestRawLocalContractBulkDelete extends AbstractContractBulkDeleteTest { - @Override - protected AbstractFSContract createContract(Configuration conf) { - return new RawlocalFSContract(conf); - } + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new RawlocalFSContract(conf); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractBulkDelete.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractBulkDelete.java index 44ed8a9deabbf..3a851b6ff1c37 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractBulkDelete.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractBulkDelete.java @@ -32,18 +32,18 @@ */ public class TestHDFSContractBulkDelete extends AbstractContractBulkDeleteTest { - @Override - protected AbstractFSContract createContract(Configuration conf) { - return new HDFSContract(conf); - } - - @BeforeClass - public static void createCluster() throws IOException { - HDFSContract.createCluster(); - } - - @AfterClass - public static void teardownCluster() throws IOException { - HDFSContract.destroyCluster(); - } + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new HDFSContract(conf); + } + + @BeforeClass + public static void createCluster() throws IOException { + HDFSContract.createCluster(); + } + + @AfterClass + public static void teardownCluster() throws IOException { + HDFSContract.destroyCluster(); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java index 40f649a7378b6..44bac68e71b8f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java @@ -52,48 +52,57 @@ public class S3AStoreBuilder { private AuditSpanSource auditSpanSource; - public S3AStoreBuilder withStoreContextFactory(final StoreContextFactory storeContextFactory) { - this.storeContextFactory = storeContextFactory; + public S3AStoreBuilder withStoreContextFactory( + final StoreContextFactory storeContextFactoryValue) { + this.storeContextFactory = storeContextFactoryValue; return this; } - public S3AStoreBuilder withS3Client(final S3Client s3Client) { - this.s3Client = s3Client; + public S3AStoreBuilder withS3Client( + final S3Client s3ClientValue) { + this.s3Client = s3ClientValue; return this; } - public S3AStoreBuilder withDurationTrackerFactory(final DurationTrackerFactory durationTrackerFactory) { - this.durationTrackerFactory = durationTrackerFactory; + public S3AStoreBuilder withDurationTrackerFactory( + final DurationTrackerFactory durationTrackerFactoryValue) { + this.durationTrackerFactory = durationTrackerFactoryValue; return this; } - public S3AStoreBuilder withInstrumentation(final S3AInstrumentation instrumentation) { - this.instrumentation = instrumentation; + public S3AStoreBuilder withInstrumentation( + final S3AInstrumentation instrumentationValue) { + this.instrumentation = instrumentationValue; return this; } - public S3AStoreBuilder withStatisticsContext(final S3AStatisticsContext statisticsContext) { - this.statisticsContext = statisticsContext; + public S3AStoreBuilder withStatisticsContext( + final S3AStatisticsContext statisticsContextValue) { + this.statisticsContext = statisticsContextValue; return this; } - public S3AStoreBuilder withStorageStatistics(final S3AStorageStatistics storageStatistics) { - this.storageStatistics = storageStatistics; + public S3AStoreBuilder withStorageStatistics( + final S3AStorageStatistics storageStatisticsValue) { + this.storageStatistics = storageStatisticsValue; return this; } - public S3AStoreBuilder withReadRateLimiter(final RateLimiting readRateLimiter) { - this.readRateLimiter = readRateLimiter; + public S3AStoreBuilder withReadRateLimiter( + final RateLimiting readRateLimiterValue) { + this.readRateLimiter = readRateLimiterValue; return this; } - public S3AStoreBuilder withWriteRateLimiter(final RateLimiting writeRateLimiter) { - this.writeRateLimiter = writeRateLimiter; + public S3AStoreBuilder withWriteRateLimiter( + final RateLimiting writeRateLimiterValue) { + this.writeRateLimiter = writeRateLimiterValue; return this; } - public S3AStoreBuilder withAuditSpanSource(final AuditSpanSource auditSpanSource) { - this.auditSpanSource = auditSpanSource; + public S3AStoreBuilder withAuditSpanSource( + final AuditSpanSource auditSpanSourceValue) { + this.auditSpanSource = auditSpanSourceValue; return this; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java index d7b7f4115063b..8df5eb90e6f4c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java @@ -297,7 +297,8 @@ private void blockRootDelete(String key) throws IllegalArgumentException { */ @Override @Retries.RetryRaw - public Map.Entry deleteObjects(final DeleteObjectsRequest deleteRequest) + public Map.Entry deleteObjects( + final DeleteObjectsRequest deleteRequest) throws SdkException { DeleteObjectsResponse response; @@ -317,19 +318,22 @@ public Map.Entry deleteObjects(final DeleteObje try (DurationInfo d = new DurationInfo(LOG, false, "DELETE %d keys", keyCount)) { response = - invoker.retryUntranslated("delete", DELETE_CONSIDERED_IDEMPOTENT, (text, e, r, i) -> { - // handle the failure - retryHandler.bulkDeleteRetried(deleteRequest, e); - }, - // duration is tracked in the bulk delete counters - trackDurationOfOperation(getDurationTrackerFactory(), - OBJECT_BULK_DELETE_REQUEST.getSymbol(), () -> { - // acquire the write capacity for the number of keys to delete and record the duration. - Duration durationToAcquireWriteCapacity = acquireWriteCapacity(keyCount); - instrumentation.recordDuration(STORE_IO_RATE_LIMITED, true, durationToAcquireWriteCapacity); - incrementStatistic(OBJECT_DELETE_OBJECTS, keyCount); - return s3Client.deleteObjects(deleteRequest); - })); + invoker.retryUntranslated("delete", + DELETE_CONSIDERED_IDEMPOTENT, (text, e, r, i) -> { + // handle the failure + retryHandler.bulkDeleteRetried(deleteRequest, e); + }, + // duration is tracked in the bulk delete counters + trackDurationOfOperation(getDurationTrackerFactory(), + OBJECT_BULK_DELETE_REQUEST.getSymbol(), () -> { + // acquire the write capacity for the number of keys to delete and record the duration. + Duration durationToAcquireWriteCapacity = acquireWriteCapacity(keyCount); + instrumentation.recordDuration(STORE_IO_RATE_LIMITED, + true, + durationToAcquireWriteCapacity); + incrementStatistic(OBJECT_DELETE_OBJECTS, keyCount); + return s3Client.deleteObjects(deleteRequest); + })); if (!response.errors().isEmpty()) { // one or more of the keys could not be deleted. // log and then throw @@ -356,23 +360,26 @@ public Map.Entry deleteObjects(final DeleteObje */ @Override @Retries.RetryRaw - public Map.Entry> deleteObject(final DeleteObjectRequest request) - throws SdkException { + public Map.Entry> deleteObject( + final DeleteObjectRequest request) + throws SdkException { String key = request.key(); blockRootDelete(key); DurationInfo d = new DurationInfo(LOG, false, "deleting %s", key); try { DeleteObjectResponse response = - invoker.retryUntranslated(String.format("Delete %s:/%s", bucket, key), - DELETE_CONSIDERED_IDEMPOTENT, trackDurationOfOperation(getDurationTrackerFactory(), - OBJECT_DELETE_REQUEST.getSymbol(), () -> { - incrementStatistic(OBJECT_DELETE_OBJECTS); - // We try to acquire write capacity just before delete call. - Duration durationToAcquireWriteCapacity = acquireWriteCapacity(1); - instrumentation.recordDuration(STORE_IO_RATE_LIMITED, true, durationToAcquireWriteCapacity); - return s3Client.deleteObject(request); - })); + invoker.retryUntranslated(String.format("Delete %s:/%s", bucket, key), + DELETE_CONSIDERED_IDEMPOTENT, + trackDurationOfOperation(getDurationTrackerFactory(), + OBJECT_DELETE_REQUEST.getSymbol(), () -> { + incrementStatistic(OBJECT_DELETE_OBJECTS); + // We try to acquire write capacity just before delete call. + Duration durationToAcquireWriteCapacity = acquireWriteCapacity(1); + instrumentation.recordDuration(STORE_IO_RATE_LIMITED, + true, durationToAcquireWriteCapacity); + return s3Client.deleteObject(request); + })); d.close(); return Tuples.pair(d.asDuration(), Optional.of(response)); } catch (AwsServiceException ase) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java index 27a70601ac102..7c7528fa57ae4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java @@ -21,10 +21,8 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Map; import org.assertj.core.api.Assertions; -import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -41,7 +39,6 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.statistics.MeanStatistic; -import org.apache.hadoop.io.wrappedio.WrappedIO; import static java.util.stream.Collectors.toList; import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; @@ -60,171 +57,171 @@ @RunWith(Parameterized.class) public class ITestS3AContractBulkDelete extends AbstractContractBulkDeleteTest { - private static final Logger LOG = LoggerFactory.getLogger(ITestS3AContractBulkDelete.class); - - /** - * Delete Page size: {@value}. - * This is the default page size for bulk delete operation for this contract test. - * All the tests in this class should pass number of paths equal to or less than - * this page size during the bulk delete operation. - */ - private static final int DELETE_PAGE_SIZE = 20; - - private final boolean enableMultiObjectDelete; - - @Parameterized.Parameters(name = "enableMultiObjectDelete = {0}") - public static Iterable enableMultiObjectDelete() { - return Arrays.asList(new Object[][] { - {true}, - {false} - }); - } - - public ITestS3AContractBulkDelete(boolean enableMultiObjectDelete) { - this.enableMultiObjectDelete = enableMultiObjectDelete; - } - - @Override - protected Configuration createConfiguration() { - Configuration conf = super.createConfiguration(); - S3ATestUtils.disableFilesystemCaching(conf); - conf = propagateBucketOptions(conf, getTestBucketName(conf)); - skipIfNotEnabled(conf, Constants.ENABLE_MULTI_DELETE, - "Bulk delete is explicitly disabled for this bucket"); - S3ATestUtils.removeBaseAndBucketOverrides(conf, - Constants.BULK_DELETE_PAGE_SIZE); - conf.setInt(Constants.BULK_DELETE_PAGE_SIZE, DELETE_PAGE_SIZE); - conf.setBoolean(Constants.ENABLE_MULTI_DELETE, enableMultiObjectDelete); - return conf; - } - - @Override - protected AbstractFSContract createContract(Configuration conf) { - return new S3AContract(createConfiguration()); - } - - @Override - protected int getExpectedPageSize() { - if (!enableMultiObjectDelete) { - // if multi-object delete is disabled, page size should be 1. - return 1; - } - return DELETE_PAGE_SIZE; + private static final Logger LOG = LoggerFactory.getLogger(ITestS3AContractBulkDelete.class); + + /** + * Delete Page size: {@value}. + * This is the default page size for bulk delete operation for this contract test. + * All the tests in this class should pass number of paths equal to or less than + * this page size during the bulk delete operation. + */ + private static final int DELETE_PAGE_SIZE = 20; + + private final boolean enableMultiObjectDelete; + + @Parameterized.Parameters(name = "enableMultiObjectDelete = {0}") + public static Iterable enableMultiObjectDelete() { + return Arrays.asList(new Object[][]{ + {true}, + {false} + }); + } + + public ITestS3AContractBulkDelete(boolean enableMultiObjectDelete) { + this.enableMultiObjectDelete = enableMultiObjectDelete; + } + + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + S3ATestUtils.disableFilesystemCaching(conf); + conf = propagateBucketOptions(conf, getTestBucketName(conf)); + skipIfNotEnabled(conf, Constants.ENABLE_MULTI_DELETE, + "Bulk delete is explicitly disabled for this bucket"); + S3ATestUtils.removeBaseAndBucketOverrides(conf, + Constants.BULK_DELETE_PAGE_SIZE); + conf.setInt(Constants.BULK_DELETE_PAGE_SIZE, DELETE_PAGE_SIZE); + conf.setBoolean(Constants.ENABLE_MULTI_DELETE, enableMultiObjectDelete); + return conf; + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new S3AContract(createConfiguration()); + } + + @Override + protected int getExpectedPageSize() { + if (!enableMultiObjectDelete) { + // if multi-object delete is disabled, page size should be 1. + return 1; } - - @Override - public void validatePageSize() throws Exception { - Assertions.assertThat(pageSize) - .describedAs("Page size should match the configured page size") - .isEqualTo(getExpectedPageSize()); + return DELETE_PAGE_SIZE; + } + + @Override + public void validatePageSize() throws Exception { + Assertions.assertThat(pageSize) + .describedAs("Page size should match the configured page size") + .isEqualTo(getExpectedPageSize()); + } + + @Test + public void testBulkDeleteZeroPageSizePrecondition() throws Exception { + if (!enableMultiObjectDelete) { + // if multi-object delete is disabled, skip this test as + // page size is always 1. + skip("Multi-object delete is disabled"); } - - @Test - public void testBulkDeleteZeroPageSizePrecondition() throws Exception { - if(!enableMultiObjectDelete) { - // if multi-object delete is disabled, skip this test as - // page size is always 1. - skip("Multi-object delete is disabled"); - } - Configuration conf = getContract().getConf(); - conf.setInt(Constants.BULK_DELETE_PAGE_SIZE, 0); - Path testPath = path(getMethodName()); - try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) { - intercept(IllegalArgumentException.class, - () -> fs.createBulkDelete(testPath)); - } - } - - @Test - public void testPageSizeWhenMultiObjectsDisabled() throws Exception { - Configuration conf = getContract().getConf(); - conf.setBoolean(Constants.ENABLE_MULTI_DELETE, false); - Path testPath = path(getMethodName()); - try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) { - BulkDelete bulkDelete = fs.createBulkDelete(testPath); - Assertions.assertThat(bulkDelete.pageSize()) - .describedAs("Page size should be 1 when multi-object delete is disabled") - .isEqualTo(1); - } + Configuration conf = getContract().getConf(); + conf.setInt(Constants.BULK_DELETE_PAGE_SIZE, 0); + Path testPath = path(getMethodName()); + try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) { + intercept(IllegalArgumentException.class, + () -> fs.createBulkDelete(testPath)); } - - @Override - public void testDeletePathsDirectory() throws Exception { - List paths = new ArrayList<>(); - Path dirPath = new Path(basePath, "dir"); - fs.mkdirs(dirPath); - paths.add(dirPath); - Path filePath = new Path(dirPath, "file"); - touch(fs, filePath); - if (enableMultiObjectDelete) { - // Adding more paths only if multi-object delete is enabled. - paths.add(filePath); - } - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); - // During the bulk delete operation, the directories are not deleted in S3A. - assertIsDirectory(dirPath); + } + + @Test + public void testPageSizeWhenMultiObjectsDisabled() throws Exception { + Configuration conf = getContract().getConf(); + conf.setBoolean(Constants.ENABLE_MULTI_DELETE, false); + Path testPath = path(getMethodName()); + try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) { + BulkDelete bulkDelete = fs.createBulkDelete(testPath); + Assertions.assertThat(bulkDelete.pageSize()) + .describedAs("Page size should be 1 when multi-object delete is disabled") + .isEqualTo(1); } - - @Test - public void testBulkDeleteParentDirectoryWithDirectories() throws Exception { - List paths = new ArrayList<>(); - Path dirPath = new Path(basePath, "dir"); - fs.mkdirs(dirPath); - Path subDir = new Path(dirPath, "subdir"); - fs.mkdirs(subDir); - // adding parent directory to the list of paths. - paths.add(dirPath); - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); - // During the bulk delete operation, the directories are not deleted in S3A. - assertIsDirectory(dirPath); - assertIsDirectory(subDir); + } + + @Override + public void testDeletePathsDirectory() throws Exception { + List paths = new ArrayList<>(); + Path dirPath = new Path(basePath, "dir"); + fs.mkdirs(dirPath); + paths.add(dirPath); + Path filePath = new Path(dirPath, "file"); + touch(fs, filePath); + if (enableMultiObjectDelete) { + // Adding more paths only if multi-object delete is enabled. + paths.add(filePath); } - - public void testBulkDeleteParentDirectoryWithFiles() throws Exception { - List paths = new ArrayList<>(); - Path dirPath = new Path(basePath, "dir"); - fs.mkdirs(dirPath); - Path file = new Path(dirPath, "file"); - touch(fs, file); - // adding parent directory to the list of paths. - paths.add(dirPath); - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); - // During the bulk delete operation, - // the directories are not deleted in S3A. - assertIsDirectory(dirPath); + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + // During the bulk delete operation, the directories are not deleted in S3A. + assertIsDirectory(dirPath); + } + + @Test + public void testBulkDeleteParentDirectoryWithDirectories() throws Exception { + List paths = new ArrayList<>(); + Path dirPath = new Path(basePath, "dir"); + fs.mkdirs(dirPath); + Path subDir = new Path(dirPath, "subdir"); + fs.mkdirs(subDir); + // adding parent directory to the list of paths. + paths.add(dirPath); + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + // During the bulk delete operation, the directories are not deleted in S3A. + assertIsDirectory(dirPath); + assertIsDirectory(subDir); + } + + public void testBulkDeleteParentDirectoryWithFiles() throws Exception { + List paths = new ArrayList<>(); + Path dirPath = new Path(basePath, "dir"); + fs.mkdirs(dirPath); + Path file = new Path(dirPath, "file"); + touch(fs, file); + // adding parent directory to the list of paths. + paths.add(dirPath); + assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + // During the bulk delete operation, + // the directories are not deleted in S3A. + assertIsDirectory(dirPath); + } + + + @Test + public void testRateLimiting() throws Exception { + if (!enableMultiObjectDelete) { + skip("Multi-object delete is disabled so hard to trigger rate limiting"); } - - - @Test - public void testRateLimiting() throws Exception { - if (!enableMultiObjectDelete) { - skip("Multi-object delete is disabled so hard to trigger rate limiting"); - } - Configuration conf = getContract().getConf(); - conf.setInt(Constants.S3A_IO_RATE_LIMIT, 5); - Path basePath = path(getMethodName()); - try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) { - createFiles(fs, basePath, 1, 20, 0); - FileStatus[] fileStatuses = fs.listStatus(basePath); - List paths = Arrays.stream(fileStatuses) - .map(FileStatus::getPath) - .collect(toList()); - pageSizePreconditionForTest(paths.size()); - BulkDelete bulkDelete = fs.createBulkDelete(basePath); - bulkDelete.bulkDelete(paths); - MeanStatistic meanStatisticBefore = lookupMeanStatistic(fs.getIOStatistics(), - STORE_IO_RATE_LIMITED_DURATION + SUFFIX_MEAN); - Assertions.assertThat(meanStatisticBefore.mean()) - .describedAs("Rate limiting should not have happened during first delete call") - .isEqualTo(0.0); - bulkDelete.bulkDelete(paths); - bulkDelete.bulkDelete(paths); - bulkDelete.bulkDelete(paths); - MeanStatistic meanStatisticAfter = lookupMeanStatistic(fs.getIOStatistics(), - STORE_IO_RATE_LIMITED_DURATION + SUFFIX_MEAN); - Assertions.assertThat(meanStatisticAfter.mean()) - .describedAs("Rate limiting should have happened during multiple delete calls") - .isGreaterThan(0.0); - } + Configuration conf = getContract().getConf(); + conf.setInt(Constants.S3A_IO_RATE_LIMIT, 5); + Path basePath = path(getMethodName()); + try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) { + createFiles(fs, basePath, 1, 20, 0); + FileStatus[] fileStatuses = fs.listStatus(basePath); + List paths = Arrays.stream(fileStatuses) + .map(FileStatus::getPath) + .collect(toList()); + pageSizePreconditionForTest(paths.size()); + BulkDelete bulkDelete = fs.createBulkDelete(basePath); + bulkDelete.bulkDelete(paths); + MeanStatistic meanStatisticBefore = lookupMeanStatistic(fs.getIOStatistics(), + STORE_IO_RATE_LIMITED_DURATION + SUFFIX_MEAN); + Assertions.assertThat(meanStatisticBefore.mean()) + .describedAs("Rate limiting should not have happened during first delete call") + .isEqualTo(0.0); + bulkDelete.bulkDelete(paths); + bulkDelete.bulkDelete(paths); + bulkDelete.bulkDelete(paths); + MeanStatistic meanStatisticAfter = lookupMeanStatistic(fs.getIOStatistics(), + STORE_IO_RATE_LIMITED_DURATION + SUFFIX_MEAN); + Assertions.assertThat(meanStatisticAfter.mean()) + .describedAs("Rate limiting should have happened during multiple delete calls") + .isGreaterThan(0.0); } + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsContractBulkDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsContractBulkDelete.java index 3609176786a9b..7ec11abe733b7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsContractBulkDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsContractBulkDelete.java @@ -24,27 +24,27 @@ public class ITestAbfsContractBulkDelete extends AbstractContractBulkDeleteTest { - private final boolean isSecure; - private final ABFSContractTestBinding binding; - - public ITestAbfsContractBulkDelete() throws Exception { - binding = new ABFSContractTestBinding(); - this.isSecure = binding.isSecureMode(); - } - - @Override - public void setup() throws Exception { - binding.setup(); - super.setup(); - } - - @Override - protected Configuration createConfiguration() { - return binding.getRawConfiguration(); - } - - @Override - protected AbstractFSContract createContract(Configuration conf) { - return new AbfsFileSystemContract(conf, isSecure); - } + private final boolean isSecure; + private final ABFSContractTestBinding binding; + + public ITestAbfsContractBulkDelete() throws Exception { + binding = new ABFSContractTestBinding(); + this.isSecure = binding.isSecureMode(); + } + + @Override + public void setup() throws Exception { + binding.setup(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return binding.getRawConfiguration(); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new AbfsFileSystemContract(conf, isSecure); + } } From 01b2ae86f5863476fefcc32d5e0c41346a9a1270 Mon Sep 17 00:00:00 2001 From: Mukund Thakur Date: Mon, 29 Apr 2024 15:54:23 -0500 Subject: [PATCH 15/19] some more fixes --- .../src/main/java/org/apache/hadoop/fs/BulkDelete.java | 3 +-- .../org/apache/hadoop/fs/DefaultBulkDeleteOperation.java | 4 ++-- .../src/site/markdown/filesystem/bulkdelete.md | 8 ++++---- .../main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java | 2 ++ .../src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java | 1 + .../fs/contract/s3a/ITestS3AContractBulkDelete.java | 7 +++++-- .../org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java | 2 +- 7 files changed, 16 insertions(+), 11 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java index a6d473c7c04e0..ab5f73b5624ff 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDelete.java @@ -45,7 +45,6 @@ * are attempts at parallel submissions across multiple threads. * @see HADOOP-16823. * Large DeleteObject requests are their own Thundering Herd - *

*/ @InterfaceAudience.Public @InterfaceStability.Unstable @@ -60,6 +59,7 @@ public interface BulkDelete extends IOStatisticsSource, Closeable { /** * Base path of a bulk delete operation. * All paths submitted in {@link #bulkDelete(Collection)} must be under this path. + * @return base path of a bulk delete operation. */ Path basePath(); @@ -84,7 +84,6 @@ public interface BulkDelete extends IOStatisticsSource, Closeable { * @throws IOException IO problems including networking, authentication and more. * @throws IllegalArgumentException if a path argument is invalid. */ - List> bulkDelete(Collection paths) throws IOException, IllegalArgumentException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java index 40d38998b5a81..ed30af8b91a2c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java @@ -89,12 +89,12 @@ public List> bulkDelete(Collection paths) } catch (FileNotFoundException e) { // Ignore FNFE and don't add to the result list. LOG.debug("Couldn't delete {} - does not exist: {}", pathToDelete, e.toString()); - } catch (Exception e) { + } catch (IOException e) { LOG.debug("Couldn't delete {} - exception occurred: {}", pathToDelete, e.toString()); result.add(Tuples.pair(pathToDelete, e.toString())); } } - } catch (Exception ex) { + } catch (IOException ex) { LOG.debug("Couldn't delete {} - exception occurred: {}", pathToDelete, ex.toString()); result.add(Tuples.pair(pathToDelete, ex.toString())); } diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md index f3913b9f7e9b0..a5bc1f4ce243e 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md @@ -90,10 +90,10 @@ No other restrictions are placed upon the outcome. ### Availability The `BulkDeleteSource` interface is exported by `FileSystem` and `FileContext` storage clients -which is available for all FS via `org.apache.hadoop.fs.DefalutBulkDeleteSource` -Some FS MAY still decide to not support the API by overwriting the `createBulkDelete()` method -with an UnsupportedOperationException. While doing so they must also declare the path -capability `fs.capability.bulk.delete` as false. +which is available for all FS via `org.apache.hadoop.fs.DefalutBulkDeleteSource`. For the +ICEBERG integration to work seamlessly, all FS which supports delete() MUST leave the +default implementation in place by NEVER overwriting the `createBulkDelete()` method +with an UnsupportedOperationException. Use the `PathCapabilities` probe `fs.capability.bulk.delete`. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index b7392089141e9..00139da9cc359 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -5756,6 +5756,8 @@ public BulkDelete createBulkDelete(final Path path) /** * Create the callbacks for the bulk delete operation. + * @param path path to delete. + * @param pageSize page size. * @param span span for operations. * @return an instance of the Bulk Delete callbacks. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java index 68abe40bd63bb..68eacc35b1887 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AStore.java @@ -96,6 +96,7 @@ public interface S3AStore extends IOStatisticsSource { * @throws MultiObjectDeleteException one or more of the keys could not * be deleted. * @throws SdkException amazon-layer failure. + * @throws IOException IO problems. */ @Retries.RetryRaw Map.Entry deleteObjects(DeleteObjectsRequest deleteRequest) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java index 7c7528fa57ae4..6d69a3c64c4f1 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java @@ -86,8 +86,11 @@ protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); S3ATestUtils.disableFilesystemCaching(conf); conf = propagateBucketOptions(conf, getTestBucketName(conf)); - skipIfNotEnabled(conf, Constants.ENABLE_MULTI_DELETE, - "Bulk delete is explicitly disabled for this bucket"); + if (enableMultiObjectDelete) { + // if multi-object delete is disabled, skip the test. + skipIfNotEnabled(conf, Constants.ENABLE_MULTI_DELETE, + "Bulk delete is explicitly disabled for this bucket"); + } S3ATestUtils.removeBaseAndBucketOverrides(conf, Constants.BULK_DELETE_PAGE_SIZE); conf.setInt(Constants.BULK_DELETE_PAGE_SIZE, DELETE_PAGE_SIZE); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java index 46f706ed8fab5..5740649a1be0b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java @@ -770,7 +770,7 @@ private void executeBulkDeleteOnSomeReadOnlyFiles(Configuration assumedRoleConfi roleFS = (S3AFileSystem) destDir.getFileSystem(assumedRoleConfig); S3AFileSystem fs = getFileSystem(); if (WrappedIO.bulkDeletePageSize(fs, destDir) == 1) { - String msg = "Skipping as this test requires more than one paths to be deleted in bulk"; + String msg = "Skipping as this test requires more than one path to be deleted in bulk"; LOG.debug(msg); skip(msg); } From 89e87d41a945a0a5d0a3b2c6d66370f5625cd793 Mon Sep 17 00:00:00 2001 From: Mukund Thakur Date: Tue, 30 Apr 2024 14:57:19 -0500 Subject: [PATCH 16/19] review comments --- .../DefaultBulkDeleteOperation.java | 30 ++++++------------- .../DefaultBulkDeleteSource.java} | 11 +++++-- .../apache/hadoop/io/wrappedio/WrappedIO.java | 5 ++-- .../site/markdown/filesystem/bulkdelete.md | 10 +++---- .../AbstractContractBulkDeleteTest.java | 8 +++-- .../hadoop/fs/s3a/impl/S3AStoreBuilder.java | 2 +- .../hadoop/fs/s3a/impl/S3AStoreImpl.java | 2 +- .../markdown/tools/hadoop-aws/performance.md | 5 ++-- .../src/test/resources/log4j.properties | 2 +- 9 files changed, 36 insertions(+), 39 deletions(-) rename hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/{ => impl}/DefaultBulkDeleteOperation.java (72%) rename hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/{DefalutBulkDeleteSource.java => impl/DefaultBulkDeleteSource.java} (79%) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/DefaultBulkDeleteOperation.java similarity index 72% rename from hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/DefaultBulkDeleteOperation.java index ed30af8b91a2c..56f6a4622f877 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefaultBulkDeleteOperation.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/DefaultBulkDeleteOperation.java @@ -15,9 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.fs; +package org.apache.hadoop.fs.impl; -import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -27,6 +26,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.BulkDelete; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.util.functional.Tuples; import static java.util.Objects.requireNonNull; @@ -65,7 +67,9 @@ public Path basePath() { } /** - * {@inheritDoc} + * {@inheritDoc}. + * The default impl just calls {@code FileSystem.delete(path, false)} + * on the single path in the list. */ @Override public List> bulkDelete(Collection paths) @@ -77,25 +81,9 @@ public List> bulkDelete(Collection paths) // path in the collection. Path pathToDelete = paths.iterator().next(); try { - boolean deleted = fs.delete(pathToDelete, false); - if (deleted) { - return result; - } else { - try { - FileStatus fileStatus = fs.getFileStatus(pathToDelete); - if (fileStatus.isDirectory()) { - result.add(Tuples.pair(pathToDelete, "Path is a directory")); - } - } catch (FileNotFoundException e) { - // Ignore FNFE and don't add to the result list. - LOG.debug("Couldn't delete {} - does not exist: {}", pathToDelete, e.toString()); - } catch (IOException e) { - LOG.debug("Couldn't delete {} - exception occurred: {}", pathToDelete, e.toString()); - result.add(Tuples.pair(pathToDelete, e.toString())); - } - } + fs.delete(pathToDelete, false); } catch (IOException ex) { - LOG.debug("Couldn't delete {} - exception occurred: {}", pathToDelete, ex.toString()); + LOG.debug("Couldn't delete {} - exception occurred: {}", pathToDelete, ex); result.add(Tuples.pair(pathToDelete, ex.toString())); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefalutBulkDeleteSource.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/DefaultBulkDeleteSource.java similarity index 79% rename from hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefalutBulkDeleteSource.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/DefaultBulkDeleteSource.java index 5b769b53494f6..d97e3f6ae568a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DefalutBulkDeleteSource.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/DefaultBulkDeleteSource.java @@ -15,18 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.fs; +package org.apache.hadoop.fs.impl; import java.io.IOException; +import org.apache.hadoop.fs.BulkDelete; +import org.apache.hadoop.fs.BulkDeleteSource; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + /** * Default implementation of {@link BulkDeleteSource}. */ -public class DefalutBulkDeleteSource implements BulkDeleteSource { +public class DefaultBulkDeleteSource implements BulkDeleteSource { private final FileSystem fs; - public DefalutBulkDeleteSource(FileSystem fs) { + public DefaultBulkDeleteSource(FileSystem fs) { this.fs = fs; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java index e75fecb903f3b..d196912140d9b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java @@ -27,8 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.BulkDelete; import org.apache.hadoop.fs.BulkDeleteSource; -import org.apache.hadoop.fs.DefalutBulkDeleteSource; -import org.apache.hadoop.fs.DefaultBulkDeleteOperation; +import org.apache.hadoop.fs.impl.DefaultBulkDeleteSource; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -83,7 +82,7 @@ private static BulkDeleteSource toBulkDeleteSource(final FileSystem fs) { if (fs instanceof BulkDeleteSource) { return (BulkDeleteSource) fs; } - return new DefalutBulkDeleteSource(fs); + return new DefaultBulkDeleteSource(fs); } /** diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md index a5bc1f4ce243e..044fdf28192b2 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md @@ -90,10 +90,10 @@ No other restrictions are placed upon the outcome. ### Availability The `BulkDeleteSource` interface is exported by `FileSystem` and `FileContext` storage clients -which is available for all FS via `org.apache.hadoop.fs.DefalutBulkDeleteSource`. For the -ICEBERG integration to work seamlessly, all FS which supports delete() MUST leave the -default implementation in place by NEVER overwriting the `createBulkDelete()` method -with an UnsupportedOperationException. +which is available for all FS via `org.apache.hadoop.fs.impl.DefaultBulkDeleteSource`. For +integration in applications like Apache Iceberg to work seamlessly, all implementations +of this interface MUST NOT reject the request but instead return a BulkDelete instance +of size >= 1. Use the `PathCapabilities` probe `fs.capability.bulk.delete`. @@ -126,7 +126,7 @@ through reflection. #### Default Implementation The default implementation which will be used by all implementation of `FileSystem` of the -`BulkDelete` interface is `org.apache.hadoop.fs.DefaultBulkDeleteOperation` which fixes the page +`BulkDelete` interface is `org.apache.hadoop.fs.impl.DefaultBulkDeleteOperation` which fixes the page size to be 1 and calls `FileSystem.delete(path, false)` on the single path in the list. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java index a0927ecd2638e..0368e941b40a8 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java @@ -137,22 +137,26 @@ public void validatePathCapabilityDeclared() throws Exception { .isTrue(); } + /** + * This test should fail as path is not under the base path. + */ @Test public void testDeletePathsNotUnderBase() throws Exception { List paths = new ArrayList<>(); Path pathNotUnderBase = path("not-under-base"); paths.add(pathNotUnderBase); - // Should fail as path is not under the base path. intercept(IllegalArgumentException.class, () -> bulkDelete(getFileSystem(), basePath, paths)); } + /** + * This test should fail as path is not absolute. + */ @Test public void testDeletePathsNotAbsolute() throws Exception { List paths = new ArrayList<>(); Path pathNotAbsolute = new Path("not-absolute"); paths.add(pathNotAbsolute); - // Should fail as path is not absolute. intercept(IllegalArgumentException.class, () -> bulkDelete(getFileSystem(), basePath, paths)); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java index 44bac68e71b8f..c1a6fcffab487 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreBuilder.java @@ -110,4 +110,4 @@ public S3AStore build() { return new S3AStoreImpl(storeContextFactory, s3Client, durationTrackerFactory, instrumentation, statisticsContext, storageStatistics, readRateLimiter, writeRateLimiter, auditSpanSource); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java index 8df5eb90e6f4c..6bfe42767d8b1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AStoreImpl.java @@ -154,7 +154,7 @@ public Duration acquireReadCapacity(final int capacity) { } /** - * Create the store context. + * Create a new store context. * @return a new store context. */ private StoreContext createStoreContext() { diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md index 45445f51b150b..954823f2172ea 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md @@ -153,11 +153,12 @@ However, the repeated request will still include the same number of objects and This can lead to a pathological situation where the repeated requests will never be satisfied because the request itself is sufficient to overload the store. -See [HADOOP-16823.Large DeleteObject requests are their own Thundering Herd](https://issues.apache.org/jira/browse/HADOOP-16823) +See [HADOOP-16823.Large DeleteObject requests are their own Thundering Herd] +(https://issues.apache.org/jira/browse/HADOOP-16823) for an example of where this did actually surface in production. This is why the default page size of S3A clients is 250 paths, not the store limit of 1000 entries. -It is also why the S3A delete/rename Operations do not attempt to do massive parallel deletions, +It is also why the S3A delete/rename operations do not attempt to do massive parallel deletions, Instead bulk delete requests are queued for a single blocking thread to issue. Consider a similar design. diff --git a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties index 14bca5a85a84e..64562ecdcf047 100644 --- a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties @@ -26,7 +26,7 @@ log4j.logger.org.apache.hadoop.fs.azure.AzureFileSystemThreadPoolExecutor=DEBUG log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG log4j.logger.org.apache.hadoop.fs.azurebfs.contracts.services.TracingService=TRACE log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsClient=DEBUG -log4j.logger.org.apache.hadoop.fs.DefaultBulkDeleteOperation=DEBUG +log4j.logger.org.apache.hadoop.fs.impl.DefaultBulkDeleteOperation=DEBUG # after here: turn off log messages from other parts of the system # which only clutter test reports. From 058d0995a2cb15cf38ddd2810154fe1e8c23045b Mon Sep 17 00:00:00 2001 From: Mukund Thakur Date: Wed, 1 May 2024 12:13:49 -0500 Subject: [PATCH 17/19] BulkDeleteSource to be implemented by base FileSystem --- .../java/org/apache/hadoop/fs/FileSystem.java | 17 +++++++- .../fs/impl/DefaultBulkDeleteSource.java | 43 ------------------- .../apache/hadoop/io/wrappedio/WrappedIO.java | 33 +++----------- .../AbstractContractBulkDeleteTest.java | 22 +++++++++- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 2 +- 5 files changed, 42 insertions(+), 75 deletions(-) delete mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/DefaultBulkDeleteSource.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java index 86e7c1b01d5b3..9d16ccc9c881b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java @@ -56,6 +56,7 @@ import org.apache.hadoop.fs.Options.HandleOpt; import org.apache.hadoop.fs.Options.Rename; import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl; +import org.apache.hadoop.fs.impl.DefaultBulkDeleteOperation; import org.apache.hadoop.fs.impl.FutureDataInputStreamBuilderImpl; import org.apache.hadoop.fs.impl.OpenFileParameters; import org.apache.hadoop.fs.permission.AclEntry; @@ -169,7 +170,8 @@ @InterfaceAudience.Public @InterfaceStability.Stable public abstract class FileSystem extends Configured - implements Closeable, DelegationTokenIssuer, PathCapabilities { + implements Closeable, DelegationTokenIssuer, + PathCapabilities, BulkDeleteSource { public static final String FS_DEFAULT_NAME_KEY = CommonConfigurationKeys.FS_DEFAULT_NAME_KEY; public static final String DEFAULT_FS = @@ -4980,4 +4982,17 @@ public MultipartUploaderBuilder createMultipartUploader(Path basePath) methodNotSupported(); return null; } + + /** + * Create a default bulk delete operation to be used for any FileSystem. + * @param path base path for the operation. + * @return an instance of the bulk delete. + * @throws IllegalArgumentException any argument is invalid. + * @throws IOException if there is an IO problem. + */ + @Override + public BulkDelete createBulkDelete(Path path) + throws IllegalArgumentException, IOException { + return new DefaultBulkDeleteOperation(path, this); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/DefaultBulkDeleteSource.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/DefaultBulkDeleteSource.java deleted file mode 100644 index d97e3f6ae568a..0000000000000 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/DefaultBulkDeleteSource.java +++ /dev/null @@ -1,43 +0,0 @@ -/** - * 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.hadoop.fs.impl; - -import java.io.IOException; - -import org.apache.hadoop.fs.BulkDelete; -import org.apache.hadoop.fs.BulkDeleteSource; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; - -/** - * Default implementation of {@link BulkDeleteSource}. - */ -public class DefaultBulkDeleteSource implements BulkDeleteSource { - - private final FileSystem fs; - - public DefaultBulkDeleteSource(FileSystem fs) { - this.fs = fs; - } - - @Override - public BulkDelete createBulkDelete(Path base) - throws UnsupportedOperationException, IllegalArgumentException, IOException { - return new DefaultBulkDeleteOperation(base, fs); - } -} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java index d196912140d9b..f3de26d5010c4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java @@ -26,8 +26,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.BulkDelete; -import org.apache.hadoop.fs.BulkDeleteSource; -import org.apache.hadoop.fs.impl.DefaultBulkDeleteSource; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -47,17 +45,6 @@ public final class WrappedIO { private WrappedIO() { } - /** - * Is bulk delete available on a filesystem? - * @param fs filesystem - * @param path path to delete under. - * @return true if bulk delete is available. - */ - public static boolean isBulkDeleteAvailable(FileSystem fs, Path path) { - return (fs instanceof BulkDeleteSource) - && ((BulkDeleteSource) fs).isBulkDeleteAvailable(path); - } - /** * Get the maximum number of objects/files to delete in a single request. * @param fs filesystem @@ -68,23 +55,11 @@ public static boolean isBulkDeleteAvailable(FileSystem fs, Path path) { * @throws IOException problems resolving paths */ public static int bulkDeletePageSize(FileSystem fs, Path path) throws IOException { - try (BulkDelete bulk = toBulkDeleteSource(fs).createBulkDelete(path)) { + try (BulkDelete bulk = fs.createBulkDelete(path)) { return bulk.pageSize(); } } - /** - * Convert a filesystem to a bulk delete source. - * @param fs filesystem - * @return cast fs or a default implementation. - */ - private static BulkDeleteSource toBulkDeleteSource(final FileSystem fs) { - if (fs instanceof BulkDeleteSource) { - return (BulkDeleteSource) fs; - } - return new DefaultBulkDeleteSource(fs); - } - /** * Delete a list of files/objects. *

    @@ -107,9 +82,11 @@ private static BulkDeleteSource toBulkDeleteSource(final FileSystem fs) { * @throws IOException IO problems including networking, authentication and more. * @throws IllegalArgumentException if a path argument is invalid. */ - public static List> bulkDelete(FileSystem fs, Path base, Collection paths) + public static List> bulkDelete(FileSystem fs, + Path base, + Collection paths) throws IOException { - try (BulkDelete bulk = toBulkDeleteSource(fs).createBulkDelete(base)) { + try (BulkDelete bulk = fs.createBulkDelete(base)) { return bulk.bulkDelete(paths); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java index 0368e941b40a8..17a8516c244d1 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.contract; +import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -114,6 +115,15 @@ public void testPathsSizeLessThanPageSizePrecondition() throws Exception { @Test public void testBulkDeleteSuccessful() throws Exception { + runBulkDelete(false); + } + + @Test + public void testBulkDeleteSuccessfulUsingDirectFS() throws Exception { + runBulkDelete(true); + } + + private void runBulkDelete(boolean useDirectFS) throws IOException { List listOfPaths = createListOfPaths(pageSize, basePath); for (Path path : listOfPaths) { touch(fs, path); @@ -122,14 +132,22 @@ public void testBulkDeleteSuccessful() throws Exception { Assertions.assertThat(fileStatuses) .describedAs("File count after create") .hasSize(pageSize); - assertSuccessfulBulkDelete( - bulkDelete(getFileSystem(), basePath, listOfPaths)); + if (useDirectFS) { + assertSuccessfulBulkDelete( + fs.createBulkDelete(basePath).bulkDelete(listOfPaths)); + } else { + // Using WrappedIO to call bulk delete. + assertSuccessfulBulkDelete( + bulkDelete(getFileSystem(), basePath, listOfPaths)); + } + FileStatus[] fileStatusesAfterDelete = fs.listStatus(basePath); Assertions.assertThat(fileStatusesAfterDelete) .describedAs("File statuses should be empty after delete") .isEmpty(); } + @Test public void validatePathCapabilityDeclared() throws Exception { Assertions.assertThat(fs.hasPathCapability(basePath, CommonPathCapabilities.BULK_DELETE)) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 00139da9cc359..bbf2e2740f464 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -288,7 +288,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, AWSPolicyProvider, DelegationTokenProvider, IOStatisticsSource, AuditSpanSource, ActiveThreadSpanSource, - BulkDeleteSource, StoreContextFactory { + StoreContextFactory { /** * Default blocksize as used in blocksize and FS status queries. From e37d88f764665c8530097bbed890a5935a5fd1f0 Mon Sep 17 00:00:00 2001 From: Mukund Thakur Date: Thu, 2 May 2024 13:22:47 -0500 Subject: [PATCH 18/19] Removing default impl from BulkDeleteSource --- .../org/apache/hadoop/fs/BulkDeleteSource.java | 15 ++------------- .../src/site/markdown/filesystem/bulkdelete.md | 2 +- .../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 1 - .../fs/s3a/scale/AbstractSTestS3AHugeFiles.java | 2 ++ 4 files changed, 5 insertions(+), 15 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java index 53e45408638ac..cad24babb344a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BulkDeleteSource.java @@ -47,18 +47,7 @@ public interface BulkDeleteSource { * @throws IllegalArgumentException path not valid. * @throws IOException problems resolving paths */ - default BulkDelete createBulkDelete(Path path) - throws UnsupportedOperationException, IllegalArgumentException, IOException { - throw new UnsupportedOperationException("Bulk delete not supported"); - } - - /** - * Is bulk delete available on this path for this source? - * @param path path to delete under. - * @return true if bulk delete is available. - */ - default boolean isBulkDeleteAvailable(Path path) { - return true; - } + BulkDelete createBulkDelete(Path path) + throws UnsupportedOperationException, IllegalArgumentException, IOException; } diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md index 044fdf28192b2..dc826198a28dd 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md @@ -45,7 +45,7 @@ it supports the API. @InterfaceAudience.Public @InterfaceStability.Unstable public interface BulkDeleteSource { - default BulkDelete createBulkDelete(Path path) + BulkDelete createBulkDelete(Path path) throws UnsupportedOperationException, IllegalArgumentException, IOException; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index bbf2e2740f464..d04ca70a68ded 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -103,7 +103,6 @@ import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BulkDelete; -import org.apache.hadoop.fs.BulkDeleteSource; import org.apache.hadoop.fs.CommonPathCapabilities; import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.CreateFlag; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java index 79e5a93371a8d..dc81077257bcc 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java @@ -831,4 +831,6 @@ protected void delete(Path path, boolean recursive) throws IOException { timer.end("time to delete %s", path); } + + } From b80a8e6b25a2269bf7e20e5f65737fa07b8ae86e Mon Sep 17 00:00:00 2001 From: Mukund Thakur Date: Wed, 8 May 2024 12:53:14 -0500 Subject: [PATCH 19/19] doc fixes and reflection friendly api names --- hadoop-common-project/hadoop-common/pom.xml | 6 ---- .../java/org/apache/hadoop/fs/FileSystem.java | 3 +- .../apache/hadoop/io/wrappedio/WrappedIO.java | 8 ++--- .../site/markdown/filesystem/bulkdelete.md | 5 ++- .../AbstractContractBulkDeleteTest.java | 34 +++++++++---------- .../s3a/ITestS3AContractBulkDelete.java | 8 ++--- .../hadoop/fs/s3a/auth/ITestAssumeRole.java | 8 ++--- 7 files changed, 35 insertions(+), 37 deletions(-) diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml index 3a9189c4d914d..a7dcbb24a9b40 100644 --- a/hadoop-common-project/hadoop-common/pom.xml +++ b/hadoop-common-project/hadoop-common/pom.xml @@ -429,12 +429,6 @@ lz4-java provided - - org.testng - testng - RELEASE - test - diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java index 9d16ccc9c881b..2155e17328a66 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java @@ -4984,7 +4984,8 @@ public MultipartUploaderBuilder createMultipartUploader(Path basePath) } /** - * Create a default bulk delete operation to be used for any FileSystem. + * Create a bulk delete operation. + * The default implementation returns an instance of {@link DefaultBulkDeleteOperation}. * @param path base path for the operation. * @return an instance of the bulk delete. * @throws IllegalArgumentException any argument is invalid. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java index f3de26d5010c4..696055895a19b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/wrappedio/WrappedIO.java @@ -54,7 +54,7 @@ private WrappedIO() { * @throws IllegalArgumentException path not valid. * @throws IOException problems resolving paths */ - public static int bulkDeletePageSize(FileSystem fs, Path path) throws IOException { + public static int bulkDelete_PageSize(FileSystem fs, Path path) throws IOException { try (BulkDelete bulk = fs.createBulkDelete(path)) { return bulk.pageSize(); } @@ -82,9 +82,9 @@ public static int bulkDeletePageSize(FileSystem fs, Path path) throws IOExceptio * @throws IOException IO problems including networking, authentication and more. * @throws IllegalArgumentException if a path argument is invalid. */ - public static List> bulkDelete(FileSystem fs, - Path base, - Collection paths) + public static List> bulkDelete_delete(FileSystem fs, + Path base, + Collection paths) throws IOException { try (BulkDelete bulk = fs.createBulkDelete(base)) { return bulk.bulkDelete(paths); diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md index dc826198a28dd..de0e4e893ba2e 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/bulkdelete.md @@ -39,7 +39,10 @@ restrictions do not state what the outcome will be when executed on other stores ### Interface `org.apache.hadoop.fs.BulkDeleteSource` The interface `BulkDeleteSource` is offered by a FileSystem/FileContext class if -it supports the API. +it supports the API. The default implementation is implemented in base FileSystem +class that returns an instance of `org.apache.hadoop.fs.impl.DefaultBulkDeleteOperation`. +The default implementation details are provided in below sections. + ```java @InterfaceAudience.Public diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java index 17a8516c244d1..9ebf9923f39c2 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractBulkDeleteTest.java @@ -37,7 +37,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; -import static org.apache.hadoop.io.wrappedio.WrappedIO.bulkDelete; +import static org.apache.hadoop.io.wrappedio.WrappedIO.bulkDelete_delete; import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** @@ -69,7 +69,7 @@ public abstract class AbstractContractBulkDeleteTest extends AbstractFSContractT public void setUp() throws Exception { fs = getFileSystem(); basePath = path(getClass().getName()); - pageSize = WrappedIO.bulkDeletePageSize(getFileSystem(), basePath); + pageSize = WrappedIO.bulkDelete_PageSize(getFileSystem(), basePath); fs.mkdirs(basePath); } @@ -96,21 +96,21 @@ public void validatePageSize() throws Exception { public void testPathsSizeEqualsPageSizePrecondition() throws Exception { List listOfPaths = createListOfPaths(pageSize, basePath); // Bulk delete call should pass with no exception. - bulkDelete(getFileSystem(), basePath, listOfPaths); + bulkDelete_delete(getFileSystem(), basePath, listOfPaths); } @Test public void testPathsSizeGreaterThanPageSizePrecondition() throws Exception { List listOfPaths = createListOfPaths(pageSize + 1, basePath); intercept(IllegalArgumentException.class, - () -> bulkDelete(getFileSystem(), basePath, listOfPaths)); + () -> bulkDelete_delete(getFileSystem(), basePath, listOfPaths)); } @Test public void testPathsSizeLessThanPageSizePrecondition() throws Exception { List listOfPaths = createListOfPaths(pageSize - 1, basePath); // Bulk delete call should pass with no exception. - bulkDelete(getFileSystem(), basePath, listOfPaths); + bulkDelete_delete(getFileSystem(), basePath, listOfPaths); } @Test @@ -138,7 +138,7 @@ private void runBulkDelete(boolean useDirectFS) throws IOException { } else { // Using WrappedIO to call bulk delete. assertSuccessfulBulkDelete( - bulkDelete(getFileSystem(), basePath, listOfPaths)); + bulkDelete_delete(getFileSystem(), basePath, listOfPaths)); } FileStatus[] fileStatusesAfterDelete = fs.listStatus(basePath); @@ -164,7 +164,7 @@ public void testDeletePathsNotUnderBase() throws Exception { Path pathNotUnderBase = path("not-under-base"); paths.add(pathNotUnderBase); intercept(IllegalArgumentException.class, - () -> bulkDelete(getFileSystem(), basePath, paths)); + () -> bulkDelete_delete(getFileSystem(), basePath, paths)); } /** @@ -176,7 +176,7 @@ public void testDeletePathsNotAbsolute() throws Exception { Path pathNotAbsolute = new Path("not-absolute"); paths.add(pathNotAbsolute); intercept(IllegalArgumentException.class, - () -> bulkDelete(getFileSystem(), basePath, paths)); + () -> bulkDelete_delete(getFileSystem(), basePath, paths)); } @Test @@ -185,7 +185,7 @@ public void testDeletePathsNotExists() throws Exception { Path pathNotExists = new Path(basePath, "not-exists"); paths.add(pathNotExists); // bulk delete call doesn't verify if a path exist or not before deleting. - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + assertSuccessfulBulkDelete(bulkDelete_delete(getFileSystem(), basePath, paths)); } @Test @@ -199,7 +199,7 @@ public void testDeletePathsDirectory() throws Exception { fs.mkdirs(dirPath); touch(fs, filePath); // Outcome is undefined. But call shouldn't fail. - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + assertSuccessfulBulkDelete(bulkDelete_delete(getFileSystem(), basePath, paths)); } @Test @@ -211,7 +211,7 @@ public void testBulkDeleteParentDirectoryWithDirectories() throws Exception { fs.mkdirs(subDir); // adding parent directory to the list of paths. paths.add(dirPath); - List> entries = bulkDelete(getFileSystem(), basePath, paths); + List> entries = bulkDelete_delete(getFileSystem(), basePath, paths); Assertions.assertThat(entries) .describedAs("Parent non empty directory should not be deleted") .hasSize(1); @@ -228,7 +228,7 @@ public void testBulkDeleteParentDirectoryWithFiles() throws Exception { touch(fs, file); // adding parent directory to the list of paths. paths.add(dirPath); - List> entries = bulkDelete(getFileSystem(), basePath, paths); + List> entries = bulkDelete_delete(getFileSystem(), basePath, paths); Assertions.assertThat(entries) .describedAs("Parent non empty directory should not be deleted") .hasSize(1); @@ -244,14 +244,14 @@ public void testDeleteEmptyDirectory() throws Exception { fs.mkdirs(emptyDirPath); paths.add(emptyDirPath); // Should pass as empty directory. - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + assertSuccessfulBulkDelete(bulkDelete_delete(getFileSystem(), basePath, paths)); } @Test public void testDeleteEmptyList() throws Exception { List paths = new ArrayList<>(); // Empty list should pass. - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + assertSuccessfulBulkDelete(bulkDelete_delete(getFileSystem(), basePath, paths)); } @Test @@ -265,7 +265,7 @@ public void testDeleteSamePathsMoreThanOnce() throws Exception { pageSizePreconditionForTest(paths.size()); touch(fs, path); touch(fs, another); - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + assertSuccessfulBulkDelete(bulkDelete_delete(getFileSystem(), basePath, paths)); } /** @@ -291,7 +291,7 @@ public void testDeepDirectoryFilesDelete() throws Exception { Path file1 = new Path(dir3, "file1"); touch(fs, file1); paths.add(file1); - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + assertSuccessfulBulkDelete(bulkDelete_delete(getFileSystem(), basePath, paths)); } @@ -306,7 +306,7 @@ public void testChildPaths() throws Exception { paths.add(filePath); pageSizePreconditionForTest(paths.size()); // Should pass as both paths are under the base path. - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + assertSuccessfulBulkDelete(bulkDelete_delete(getFileSystem(), basePath, paths)); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java index 6d69a3c64c4f1..71c3a30359e10 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractBulkDelete.java @@ -48,7 +48,7 @@ import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupMeanStatistic; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.STORE_IO_RATE_LIMITED_DURATION; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MEAN; -import static org.apache.hadoop.io.wrappedio.WrappedIO.bulkDelete; +import static org.apache.hadoop.io.wrappedio.WrappedIO.bulkDelete_delete; import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** @@ -160,7 +160,7 @@ public void testDeletePathsDirectory() throws Exception { // Adding more paths only if multi-object delete is enabled. paths.add(filePath); } - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + assertSuccessfulBulkDelete(bulkDelete_delete(getFileSystem(), basePath, paths)); // During the bulk delete operation, the directories are not deleted in S3A. assertIsDirectory(dirPath); } @@ -174,7 +174,7 @@ public void testBulkDeleteParentDirectoryWithDirectories() throws Exception { fs.mkdirs(subDir); // adding parent directory to the list of paths. paths.add(dirPath); - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + assertSuccessfulBulkDelete(bulkDelete_delete(getFileSystem(), basePath, paths)); // During the bulk delete operation, the directories are not deleted in S3A. assertIsDirectory(dirPath); assertIsDirectory(subDir); @@ -188,7 +188,7 @@ public void testBulkDeleteParentDirectoryWithFiles() throws Exception { touch(fs, file); // adding parent directory to the list of paths. paths.add(dirPath); - assertSuccessfulBulkDelete(bulkDelete(getFileSystem(), basePath, paths)); + assertSuccessfulBulkDelete(bulkDelete_delete(getFileSystem(), basePath, paths)); // During the bulk delete operation, // the directories are not deleted in S3A. assertIsDirectory(dirPath); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java index 5740649a1be0b..0676dd5b16ed8 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java @@ -731,11 +731,11 @@ private void executeBulkDeleteOnReadOnlyFiles(Configuration assumedRoleConfig) t // the full FS S3AFileSystem fs = getFileSystem(); - WrappedIO.bulkDelete(fs, destDir, new ArrayList<>()); + WrappedIO.bulkDelete_delete(fs, destDir, new ArrayList<>()); bindReadOnlyRolePolicy(assumedRoleConfig, readOnlyDir); roleFS = (S3AFileSystem) destDir.getFileSystem(assumedRoleConfig); - int bulkDeletePageSize = WrappedIO.bulkDeletePageSize(roleFS, destDir); + int bulkDeletePageSize = WrappedIO.bulkDelete_PageSize(roleFS, destDir); int range = bulkDeletePageSize == 1 ? bulkDeletePageSize : 10; touchFiles(fs, readOnlyDir, range); touchFiles(roleFS, destDir, range); @@ -769,12 +769,12 @@ private void executeBulkDeleteOnSomeReadOnlyFiles(Configuration assumedRoleConfi bindReadOnlyRolePolicy(assumedRoleConfig, readOnlyDir); roleFS = (S3AFileSystem) destDir.getFileSystem(assumedRoleConfig); S3AFileSystem fs = getFileSystem(); - if (WrappedIO.bulkDeletePageSize(fs, destDir) == 1) { + if (WrappedIO.bulkDelete_PageSize(fs, destDir) == 1) { String msg = "Skipping as this test requires more than one path to be deleted in bulk"; LOG.debug(msg); skip(msg); } - WrappedIO.bulkDelete(fs, destDir, new ArrayList<>()); + WrappedIO.bulkDelete_delete(fs, destDir, new ArrayList<>()); // creating 5 files in the read only dir. int readOnlyRange = 5; int readWriteRange = 3;