diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 523fc74791e1e..edb3a318993fd 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -399,6 +399,34 @@ public class AbfsConfiguration{ FS_AZURE_ENABLE_PAGINATED_DELETE, DefaultValue = DEFAULT_ENABLE_PAGINATED_DELETE) private boolean isPaginatedDeleteEnabled; + @LongConfigurationValidatorAnnotation(ConfigurationKey = + FS_AZURE_BLOB_COPY_PROGRESS_WAIT_MILLIS, DefaultValue = DEFAULT_AZURE_BLOB_COPY_PROGRESS_WAIT_MILLIS) + private long blobCopyProgressPollWaitMillis; + + @LongConfigurationValidatorAnnotation(ConfigurationKey = + FS_AZURE_BLOB_COPY_MAX_WAIT_MILLIS, DefaultValue = DEFAULT_AZURE_BLOB_COPY_MAX_WAIT_MILLIS) + private long blobCopyProgressMaxWaitMillis; + + @LongConfigurationValidatorAnnotation(ConfigurationKey = + FS_AZURE_BLOB_ATOMIC_RENAME_LEASE_REFRESH_DURATION, DefaultValue = DEFAULT_AZURE_BLOB_ATOMIC_RENAME_LEASE_REFRESH_DURATION) + private long blobAtomicRenameLeaseRefreshDuration; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = + FS_AZURE_PRODUCER_QUEUE_MAX_SIZE, DefaultValue = DEFAULT_FS_AZURE_PRODUCER_QUEUE_MAX_SIZE) + private int producerQueueMaxSize; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = + FS_AZURE_CONSUMER_MAX_LAG, DefaultValue = DEFAULT_FS_AZURE_CONSUMER_MAX_LAG) + private int listingMaxConsumptionLag; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = + FS_AZURE_BLOB_DIR_RENAME_MAX_THREAD, DefaultValue = DEFAULT_FS_AZURE_BLOB_RENAME_THREAD) + private int blobRenameDirConsumptionParallelism; + + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = + FS_AZURE_BLOB_DIR_DELETE_MAX_THREAD, DefaultValue = DEFAULT_FS_AZURE_BLOB_DELETE_THREAD) + private int blobDeleteDirConsumptionParallelism; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES, DefaultValue = DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES) private int maxApacheHttpClientIoExceptionsRetries; @@ -1508,4 +1536,32 @@ public boolean getIsChecksumValidationEnabled() { public void setIsChecksumValidationEnabled(boolean isChecksumValidationEnabled) { this.isChecksumValidationEnabled = isChecksumValidationEnabled; } + + public long getBlobCopyProgressPollWaitMillis() { + return blobCopyProgressPollWaitMillis; + } + + public long getBlobCopyProgressMaxWaitMillis() { + return blobCopyProgressMaxWaitMillis; + } + + public long getAtomicRenameLeaseRefreshDuration() { + return blobAtomicRenameLeaseRefreshDuration; + } + + public int getProducerQueueMaxSize() { + return producerQueueMaxSize; + } + + public int getListingMaxConsumptionLag() { + return listingMaxConsumptionLag; + } + + public int getBlobRenameDirConsumptionParallelism() { + return blobRenameDirConsumptionParallelism; + } + + public int getBlobDeleteDirConsumptionParallelism() { + return blobDeleteDirConsumptionParallelism; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index d243d95b8a9c8..cf685d7d74fa4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -415,18 +415,19 @@ public FSDataOutputStream createNonRecursive(final Path f, final FsPermission pe ERR_CREATE_ON_ROOT, null); } - final Path parent = f.getParent(); + TracingContext tracingContext = new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.CREATE_NON_RECURSIVE, tracingHeaderFormat, listener); - final FileStatus parentFileStatus = tryGetFileStatus(parent, tracingContext); - - if (parentFileStatus == null) { - throw new FileNotFoundException("Cannot create file " - + f.getName() + " because parent folder does not exist."); + try { + Path qualifiedPath = makeQualified(f); + getAbfsStore().createNonRecursivePreCheck(qualifiedPath, tracingContext); + return create(f, permission, overwrite, bufferSize, replication, + blockSize, progress); + } catch (AzureBlobFileSystemException ex) { + checkException(f, ex); + return null; } - - return create(f, permission, overwrite, bufferSize, replication, blockSize, progress); } @Override @@ -526,9 +527,6 @@ public boolean rename(final Path src, final Path dst) throws IOException { Path adjustedDst = dst; if (dstFileStatus != null) { - if (!dstFileStatus.isDirectory()) { - return qualifiedSrcPath.equals(qualifiedDstPath); - } adjustedDst = new Path(dst, sourceFileName); } @@ -542,6 +540,8 @@ public boolean rename(final Path src, final Path dst) throws IOException { src, ex, AzureServiceErrorCode.PATH_ALREADY_EXISTS, + AzureServiceErrorCode.BLOB_ALREADY_EXISTS, + AzureServiceErrorCode.BLOB_PATH_NOT_FOUND, AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH, AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND, AzureServiceErrorCode.INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE, @@ -649,7 +649,10 @@ public boolean delete(final Path f, final boolean recursive) throws IOException abfsStore.delete(qualifiedPath, recursive, tracingContext); return true; } catch (AzureBlobFileSystemException ex) { - checkException(f, ex, AzureServiceErrorCode.PATH_NOT_FOUND); + checkException(f, + ex, + AzureServiceErrorCode.PATH_NOT_FOUND, + AzureServiceErrorCode.BLOB_PATH_NOT_FOUND); return false; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index aa4b3b95ab9c2..cc34d1a406744 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -159,6 +159,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_FOOTER_READ_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BUFFERED_PREAD_DISABLE; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_IDENTITY_TRANSFORM_CLASS; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.INFINITE_LEASE_DURATION; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_BLOB_DOMAIN_NAME; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT; @@ -188,7 +189,6 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport { private final Map leaseRefs; private final AbfsConfiguration abfsConfiguration; - private final Set azureAtomicRenameDirSet; private Set azureInfiniteLeaseDirSet; private volatile Trilean isNamespaceEnabled; private final AuthType authType; @@ -256,8 +256,6 @@ public AzureBlobFileSystemStore( } LOG.trace("primaryUserGroup is {}", this.primaryUserGroup); - this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList( - abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA))); updateInfiniteLeaseDirs(); this.authType = abfsConfiguration.getAuthType(accountName); boolean usingOauth = (authType == AuthType.OAuth); @@ -630,6 +628,21 @@ public void deleteFilesystem(TracingContext tracingContext) } } + /** + * Checks existence of parent of the given path. + * + * @param path Path to check. + * @param tracingContext tracing context + * + * @throws IOException if there is an issue with the operation. + */ + public void createNonRecursivePreCheck(final Path path, + TracingContext tracingContext) + throws IOException { + getClient().createNonRecursivePreCheck(path.getParent(), + tracingContext); + } + public OutputStream createFile(final Path path, final FileSystem.Statistics statistics, final boolean overwrite, final FsPermission permission, final FsPermission umask, @@ -745,8 +758,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa } } - String eTag = op.getResult() - .getResponseHeader(HttpHeaderConfigurations.ETAG); + String eTag = extractEtagHeader(op.getResult()); try { // overwrite only if eTag matches with the file properties fetched befpre @@ -825,6 +837,16 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext( .build(); } + /** + * Creates a directory. + * + * @param path Path of the directory to create. + * @param permission Permission of the directory. + * @param umask Umask of the directory. + * @param tracingContext tracing context + * + * @throws AzureBlobFileSystemException server error. + */ public void createDirectory(final Path path, final FsPermission permission, final FsPermission umask, TracingContext tracingContext) throws IOException { @@ -1063,11 +1085,6 @@ public boolean rename(final Path source, long countAggregate = 0; boolean shouldContinue; - if (isAtomicRenameKey(source.getName())) { - LOG.warn("The atomic rename feature is not supported by the ABFS scheme; however rename," - +" create and delete operations are atomic if Namespace is enabled for your Azure Storage account."); - } - LOG.debug("renameAsync filesystem: {} source: {} destination: {}", getClient().getFileSystem(), source, @@ -1089,8 +1106,11 @@ public boolean rename(final Path source, isNamespaceEnabled); AbfsRestOperation op = abfsClientRenameResult.getOp(); - perfInfo.registerResult(op.getResult()); - continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); + if (op != null) { + perfInfo.registerResult(op.getResult()); + continuation = op.getResult() + .getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); + } perfInfo.registerSuccess(true); countAggregate++; shouldContinue = continuation != null && !continuation.isEmpty(); @@ -1124,8 +1144,11 @@ public void delete(final Path path, final boolean recursive, try (AbfsPerfInfo perfInfo = startTracking("delete", "deletePath")) { AbfsRestOperation op = getClient().deletePath(relativePath, recursive, continuation, tracingContext, getIsNamespaceEnabled(tracingContext)); - perfInfo.registerResult(op.getResult()); - continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); + if (op != null) { + perfInfo.registerResult(op.getResult()); + continuation = op.getResult() + .getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); + } perfInfo.registerSuccess(true); countAggregate++; shouldContinue = continuation != null && !continuation.isEmpty(); @@ -1722,10 +1745,6 @@ public void access(final Path path, final FsAction mode, } } - public boolean isAtomicRenameKey(String key) { - return isKeyForDirectorySet(key, azureAtomicRenameDirSet); - } - public boolean isInfiniteLeaseKey(String key) { if (azureInfiniteLeaseDirSet.isEmpty()) { return false; @@ -1924,7 +1943,7 @@ private Hashtable parseCommaSeparatedXmsProperties(String xMsPro return properties; } - private boolean isKeyForDirectorySet(String key, Set dirSet) { + public static boolean isKeyForDirectorySet(String key, Set dirSet) { for (String dir : dirSet) { if (dir.isEmpty() || key.startsWith(dir + AbfsHttpConstants.FORWARD_SLASH)) { return true; @@ -2203,7 +2222,8 @@ private AbfsLease maybeCreateLease(String relativePath, TracingContext tracingCo if (!enableInfiniteLease) { return null; } - AbfsLease lease = new AbfsLease(getClient(), relativePath, tracingContext); + AbfsLease lease = new AbfsLease(getClient(), relativePath, true, + INFINITE_LEASE_DURATION, null, tracingContext); leaseRefs.put(lease, null); return lease; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java index 4e64442a35b69..3ba2050da2e1a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java @@ -290,5 +290,13 @@ public static ApiVersion getCurrentVersion() { public static final String JDK_FALLBACK = "JDK_fallback"; public static final String KEEP_ALIVE_CACHE_CLOSED = "KeepAliveCache is closed"; + public static final String COPY_STATUS_SUCCESS = "success"; + public static final String COPY_STATUS_PENDING = "pending"; + public static final String COPY_STATUS_ABORTED = "aborted"; + public static final String COPY_STATUS_FAILED = "failed"; + + public static final String ATOMIC_DIR_RENAME_RECOVERY_ON_GET_PATH_EXCEPTION = + "Path had to be recovered from atomic rename operation."; + private AbfsHttpConstants() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index 620182f5bd20a..9c7ab42f370aa 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -359,5 +359,30 @@ public static String accountProperty(String property, String account) { public static final String FS_AZURE_APACHE_HTTP_CLIENT_MAX_CACHE_CONNECTION_SIZE = "fs.azure.apache.http.client.max.cache.connection.size"; /**Maximum idle time for a ApacheHttpClient-connection: {@value}*/ public static final String FS_AZURE_APACHE_HTTP_CLIENT_IDLE_CONNECTION_TTL = "fs.azure.apache.http.client.idle.connection.ttl"; + /** + * Blob copy API is an async API, this configuration defines polling duration + * for checking copy status: {@value} + */ + public static final String FS_AZURE_BLOB_COPY_PROGRESS_WAIT_MILLIS = "fs.azure.blob.copy.progress.wait.millis"; + /** + * Maximum time to wait for a blob copy operation to complete: {@value} + */ + public static final String FS_AZURE_BLOB_COPY_MAX_WAIT_MILLIS = "fs.azure.blob.copy.max.wait.millis"; + /**Blob rename lease refresh duration: {@value}*/ + public static final String FS_AZURE_BLOB_ATOMIC_RENAME_LEASE_REFRESH_DURATION + = "fs.azure.blob.atomic.rename.lease.refresh.duration"; + /**Maximum number of blob information enqueued in memory for rename or delete orchestration: {@value}*/ + public static final String FS_AZURE_PRODUCER_QUEUE_MAX_SIZE = "fs.azure.blob.dir.list.producer.queue.max.size"; + /** + * Maximum consumer lag (count of blob information which is yet to be taken for operation) + * in blob listing which can be tolerated before making producer to wait for + * consumer lag to become tolerable: {@value}. + */ + public static final String FS_AZURE_CONSUMER_MAX_LAG = "fs.azure.blob.dir.list.consumer.max.lag"; + /**Maximum number of thread per blob-rename orchestration: {@value}*/ + public static final String FS_AZURE_BLOB_DIR_RENAME_MAX_THREAD = "fs.azure.blob.dir.rename.max.thread"; + /**Maximum number of thread per blob-delete orchestration: {@value}*/ + public static final String FS_AZURE_BLOB_DIR_DELETE_MAX_THREAD = "fs.azure.blob.dir.delete.max.thread"; + private ConfigurationKeys() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index 6ada0e8bcca74..8978f6564b83a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -36,7 +36,7 @@ public final class FileSystemConfigurations { public static final boolean DEFAULT_FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED = true; public static final String USER_HOME_DIRECTORY_PREFIX = "/user"; - private static final int SIXTY_SECONDS = 60_000; + public static final int SIXTY_SECONDS = 60_000; // Retry parameter defaults. public static final int DEFAULT_MIN_BACKOFF_INTERVAL = 500; // 500ms @@ -177,5 +177,23 @@ public final class FileSystemConfigurations { public static final int DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS = 5; + public static final long DEFAULT_AZURE_BLOB_COPY_PROGRESS_WAIT_MILLIS = 1_000L; + + public static final long DEFAULT_AZURE_BLOB_COPY_MAX_WAIT_MILLIS = 300_000L; + + public static final long DEFAULT_AZURE_BLOB_ATOMIC_RENAME_LEASE_REFRESH_DURATION = 60_000L; + + public static final int DEFAULT_FS_AZURE_PRODUCER_QUEUE_MAX_SIZE = 2 * DEFAULT_AZURE_LIST_MAX_RESULTS; + + public static final int DEFAULT_FS_AZURE_CONSUMER_MAX_LAG = DEFAULT_AZURE_LIST_MAX_RESULTS; + + public static final int DEFAULT_FS_AZURE_LISTING_ACTION_THREADS = 5; + + public static final int DEFAULT_FS_AZURE_BLOB_RENAME_THREAD = DEFAULT_FS_AZURE_LISTING_ACTION_THREADS; + + public static final int DEFAULT_FS_AZURE_BLOB_DELETE_THREAD = DEFAULT_FS_AZURE_LISTING_ACTION_THREADS; + + public static final int BLOCK_ID_LENGTH = 60; + private FileSystemConfigurations() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java index 53020750ab310..33e7946be566f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java @@ -104,5 +104,27 @@ public final class HttpHeaderConfigurations { */ public static final String X_MS_BLOB_CONTENT_MD5 = "x-ms-blob-content-md5"; + /** + * Http Response Header for denoting the copy id of the operation. + * {@value} + */ + public static final String X_MS_COPY_ID = "x-ms-copy-id"; + + /** + * Http Response Header for denoting the copy status description of the operation. + * {@value} + */ + public static final String X_MS_COPY_STATUS_DESCRIPTION = "x-ms-copy-status-description"; + + /** + * Http Response Header for denoting the copy status of the operation. + * Expected Values: + * {@link AbfsHttpConstants#COPY_STATUS_SUCCESS} + * {@link AbfsHttpConstants#COPY_STATUS_FAILED} + * {@link AbfsHttpConstants#COPY_STATUS_PENDING} + * {@link AbfsHttpConstants#COPY_STATUS_ABORTED} + */ + public static final String X_MS_COPY_STATUS = "x-ms-copy-status"; + private HttpHeaderConfigurations() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java index db1560d541430..cda1140407570 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java @@ -45,6 +45,8 @@ public enum AzureServiceErrorCode { INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE("InvalidSourceOrDestinationResourceType", HttpURLConnection.HTTP_CONFLICT, null), RENAME_DESTINATION_PARENT_PATH_NOT_FOUND("RenameDestinationParentPathNotFound", HttpURLConnection.HTTP_NOT_FOUND, null), INVALID_RENAME_SOURCE_PATH("InvalidRenameSourcePath", HttpURLConnection.HTTP_CONFLICT, null), + NON_EMPTY_DIRECTORY_DELETE("DirectoryNotEmpty", HttpURLConnection.HTTP_CONFLICT, + "The recursive query parameter value must be true to delete a non-empty directory"), INGRESS_OVER_ACCOUNT_LIMIT("ServerBusy", HttpURLConnection.HTTP_UNAVAILABLE, "Ingress is over the account limit."), EGRESS_OVER_ACCOUNT_LIMIT("ServerBusy", HttpURLConnection.HTTP_UNAVAILABLE, @@ -54,10 +56,13 @@ public enum AzureServiceErrorCode { OTHER_SERVER_THROTTLING("ServerBusy", HttpURLConnection.HTTP_UNAVAILABLE, "The server is currently unable to receive requests. Please retry your request."), INVALID_QUERY_PARAMETER_VALUE("InvalidQueryParameterValue", HttpURLConnection.HTTP_BAD_REQUEST, null), + INVALID_RENAME_DESTINATION("InvalidRenameDestinationPath", HttpURLConnection.HTTP_BAD_REQUEST, null), AUTHORIZATION_PERMISSION_MISS_MATCH("AuthorizationPermissionMismatch", HttpURLConnection.HTTP_FORBIDDEN, null), ACCOUNT_REQUIRES_HTTPS("AccountRequiresHttps", HttpURLConnection.HTTP_BAD_REQUEST, null), MD5_MISMATCH("Md5Mismatch", HttpURLConnection.HTTP_BAD_REQUEST, "The MD5 value specified in the request did not match with the MD5 value calculated by the server."), + COPY_BLOB_FAILED("CopyBlobFailed", HttpURLConnection.HTTP_INTERNAL_ERROR, null), + COPY_BLOB_ABORTED("CopyBlobAborted", HttpURLConnection.HTTP_INTERNAL_ERROR, null), UNKNOWN(null, -1, null); private final String errorCode; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/enums/BlobCopyProgress.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/enums/BlobCopyProgress.java new file mode 100644 index 0000000000000..9d5237a6730e2 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/enums/BlobCopyProgress.java @@ -0,0 +1,41 @@ +/** + * 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.enums; + +/** + * Enum for BlobCopyProgress. + */ +public enum BlobCopyProgress { + /** + * Copy is successful. + */ + SUCCESS, + /** + * Copy is failed. + */ + FAILURE, + /** + * Copy is aborted. + */ + ABORTED, + /** + * Copy is pending. + */ + PENDING; +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java index e3552b777045c..a4fe7ad9bd9c8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobClient.java @@ -38,6 +38,8 @@ import java.util.Map; import java.util.TreeMap; import java.util.UUID; +import java.util.Arrays; +import java.util.HashSet; import org.w3c.dom.Document; import org.w3c.dom.Node; @@ -68,9 +70,14 @@ import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; -import static java.net.HttpURLConnection.HTTP_NOT_FOUND; import static java.net.HttpURLConnection.HTTP_OK; +import static java.net.HttpURLConnection.HTTP_NOT_FOUND; +import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR; import static java.net.HttpURLConnection.HTTP_PRECON_FAILED; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ACQUIRE_LEASE_ACTION; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPLICATION_JSON; @@ -109,6 +116,8 @@ import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_TAG_NAME; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XMS_PROPERTIES_ENCODING_ASCII; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XMS_PROPERTIES_ENCODING_UNICODE; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ATOMIC_DIR_RENAME_RECOVERY_ON_GET_PATH_EXCEPTION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.AND_MARK; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.ACCEPT; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.CONTENT_LENGTH; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.CONTENT_MD5; @@ -140,12 +149,17 @@ import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_MAX_RESULTS; import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_PREFIX; import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_RESTYPE; +import static org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.isKeyForDirectorySet; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_DELETE_BLOB; +import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_RENAME_BLOB; /** * AbfsClient interacting with Blob endpoint. */ public class AbfsBlobClient extends AbfsClient { + private final HashSet azureAtomicRenameDirSet; + public AbfsBlobClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, final AbfsConfiguration abfsConfiguration, @@ -154,6 +168,8 @@ public AbfsBlobClient(final URL baseUrl, final AbfsClientContext abfsClientContext) throws IOException { super(baseUrl, sharedKeyCredentials, abfsConfiguration, tokenProvider, encryptionContextProvider, abfsClientContext); + this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList( + abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA))); } public AbfsBlobClient(final URL baseUrl, @@ -164,6 +180,8 @@ public AbfsBlobClient(final URL baseUrl, final AbfsClientContext abfsClientContext) throws IOException { super(baseUrl, sharedKeyCredentials, abfsConfiguration, sasTokenProvider, encryptionContextProvider, abfsClientContext); + this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList( + abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA))); } /** @@ -338,6 +356,8 @@ public AbfsRestOperation listPath(final String relativePath, final boolean recur requestHeaders); op.execute(tracingContext); + // Filter the paths for which no rename redo operation is performed. + fixAtomicEntriesInListResults(op, tracingContext); if (isEmptyListResults(op.getResult()) && is404CheckRequired) { // If the list operation returns no paths, we need to check if the path is a file. // If it is a file, we need to return the file in the list. @@ -359,6 +379,52 @@ public AbfsRestOperation listPath(final String relativePath, final boolean recur return op; } + /** + * Filter the paths for which no rename redo operation is performed. + * Update BlobListResultSchema path with filtered entries. + * + * @param op blob list operation + * @param tracingContext tracing context + * @throws AzureBlobFileSystemException if rest operation or response parsing fails. + */ + private void fixAtomicEntriesInListResults(final AbfsRestOperation op, + final TracingContext tracingContext) throws AzureBlobFileSystemException { + /* + * Crashed HBase log rename recovery is done by Filesystem.getFileStatus and + * Filesystem.listStatus. + */ + if (tracingContext == null + || tracingContext.getOpType() != FSOperationType.LISTSTATUS + || op == null || op.getResult() == null + || op.getResult().getStatusCode() != HTTP_OK) { + return; + } + BlobListResultSchema listResultSchema + = (BlobListResultSchema) op.getResult().getListResultSchema(); + if (listResultSchema == null) { + return; + } + List filteredEntries = new ArrayList<>(); + for (BlobListResultEntrySchema entry : listResultSchema.paths()) { + if (!takeListPathAtomicRenameKeyAction(entry.path(), + entry.contentLength().intValue(), tracingContext)) { + filteredEntries.add(entry); + } + } + + listResultSchema.withPaths(filteredEntries); + } + + /**{@inheritDoc}*/ + @Override + public void createNonRecursivePreCheck(Path parentPath, TracingContext tracingContext) + throws IOException { + if (isAtomicRenameKey(parentPath.toUri().getPath())) { + takeGetPathStatusAtomicRenameKeyAction(parentPath, tracingContext); + } + super.createNonRecursivePreCheck(parentPath, tracingContext); + } + /** * Get Rest Operation for API * Put Blob. @@ -391,12 +457,17 @@ public AbfsRestOperation createPath(final String path, * @throws AzureBlobFileSystemException if rest operation fails. */ @Override - public AbfsRestOperation acquireLease(final String path, final int duration, - TracingContext tracingContext) throws AzureBlobFileSystemException { + public AbfsRestOperation acquireLease(final String path, + final int duration, + final String eTag, + TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, ACQUIRE_LEASE_ACTION)); requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_DURATION, Integer.toString(duration))); requestHeaders.add(new AbfsHttpHeader(X_MS_PROPOSED_LEASE_ID, UUID.randomUUID().toString())); + if (StringUtils.isNotEmpty(eTag)) { + requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag)); + } final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, LEASE); @@ -490,7 +561,15 @@ public AbfsRestOperation breakLease(final String path, } /** - * Get results for the rename operation. + * Rename a file or directory. + * If a source etag is passed in, the operation will attempt to recover + * from a missing source file by probing the destination for + * existence and comparing etags. + * The second value in the result will be true to indicate that this + * took place. + * As rename recovery is only attempted if the source etag is non-empty, + * in normal rename operations rename recovery will never happen. + * * @param source path to source file * @param destination destination of rename. * @param continuation continuation. @@ -499,21 +578,50 @@ public AbfsRestOperation breakLease(final String path, * @param isMetadataIncompleteState was there a rename failure due to * incomplete metadata state? * @param isNamespaceEnabled whether namespace enabled account or not - * @return result of rename operation - * @throws IOException if rename operation fails. + * + * @return AbfsClientRenameResult result of rename operation indicating the + * AbfsRest operation, rename recovery and incomplete metadata state failure. + * + * @throws IOException failure, excluding any recovery from overload failures. */ @Override public AbfsClientRenameResult renamePath(final String source, - final String destination, - final String continuation, - final TracingContext tracingContext, - final String sourceEtag, - final boolean isMetadataIncompleteState, - final boolean isNamespaceEnabled) throws IOException { - /** - * TODO: [FnsOverBlob] To be implemented as part of rename-delete over blob endpoint work. HADOOP-19233. - */ - throw new NotImplementedException("Rename operation on Blob endpoint yet to be implemented."); + final String destination, + final String continuation, + final TracingContext tracingContext, + String sourceEtag, + boolean isMetadataIncompleteState, + boolean isNamespaceEnabled) + throws IOException { + BlobRenameHandler blobRenameHandler = getBlobRenameHandler(source, + destination, sourceEtag, isAtomicRenameKey(source), tracingContext + ); + incrementAbfsRenamePath(); + if (blobRenameHandler.execute()) { + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + final URL url = createRequestUrl(destination, abfsUriQueryBuilder.toString()); + final List requestHeaders = createDefaultHeaders(); + final AbfsRestOperation successOp = getAbfsRestOperation( + AbfsRestOperationType.RenamePath, HTTP_METHOD_PUT, + url, requestHeaders); + successOp.hardSetResult(HttpURLConnection.HTTP_OK); + return new AbfsClientRenameResult(successOp, true, false); + } else { + throw new AbfsRestOperationException(HTTP_INTERNAL_ERROR, + AzureServiceErrorCode.UNKNOWN.getErrorCode(), + ERR_RENAME_BLOB + source + SINGLE_WHITE_SPACE + AND_MARK + SINGLE_WHITE_SPACE + destination, + null); + } + } + + @VisibleForTesting + BlobRenameHandler getBlobRenameHandler(final String source, + final String destination, + final String sourceEtag, + final boolean isAtomicRename, + final TracingContext tracingContext) { + return new BlobRenameHandler(source, + destination, this, sourceEtag, isAtomicRename, false, tracingContext); } /** @@ -786,9 +894,17 @@ public AbfsRestOperation getPathStatus(final String path, final TracingContext tracingContext, final ContextEncryptionAdapter contextEncryptionAdapter) throws AzureBlobFileSystemException { - return this.getPathStatus(path, tracingContext, - contextEncryptionAdapter, true); - + AbfsRestOperation op = this.getPathStatus(path, tracingContext, + contextEncryptionAdapter, true); + /* + * Crashed HBase log-folder rename can be recovered by FileSystem#getFileStatus + * and FileSystem#listStatus calls. + */ + if (tracingContext.getOpType() == FSOperationType.GET_FILESTATUS + && op.getResult() != null && checkIsDir(op.getResult())) { + takeGetPathStatusAtomicRenameKeyAction(new Path(path), tracingContext); + } + return op; } /** @@ -909,23 +1025,46 @@ public AbfsRestOperation read(final String path, } /** - * Orchestration for delete operation to be implemented. + * Get Rest Operation for API . + * Delete the file or directory at specified path. * @param path to be deleted. * @param recursive if the path is a directory, delete recursively. * @param continuation to specify continuation token. - * @param tracingContext for tracing the server calls. - * @param isNamespaceEnabled specify if the namespace is enabled. + * @param tracingContext TracingContext instance to track identifiers + * @param isNamespaceEnabled whether namespace enabled account or not * @return executed rest operation containing response from server. * @throws AzureBlobFileSystemException if rest operation fails. */ @Override public AbfsRestOperation deletePath(final String path, - final boolean recursive, - final String continuation, - TracingContext tracingContext, - final boolean isNamespaceEnabled) throws AzureBlobFileSystemException { - // TODO: [FnsOverBlob][HADOOP-19233] To be implemented as part of rename-delete over blob endpoint work. - throw new NotImplementedException("Delete operation on Blob endpoint will be implemented in future."); + final boolean recursive, + final String continuation, + final TracingContext tracingContext, + final boolean isNamespaceEnabled) throws AzureBlobFileSystemException { + BlobDeleteHandler blobDeleteHandler = getBlobDeleteHandler(path, recursive, tracingContext); + if (blobDeleteHandler.execute()) { + final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); + final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); + final List requestHeaders = createDefaultHeaders(); + final AbfsRestOperation successOp = getAbfsRestOperation( + AbfsRestOperationType.DeletePath, HTTP_METHOD_DELETE, + url, requestHeaders); + successOp.hardSetResult(HttpURLConnection.HTTP_OK); + return successOp; + } else { + throw new AbfsRestOperationException(HTTP_INTERNAL_ERROR, + AzureServiceErrorCode.UNKNOWN.getErrorCode(), + ERR_DELETE_BLOB + path, + null); + } + } + + @VisibleForTesting + public BlobDeleteHandler getBlobDeleteHandler(final String path, + final boolean recursive, + final TracingContext tracingContext) { + return new BlobDeleteHandler(new Path(path), recursive, this, + tracingContext); } /** @@ -1318,6 +1457,127 @@ public static String getDirectoryQueryParameter(final String path) { return directory; } + public boolean isAtomicRenameKey(String key) { + return isKeyForDirectorySet(key, azureAtomicRenameDirSet); + } + + /** + * Action to be taken when atomic-key is present on a getPathStatus path. + * + * @param path path of the pendingJson for the atomic path. + * @param tracingContext tracing context. + * + * @throws AzureBlobFileSystemException server error or the path is renamePending json file and action is taken. + */ + public void takeGetPathStatusAtomicRenameKeyAction(final Path path, + final TracingContext tracingContext) + throws AzureBlobFileSystemException { + if (path == null || path.isRoot() || !isAtomicRenameKey(path.toUri().getPath())) { + return; + } + AbfsRestOperation pendingJsonFileStatus; + Path pendingJsonPath = new Path(path.getParent(), + path.toUri().getPath() + RenameAtomicity.SUFFIX); + try { + pendingJsonFileStatus = getPathStatus( + pendingJsonPath.toUri().getPath(), tracingContext, + null, false); + if (checkIsDir(pendingJsonFileStatus.getResult())) { + return; + } + } catch (AbfsRestOperationException ex) { + if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { + return; + } + throw ex; + } + + boolean renameSrcHasChanged; + try { + RenameAtomicity renameAtomicity = getRedoRenameAtomicity( + pendingJsonPath, Integer.parseInt(pendingJsonFileStatus.getResult() + .getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)), + tracingContext); + renameAtomicity.redo(); + renameSrcHasChanged = false; + } catch (AbfsRestOperationException ex) { + /* + * At this point, the source marked by the renamePending json file, might have + * already got renamed by some parallel thread, or at this point, the path + * would have got modified which would result in eTag change, which would lead + * to a HTTP_CONFLICT. In this case, no more operation needs to be taken, and + * the calling getPathStatus can return this source path as result. + */ + if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND + || ex.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { + renameSrcHasChanged = true; + } else { + throw ex; + } + } + if (!renameSrcHasChanged) { + throw new AbfsRestOperationException( + AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(), + AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(), + ATOMIC_DIR_RENAME_RECOVERY_ON_GET_PATH_EXCEPTION, + null); + } + } + + /** + * Redo the rename operation when path is present in atomic directory list + * or when path has {@link RenameAtomicity#SUFFIX} suffix. + * + * @param path path of the pendingJson for the atomic path. + * @param renamePendingJsonLen length of the pendingJson file. + * @param tracingContext tracing context. + * + * @return true if action is taken. + * @throws AzureBlobFileSystemException server error + */ + private boolean takeListPathAtomicRenameKeyAction(final Path path, + final int renamePendingJsonLen, + final TracingContext tracingContext) + throws AzureBlobFileSystemException { + if (path == null || path.isRoot() || !isAtomicRenameKey( + path.toUri().getPath()) || !path.toUri() + .getPath() + .endsWith(RenameAtomicity.SUFFIX)) { + return false; + } + try { + RenameAtomicity renameAtomicity + = getRedoRenameAtomicity(path, renamePendingJsonLen, + tracingContext); + renameAtomicity.redo(); + } catch (AbfsRestOperationException ex) { + /* + * At this point, the source marked by the renamePending json file, might have + * already got renamed by some parallel thread, or at this point, the path + * would have got modified which would result in eTag change, which would lead + * to a HTTP_CONFLICT. In this case, no more operation needs to be taken, but + * since this is a renamePendingJson file and would be deleted by the redo operation, + * the calling listPath should not return this json path as result. + */ + if (ex.getStatusCode() != HttpURLConnection.HTTP_NOT_FOUND + && ex.getStatusCode() != HttpURLConnection.HTTP_CONFLICT) { + throw ex; + } + } + return true; + } + + @VisibleForTesting + RenameAtomicity getRedoRenameAtomicity(final Path renamePendingJsonPath, + int fileLen, + final TracingContext tracingContext) { + return new RenameAtomicity(renamePendingJsonPath, + fileLen, + tracingContext, + null, + this); + } + /** * Checks if the value contains pure ASCII characters or not. * @param value to be checked. diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 0eedeeea45968..3b253c2dd0f40 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.UnsupportedEncodingException; +import java.io.FileNotFoundException; import java.net.HttpURLConnection; import java.net.InetAddress; import java.net.MalformedURLException; @@ -44,6 +45,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; @@ -90,6 +92,7 @@ import org.apache.hadoop.util.concurrent.HadoopExecutors; import static org.apache.commons.lang3.StringUtils.isNotEmpty; +import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.CALL_GET_FILE_STATUS; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.RENAME_PATH_ATTEMPTS; import static org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.extractEtagHeader; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APN_VERSION; @@ -549,16 +552,44 @@ public abstract AbfsRestOperation createPath(String path, ContextEncryptionAdapter contextEncryptionAdapter, TracingContext tracingContext) throws AzureBlobFileSystemException; + /** + * Performs a pre-check for a createNonRecursive operation. Checks if parentPath + * exists or not. + * + * @param parentPath parent path of the file to be created. + * @param tracingContext trace context + * + * @throws IOException if parentPath does not exist or server error. + */ + public void createNonRecursivePreCheck(Path parentPath, TracingContext tracingContext) + throws IOException { + try { + getPathStatus(parentPath.toUri().getPath(), false, + tracingContext, null); + } catch (AbfsRestOperationException ex) { + if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { + throw new FileNotFoundException("Cannot create file " + + parentPath.toUri().getPath() + " because parent folder does not exist."); + } + throw ex; + } finally { + abfsCounters.incrementCounter(CALL_GET_FILE_STATUS, 1); + } + } + /** * Acquire lease on specified path. * @param path on which lease has to be acquired. * @param duration for which lease has to be acquired. + * @param eTag required to acquire lease on the path. * @param tracingContext for tracing the server calls. * @return executed rest operation containing response from server. * @throws AzureBlobFileSystemException if rest operation fails. */ - public abstract AbfsRestOperation acquireLease(String path, int duration, - TracingContext tracingContext) throws AzureBlobFileSystemException; + public abstract AbfsRestOperation acquireLease(String path, + int duration, + String eTag, + TracingContext tracingContext) throws AzureBlobFileSystemException; /** * Renew lease on specified path. diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java index 75081a44c25a3..85265304e2ebb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsClient.java @@ -421,8 +421,10 @@ public AbfsRestOperation createPath(final String path, * @throws AzureBlobFileSystemException if rest operation fails. */ @Override - public AbfsRestOperation acquireLease(final String path, final int duration, - TracingContext tracingContext) throws AzureBlobFileSystemException { + public AbfsRestOperation acquireLease(final String path, + final int duration, + final String eTag, + TracingContext tracingContext) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, ACQUIRE_LEASE_ACTION)); requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_DURATION, Integer.toString(duration))); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java index 86285e08f2ce3..ee469698f4f95 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsErrors.java @@ -49,5 +49,7 @@ public final class AbfsErrors { public static final String ERR_NO_LEASE_THREADS = "Lease desired but no lease threads " + "configured, set " + FS_AZURE_LEASE_THREADS; public static final String ERR_CREATE_ON_ROOT = "Cannot create file over root path"; + public static final String ERR_RENAME_BLOB = "FNS-Blob rename was not successful for source and destination path: "; + public static final String ERR_DELETE_BLOB = "FNS-Blob delete was not successful for path: "; private AbfsErrors() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java index 4c50b77860941..f1a91c65b2fde 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsLease.java @@ -20,6 +20,8 @@ import java.io.IOException; import java.util.concurrent.TimeUnit; +import java.util.Timer; +import java.util.TimerTask; import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback; @@ -29,7 +31,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; @@ -45,10 +46,10 @@ * AbfsLease manages an Azure blob lease. It acquires an infinite lease on instantiation and * releases the lease when free() is called. Use it to prevent writes to the blob by other * processes that don't have the lease. - * + *

* Creating a new Lease object blocks the caller until the Azure blob lease is acquired. It will * retry a fixed number of times before failing if there is a problem acquiring the lease. - * + *

* Call free() to release the Lease. If the holder process dies, AzureBlobFileSystem breakLease * will need to be called before another client will be able to write to the file. */ @@ -70,6 +71,10 @@ public final class AbfsLease { private volatile Throwable exception = null; private volatile int acquireRetryCount = 0; private volatile ListenableScheduledFuture future = null; + private final long leaseRefreshDuration; + private final Timer timer; + private LeaseTimerTask leaseTimerTask; + private final boolean isAsync; public static class LeaseException extends AzureBlobFileSystemException { public LeaseException(Throwable t) { @@ -81,27 +86,59 @@ public LeaseException(String s) { } } - public AbfsLease(AbfsClient client, String path, TracingContext tracingContext) throws AzureBlobFileSystemException { - this(client, path, DEFAULT_LEASE_ACQUIRE_MAX_RETRIES, - DEFAULT_LEASE_ACQUIRE_RETRY_INTERVAL, tracingContext); + /** + * Create a new lease object and acquire a lease on the given path. + * + * @param client AbfsClient + * @param path Path to acquire lease on + * @param isAsync Whether to acquire lease asynchronously + * @param leaseRefreshDuration Duration in milliseconds to renew the lease + * @param eTag ETag of the file + * @param tracingContext Tracing context + * @throws AzureBlobFileSystemException if the lease cannot be acquired + */ + public AbfsLease(AbfsClient client, String path, + final boolean isAsync, final long leaseRefreshDuration, + final String eTag, TracingContext tracingContext) throws AzureBlobFileSystemException { + this(client, path, isAsync, DEFAULT_LEASE_ACQUIRE_MAX_RETRIES, + DEFAULT_LEASE_ACQUIRE_RETRY_INTERVAL, leaseRefreshDuration, eTag, tracingContext); } + /** + * Create a new lease object and acquire a lease on the given path. + * + * @param client AbfsClient + * @param path Path to acquire lease on + * @param isAsync Whether to acquire lease asynchronously + * @param acquireMaxRetries Maximum number of retries to acquire lease + * @param acquireRetryInterval Retry interval in seconds to acquire lease + * @param leaseRefreshDuration Duration in milliseconds to renew the lease + * @param eTag ETag of the file + * @param tracingContext Tracing context + * @throws AzureBlobFileSystemException if the lease cannot be acquired + */ @VisibleForTesting - public AbfsLease(AbfsClient client, String path, int acquireMaxRetries, - int acquireRetryInterval, TracingContext tracingContext) throws AzureBlobFileSystemException { + public AbfsLease(AbfsClient client, String path, final boolean isAsync, int acquireMaxRetries, + int acquireRetryInterval, final long leaseRefreshDuration, + final String eTag, + TracingContext tracingContext) throws AzureBlobFileSystemException { this.leaseFreed = false; this.client = client; this.path = path; this.tracingContext = tracingContext; + this.leaseRefreshDuration = leaseRefreshDuration; + this.isAsync = isAsync; - if (client.getNumLeaseThreads() < 1) { + if (isAsync && client.getNumLeaseThreads() < 1) { throw new LeaseException(ERR_NO_LEASE_THREADS); } // Try to get the lease a specified number of times, else throw an error RetryPolicy retryPolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep( acquireMaxRetries, acquireRetryInterval, TimeUnit.SECONDS); - acquireLease(retryPolicy, 0, acquireRetryInterval, 0, + this.timer = new Timer( + String.format("lease-refresh-timer-%s", path), true); + acquireLease(retryPolicy, 0, acquireRetryInterval, 0, eTag, new TracingContext(tracingContext)); while (leaseID == null && exception == null) { @@ -120,20 +157,33 @@ public AbfsLease(AbfsClient client, String path, int acquireMaxRetries, LOG.debug("Acquired lease {} on {}", leaseID, path); } + /** + * Acquire a lease on the given path. + * + * @param retryPolicy Retry policy + * @param numRetries Number of retries + * @param retryInterval Retry interval in seconds + * @param delay Delay in seconds + * @param eTag ETag of the file + * @param tracingContext Tracing context + * @throws LeaseException if the lease cannot be acquired + */ private void acquireLease(RetryPolicy retryPolicy, int numRetries, - int retryInterval, long delay, TracingContext tracingContext) + int retryInterval, long delay, final String eTag, TracingContext tracingContext) throws LeaseException { LOG.debug("Attempting to acquire lease on {}, retry {}", path, numRetries); if (future != null && !future.isDone()) { throw new LeaseException(ERR_LEASE_FUTURE_EXISTS); } - future = client.schedule(() -> client.acquireLease(path, - INFINITE_LEASE_DURATION, tracingContext), - delay, TimeUnit.SECONDS); - client.addCallback(future, new FutureCallback() { + FutureCallback acquireCallback = new FutureCallback() { @Override public void onSuccess(@Nullable AbfsRestOperation op) { - leaseID = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_LEASE_ID); + if (leaseRefreshDuration != INFINITE_LEASE_DURATION) { + leaseTimerTask = new LeaseTimerTask(client, path, + leaseID, tracingContext); + timer.scheduleAtFixedRate(leaseTimerTask, leaseRefreshDuration / 2, + leaseRefreshDuration / 2); + } LOG.debug("Acquired lease {} on {}", leaseID, path); } @@ -145,7 +195,7 @@ public void onFailure(Throwable throwable) { LOG.debug("Failed to acquire lease on {}, retrying: {}", path, throwable); acquireRetryCount++; acquireLease(retryPolicy, numRetries + 1, retryInterval, - retryInterval, tracingContext); + retryInterval, eTag, tracingContext); } else { exception = throwable; } @@ -153,7 +203,21 @@ public void onFailure(Throwable throwable) { exception = throwable; } } - }); + }; + if (!isAsync) { + try { + AbfsRestOperation op = client.acquireLease(path, + INFINITE_LEASE_DURATION, eTag, tracingContext); + acquireCallback.onSuccess(op); + return; + } catch (AzureBlobFileSystemException ex) { + acquireCallback.onFailure(ex); + } + } + future = client.schedule(() -> client.acquireLease(path, + INFINITE_LEASE_DURATION, eTag, tracingContext), + delay, TimeUnit.SECONDS); + client.addCallback(future, acquireCallback); } /** @@ -170,6 +234,7 @@ public void free() { if (future != null && !future.isDone()) { future.cancel(true); } + cancelTimer(); TracingContext tracingContext = new TracingContext(this.tracingContext); tracingContext.setOperation(FSOperationType.RELEASE_LEASE); client.releaseLease(path, leaseID, tracingContext); @@ -184,21 +249,77 @@ public void free() { } } + /** + * Cancel the lease renewal timer. + * Also purge the lease refresh timer. + */ + public void cancelTimer() { + if (leaseTimerTask != null) { + leaseTimerTask.cancel(); + } + timer.purge(); + } + + /** + * Check if the lease has been freed. + * + * @return true if the lease has been freed + */ public boolean isFreed() { return leaseFreed; } + /** + * Get the lease ID. + * + * @return lease ID + */ public String getLeaseID() { return leaseID; } + /** + * Get the number of times the lease was retried. + * + * @return number of acquired retry count + */ @VisibleForTesting public int getAcquireRetryCount() { return acquireRetryCount; } + /** + * Get Tracing Context. + * + * @return TracingContext tracing context + */ @VisibleForTesting public TracingContext getTracingContext() { return tracingContext; } + + /** + * Class to track lease renewal. + * If the lease is not renewed, the lease will expire and the file will be available for write. + */ + private static class LeaseTimerTask extends TimerTask { + private final AbfsClient client; + private final String path; + private final String leaseID; + private final TracingContext tracingContext; + LeaseTimerTask(AbfsClient client, String path, String leaseID, TracingContext tracingContext) { + this.client = client; + this.path = path; + this.leaseID = leaseID; + this.tracingContext = tracingContext; + } + @Override + public void run() { + try { + client.renewLease(path, leaseID, tracingContext); + } catch (Exception e) { + LOG.error("Failed to renew lease on {}", path, e); + } + } + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java index 3ed3420a90603..5e1d76ae432a1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java @@ -263,6 +263,7 @@ String getSasToken() { * Execute a AbfsRestOperation. Track the Duration of a request if * abfsCounters isn't null. * @param tracingContext TracingContext instance to track correlation IDs + * @throws AzureBlobFileSystemException if the operation fails. */ public void execute(TracingContext tracingContext) throws AzureBlobFileSystemException { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobDeleteHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobDeleteHandler.java new file mode 100644 index 0000000000000..038c64d478bc9 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobDeleteHandler.java @@ -0,0 +1,201 @@ +/** + * 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.services; + +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +import static java.net.HttpURLConnection.HTTP_CONFLICT; +import static java.net.HttpURLConnection.HTTP_NOT_FOUND; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.NON_EMPTY_DIRECTORY_DELETE; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.PATH_NOT_FOUND; + +/** + * Orchestrator for delete over Blob endpoint. Blob endpoint for flat-namespace + * account does not support directory delete. This class is responsible for + * deleting the blobs and creating the parent directory marker file if needed. + */ +public class BlobDeleteHandler extends ListActionTaker { + + private final Path path; + + private final boolean recursive; + + private boolean nonRecursiveDeleteDirectoryFailed = false; + + private final TracingContext tracingContext; + + private final AtomicInteger deleteCount = new AtomicInteger(0); + + /** Constructor + * + * @param path path to delete. + * @param recursive if true, delete the path recursively. + * @param abfsBlobClient client to use for blob operations. + * @param tracingContext tracing context. + */ + public BlobDeleteHandler(final Path path, + final boolean recursive, + final AbfsBlobClient abfsBlobClient, + final TracingContext tracingContext) { + super(path, abfsBlobClient, tracingContext); + this.path = path; + this.recursive = recursive; + this.tracingContext = tracingContext; + } + + /**{@inheritDoc} + * + * @return the maximum number of parallelism for delete operation. + */ + @Override + int getMaxConsumptionParallelism() { + return getAbfsClient().getAbfsConfiguration() + .getBlobDeleteDirConsumptionParallelism(); + } + + /** Delete the path. + * + * @param path path to delete. + * @return true if the path is deleted. + * @throws AzureBlobFileSystemException server error. + */ + private boolean deleteInternal(final Path path) + throws AzureBlobFileSystemException { + getAbfsClient().deleteBlobPath(path, null, tracingContext); + deleteCount.incrementAndGet(); + return true; + } + + /** + * Orchestrate the delete operation. + * + * @return true if the delete operation is successful. + * @throws AzureBlobFileSystemException if deletion fails due to server error or path doesn't exist. + */ + public boolean execute() throws AzureBlobFileSystemException { + /* + * ABFS is not aware if it's a file or directory. So, we need to list the + * path and delete the listed objects. The listing returns the children of + * the path and not the path itself. + */ + listRecursiveAndTakeAction(); + if (nonRecursiveDeleteDirectoryFailed) { + throw new AbfsRestOperationException(HTTP_CONFLICT, + NON_EMPTY_DIRECTORY_DELETE.getErrorCode(), + NON_EMPTY_DIRECTORY_DELETE.getErrorMessage(), + new PathIOException(path.toString(), + "Non-recursive delete of non-empty directory")); + } + tracingContext.setOperatedBlobCount(deleteCount.get() + 1); + /* + * If path is actually deleted. + */ + boolean deleted; + try { + /* + * Delete the required path. + * Directory should be safely deleted as the path might be implicit. + */ + deleted = recursive ? safeDelete(path) : deleteInternal(path); + } finally { + tracingContext.setOperatedBlobCount(null); + } + if (deleteCount.get() == 0) { + /* + * DeleteCount can be zero only if the path does not exist. + */ + throw new AbfsRestOperationException(HTTP_NOT_FOUND, + PATH_NOT_FOUND.getErrorCode(), PATH_NOT_FOUND.getErrorMessage(), + new PathIOException(path.toString(), "Path not found")); + } + + /* + * Ensure that parent directory of the deleted path is marked as a folder. This + * is required if the parent is an implicit directory (path with no marker blob), + * and the given path is the only child of the parent, the parent would become + * non-existing. + */ + if (deleted) { + ensurePathParentExist(); + } + return deleted; + } + + /** Ensure that the parent path exists. + * + * @throws AzureBlobFileSystemException server error. + */ + private void ensurePathParentExist() + throws AzureBlobFileSystemException { + if (!path.isRoot() && !path.getParent().isRoot()) { + try { + getAbfsClient().createPath(path.getParent().toUri().getPath(), + false, + false, + null, + false, + null, + null, + tracingContext); + } catch (AbfsRestOperationException ex) { + if (ex.getStatusCode() != HTTP_CONFLICT) { + throw ex; + } + } + } + } + + /**{@inheritDoc}*/ + @Override + boolean takeAction(final Path path) throws AzureBlobFileSystemException { + if (!recursive) { + /* + * If the delete operation is non-recursive, then the path can not be a directory. + */ + nonRecursiveDeleteDirectoryFailed = true; + return false; + } + return safeDelete(path); + } + + /** + * Delete the path if it exists. Gracefully handles the case where the path does not exist. + * + * @param path path to delete. + * @return true if the path is deleted or is not found. + * @throws AzureBlobFileSystemException server error. + */ + private boolean safeDelete(final Path path) + throws AzureBlobFileSystemException { + try { + return deleteInternal(path); + } catch (AbfsRestOperationException ex) { + if (ex.getStatusCode() == HTTP_NOT_FOUND) { + return true; + } + throw ex; + } + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobRenameHandler.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobRenameHandler.java new file mode 100644 index 0000000000000..a0f4c40833694 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobRenameHandler.java @@ -0,0 +1,631 @@ +/** + * 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.services; + +import java.net.HttpURLConnection; +import java.net.MalformedURLException; +import java.net.URISyntaxException; +import java.net.URL; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TimeoutException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; +import org.apache.hadoop.fs.azurebfs.enums.BlobCopyProgress; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +import static org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.extractEtagHeader; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_ABORTED; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_FAILED; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_SUCCESS; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ROOT_PATH; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COLON; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_COPY_ID; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_COPY_SOURCE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_COPY_STATUS; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_COPY_STATUS_DESCRIPTION; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.COPY_BLOB_ABORTED; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.COPY_BLOB_FAILED; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND; + +/** + * Orchestrator for rename over Blob endpoint. Handles both directory and file + * renames. Blob Endpoint does not expose rename API, this class is responsible + * for copying the blobs and deleting the source blobs. + *

+ * For directory rename, it recursively lists the blobs in the source directory and + * copies them to the destination directory. + */ +public class BlobRenameHandler extends ListActionTaker { + + public static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); + + private final String srcEtag; + + private final Path src, dst; + + private final boolean isAtomicRename, isAtomicRenameRecovery; + + private final TracingContext tracingContext; + + private AbfsLease srcAbfsLease; + + private String srcLeaseId; + + private final List leases = new ArrayList<>(); + + private final AtomicInteger operatedBlobCount = new AtomicInteger(0); + + /** Constructor. + * + * @param src source path + * @param dst destination path + * @param abfsClient AbfsBlobClient to use for the rename operation + * @param srcEtag eTag of the source path + * @param isAtomicRename true if the rename operation is atomic + * @param isAtomicRenameRecovery true if the rename operation is a recovery of a previous failed atomic rename operation + * @param tracingContext object of tracingContext used for the tracing of the server calls. + */ + public BlobRenameHandler(final String src, + final String dst, + final AbfsBlobClient abfsClient, + final String srcEtag, + final boolean isAtomicRename, + final boolean isAtomicRenameRecovery, + final TracingContext tracingContext) { + super(new Path(src), abfsClient, tracingContext); + this.srcEtag = srcEtag; + this.tracingContext = tracingContext; + this.src = new Path(src); + this.dst = new Path(dst); + this.isAtomicRename = isAtomicRename; + this.isAtomicRenameRecovery = isAtomicRenameRecovery; + } + + /** {@inheritDoc} */ + @Override + int getMaxConsumptionParallelism() { + return getAbfsClient().getAbfsConfiguration() + .getBlobRenameDirConsumptionParallelism(); + } + + /** + * Orchestrates the rename operation. + * + * @return AbfsClientRenameResult containing the result of the rename operation + * @throws AzureBlobFileSystemException if server call fails + */ + public boolean execute() throws AzureBlobFileSystemException { + PathInformation pathInformation = getPathInformation(src, tracingContext); + boolean result = false; + if (preCheck(src, dst, pathInformation)) { + RenameAtomicity renameAtomicity = null; + if (pathInformation.getIsDirectory() + && pathInformation.getIsImplicit()) { + AbfsRestOperation createMarkerOp = getAbfsClient().createPath(src.toUri().getPath(), + false, false, null, + false, null, null, tracingContext); + pathInformation.setETag(extractEtagHeader(createMarkerOp.getResult())); + } + try { + if (isAtomicRename) { + /* + * Conditionally get a lease on the source blob to prevent other writers + * from changing it. This is used for correctness in HBase when log files + * are renamed. When the HBase master renames a log file folder, the lease + * locks out other writers. This prevents a region server that the master + * thinks is dead, but is still alive, from committing additional updates. + * This is different than when HBase runs on HDFS, where the region server + * recovers the lease on a log file, to gain exclusive access to it, before + * it splits it. + */ + if (srcAbfsLease == null) { + srcAbfsLease = takeLease(src, srcEtag); + } + srcLeaseId = srcAbfsLease.getLeaseID(); + if (!isAtomicRenameRecovery && pathInformation.getIsDirectory()) { + /* + * if it is not a resume of a previous failed atomic rename operation, + * Create the rename JSON. + */ + renameAtomicity = getRenameAtomicity(pathInformation); + renameAtomicity.preRename(); + } + } + if (pathInformation.getIsDirectory()) { + result = listRecursiveAndTakeAction() && finalSrcRename(); + } else { + result = renameInternal(src, dst); + } + } finally { + if (srcAbfsLease != null) { + // If the operation is successful, cancel the timer and no need to release + // the lease as rename on the blob-path has taken place. + if (result) { + srcAbfsLease.cancelTimer(); + } else { + srcAbfsLease.free(); + } + } + } + if (result && renameAtomicity != null) { + renameAtomicity.postRename(); + } + } + return result; + } + + /** Final rename operation after all the blobs have been copied. + * + * @return true if rename is successful + * @throws AzureBlobFileSystemException if server call fails + */ + private boolean finalSrcRename() throws AzureBlobFileSystemException { + tracingContext.setOperatedBlobCount(operatedBlobCount.get() + 1); + try { + return renameInternal(src, dst); + } finally { + tracingContext.setOperatedBlobCount(null); + } + } + + /** Gets the rename atomicity object. + * + * @param pathInformation object containing the path information of the source path + * + * @return RenameAtomicity object + */ + @VisibleForTesting + public RenameAtomicity getRenameAtomicity(final PathInformation pathInformation) { + return new RenameAtomicity(src, + dst, + new Path(src.getParent(), src.getName() + RenameAtomicity.SUFFIX), + tracingContext, + pathInformation.getETag(), + getAbfsClient()); + } + + /** Takes a lease on the path. + * + * @param path path on which the lease is to be taken + * @param eTag eTag of the path + * + * @return object containing the lease information + * @throws AzureBlobFileSystemException if server call fails + */ + private AbfsLease takeLease(final Path path, final String eTag) + throws AzureBlobFileSystemException { + AbfsLease lease = new AbfsLease(getAbfsClient(), path.toUri().getPath(), false, + getAbfsClient().getAbfsConfiguration() + .getAtomicRenameLeaseRefreshDuration(), + eTag, tracingContext); + leases.add(lease); + return lease; + } + + /** Checks if the path contains a colon. + * + * @param p path to check + * + * @return true if the path contains a colon + */ + private boolean containsColon(Path p) { + return p.toUri().getPath().contains(COLON); + } + + /** + * Since, server doesn't have a rename API and would not be able to check HDFS + * contracts, client would have to ensure that no HDFS contract is violated. + * + * @param src source path + * @param dst destination path + * @param pathInformation object in which path information of the source path would be stored + * + * @return true if the pre-checks pass + * @throws AzureBlobFileSystemException if server call fails or given paths are invalid. + */ + private boolean preCheck(final Path src, final Path dst, + final PathInformation pathInformation) + throws AzureBlobFileSystemException { + validateDestinationPath(src, dst); + validateSourcePath(pathInformation); + validateDestinationPathNotExist(src, dst, pathInformation); + validateDestinationParentExist(src, dst, pathInformation); + + return true; + } + + /** + * Validate if the format of the destination path is correct and if the destination + * path is not a sub-directory of the source path. + * + * @param src source path + * @param dst destination path + * + * @throws AbfsRestOperationException if the destination path is invalid + */ + private void validateDestinationPath(final Path src, final Path dst) + throws AbfsRestOperationException { + if (containsColon(dst)) { + throw new AbfsRestOperationException( + HttpURLConnection.HTTP_BAD_REQUEST, + AzureServiceErrorCode.INVALID_RENAME_DESTINATION.getErrorCode(), null, + new PathIOException(dst.toUri().getPath(), + "Destination path contains colon")); + } + + validateDestinationIsNotSubDir(src, dst); + } + + /** + * Validate if the destination path is not a sub-directory of the source path. + * + * @param src source path + * @param dst destination path + */ + private void validateDestinationIsNotSubDir(final Path src, + final Path dst) throws AbfsRestOperationException { + LOG.debug("Check if the destination is subDirectory"); + Path nestedDstParent = dst.getParent(); + if (nestedDstParent != null && nestedDstParent.toUri() + .getPath() + .indexOf(src.toUri().getPath()) == 0) { + LOG.info("Rename src: {} dst: {} failed as dst is subDir of src", + src, dst); + throw new AbfsRestOperationException(HttpURLConnection.HTTP_CONFLICT, + AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH.getErrorCode(), + AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH.getErrorMessage(), + new Exception( + AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH.getErrorCode())); + } + } + + /** + * Validate if the source path exists and if the client knows the ETag of the source path, + * then the ETag should match with the server. + * + * @param pathInformation object containing the path information of the source path + * + * @throws AbfsRestOperationException if the source path is not found or if the ETag of the source + * path does not match with the server. + */ + private void validateSourcePath(final PathInformation pathInformation) + throws AzureBlobFileSystemException { + if (!pathInformation.getPathExists()) { + throw new AbfsRestOperationException( + HttpURLConnection.HTTP_NOT_FOUND, + AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND.getErrorCode(), null, + new Exception( + AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND.getErrorCode())); + } + if (srcEtag != null && !srcEtag.equals(pathInformation.getETag())) { + throw new AbfsRestOperationException( + HttpURLConnection.HTTP_CONFLICT, + AzureServiceErrorCode.PATH_ALREADY_EXISTS.getErrorCode(), null, + new Exception( + AzureServiceErrorCode.PATH_ALREADY_EXISTS.getErrorCode())); + } + } + + /** Validate if the destination path does not exist. + * + * @param src source path + * @param dst destination path + * @param pathInformation object containing the path information of the source path + * + * @throws AbfsRestOperationException if the destination path already exists + */ + private void validateDestinationPathNotExist(final Path src, + final Path dst, + final PathInformation pathInformation) + throws AzureBlobFileSystemException { + /* + * Destination path name can be same to that of source path name only in the + * case of a directory rename. + * + * In case the directory is being renamed to some other name, the destination + * check would happen on the AzureBlobFileSystem#rename method. + */ + if (pathInformation.getIsDirectory() && dst.getName() + .equals(src.getName())) { + PathInformation dstPathInformation = getPathInformation( + dst, + tracingContext); + if (dstPathInformation.getPathExists()) { + LOG.info( + "Rename src: {} dst: {} failed as qualifiedDst already exists", + src, dst); + throw new AbfsRestOperationException( + HttpURLConnection.HTTP_CONFLICT, + AzureServiceErrorCode.PATH_ALREADY_EXISTS.getErrorCode(), null, + null); + } + } + } + + /** Validate if the parent of the destination path exists. + * + * @param src source path + * @param dst destination path + * @param pathInformation object containing the path information of the source path + * + * @throws AbfsRestOperationException if the parent of the destination path does not exist + */ + private void validateDestinationParentExist(final Path src, + final Path dst, + final PathInformation pathInformation) + throws AzureBlobFileSystemException { + final Path nestedDstParent = dst.getParent(); + if (!dst.isRoot() && nestedDstParent != null && !nestedDstParent.isRoot() + && ( + !pathInformation.getIsDirectory() || !dst.getName() + .equals(src.getName()))) { + PathInformation nestedDstInfo = getPathInformation( + nestedDstParent, + tracingContext); + if (!nestedDstInfo.getPathExists() || !nestedDstInfo.getIsDirectory()) { + throw new AbfsRestOperationException( + HttpURLConnection.HTTP_NOT_FOUND, + RENAME_DESTINATION_PARENT_PATH_NOT_FOUND.getErrorCode(), null, + new Exception( + RENAME_DESTINATION_PARENT_PATH_NOT_FOUND.getErrorCode())); + } + } + } + + /** {@inheritDoc} */ + @Override + boolean takeAction(final Path path) throws AzureBlobFileSystemException { + return renameInternal(path, getDstPathForBlob(dst, path, src)); + } + + /** Renames the source path to the destination path. + * + * @param path source path + * @param destinationPathForBlobPartOfRenameSrcDir destination path + * + * @return true if rename is successful + * @throws AzureBlobFileSystemException if server call fails + */ + private boolean renameInternal(final Path path, + final Path destinationPathForBlobPartOfRenameSrcDir) + throws AzureBlobFileSystemException { + final String leaseId; + AbfsLease abfsLease = null; + if (isAtomicRename) { + /* + * To maintain atomicity of rename of the path, lease is taken on the path. + */ + if (path.equals(src)) { + abfsLease = srcAbfsLease; + leaseId = srcLeaseId; + } else { + abfsLease = takeLease(path, null); + leaseId = abfsLease.getLeaseID(); + } + } else { + leaseId = null; + } + boolean operated = false; + try { + copyPath(path, destinationPathForBlobPartOfRenameSrcDir, leaseId); + getAbfsClient().deleteBlobPath(path, leaseId, tracingContext); + operated = true; + } finally { + if (abfsLease != null) { + // If the operation is successful, cancel the timer and no need to release + // the lease as delete on the blob-path has taken place. + if (operated) { + abfsLease.cancelTimer(); + } else { + abfsLease.free(); + } + } + } + operatedBlobCount.incrementAndGet(); + return true; + } + + /** Copies the source path to the destination path. + * + * @param src source path + * @param dst destination path + * @param leaseId lease id for the source path + * + * @throws AzureBlobFileSystemException if server call fails + */ + private void copyPath(final Path src, final Path dst, final String leaseId) + throws AzureBlobFileSystemException { + String copyId; + try { + AbfsRestOperation copyPathOp = getAbfsClient().copyBlob(src, dst, leaseId, + tracingContext); + final String progress = copyPathOp.getResult() + .getResponseHeader(X_MS_COPY_STATUS); + if (COPY_STATUS_SUCCESS.equalsIgnoreCase(progress)) { + return; + } + copyId = copyPathOp.getResult() + .getResponseHeader(X_MS_COPY_ID); + } catch (AbfsRestOperationException ex) { + if (ex.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { + AbfsRestOperation dstPathStatus = getAbfsClient().getPathStatus( + dst.toUri().getPath(), + tracingContext, null, false); + final String srcCopyPath = ROOT_PATH + getAbfsClient().getFileSystem() + + src.toUri().getPath(); + if (dstPathStatus != null && dstPathStatus.getResult() != null + && (srcCopyPath.equals(getDstSource(dstPathStatus)))) { + return; + } + } + throw ex; + } + final long pollWait = getAbfsClient().getAbfsConfiguration() + .getBlobCopyProgressPollWaitMillis(); + final long maxWait = getAbfsClient().getAbfsConfiguration() + .getBlobCopyProgressMaxWaitMillis(); + long startTime = System.currentTimeMillis(); + while (handleCopyInProgress(dst, tracingContext, copyId) + == BlobCopyProgress.PENDING) { + if (System.currentTimeMillis() - startTime > maxWait) { + throw new TimeoutException( + "Blob copy progress wait time exceeded for source: " + + src + " and destination: " + dst); + } + try { + Thread.sleep(pollWait); + } catch (InterruptedException ignored) { + + } + } + } + + /** Gets the source path of the copy operation. + * + * @param dstPathStatus server response for the GetBlobProperties API on the + * destination path. + * + * @return source path of the copy operation + */ + private String getDstSource(final AbfsRestOperation dstPathStatus) { + try { + String responseHeader = dstPathStatus.getResult() + .getResponseHeader(X_MS_COPY_SOURCE); + if (responseHeader == null) { + return null; + } + return new URL(responseHeader).toURI().getPath(); + } catch (URISyntaxException | MalformedURLException e) { + throw new RuntimeException(e); + } + } + + /** + * Verifies if the blob copy is success or a failure or still in progress. + * + * @param dstPath path of the destination for the copying + * @param tracingContext object of tracingContext used for the tracing of the + * server calls. + * @param copyId id returned by server on the copy server-call. This id gets + * attached to blob and is returned by GetBlobProperties API on the destination. + * + * @return BlobCopyProgress indicating the status of the copy operation + * + * @throws AzureBlobFileSystemException exception returned in making server call + * for GetBlobProperties on the path. It can be thrown if the copyStatus is failure + * or is aborted. + */ + @VisibleForTesting + public BlobCopyProgress handleCopyInProgress(final Path dstPath, + final TracingContext tracingContext, + final String copyId) throws AzureBlobFileSystemException { + AbfsRestOperation op = getAbfsClient().getPathStatus(dstPath.toUri().getPath(), + tracingContext, null, false); + + if (op.getResult() != null && copyId != null + && copyId.equals(op.getResult().getResponseHeader(X_MS_COPY_ID))) { + final String copyStatus = op.getResult() + .getResponseHeader(X_MS_COPY_STATUS); + if (COPY_STATUS_SUCCESS.equalsIgnoreCase(copyStatus)) { + return BlobCopyProgress.SUCCESS; + } + if (COPY_STATUS_FAILED.equalsIgnoreCase(copyStatus)) { + throw new AbfsRestOperationException( + COPY_BLOB_FAILED.getStatusCode(), COPY_BLOB_FAILED.getErrorCode(), + String.format("copy to path %s failed due to: %s", + dstPath.toUri().getPath(), + op.getResult().getResponseHeader(X_MS_COPY_STATUS_DESCRIPTION)), + new Exception(COPY_BLOB_FAILED.getErrorCode())); + } + if (COPY_STATUS_ABORTED.equalsIgnoreCase(copyStatus)) { + throw new AbfsRestOperationException( + COPY_BLOB_ABORTED.getStatusCode(), COPY_BLOB_ABORTED.getErrorCode(), + String.format("copy to path %s aborted", dstPath.toUri().getPath()), + new Exception(COPY_BLOB_ABORTED.getErrorCode())); + } + } + return BlobCopyProgress.PENDING; + } + + /** + * Translates the destination path for a blob part of a source directory getting + * renamed. + * + * @param destinationDir destination directory for the rename operation + * @param blobPath path of blob inside sourceDir being renamed. + * @param sourceDir source directory for the rename operation + * + * @return translated path for the blob + */ + private Path getDstPathForBlob(final Path destinationDir, + final Path blobPath, final Path sourceDir) { + String destinationPathStr = destinationDir.toUri().getPath(); + String sourcePathStr = sourceDir.toUri().getPath(); + String srcBlobPropertyPathStr = blobPath.toUri().getPath(); + if (sourcePathStr.equals(srcBlobPropertyPathStr)) { + return destinationDir; + } + return new Path( + destinationPathStr + ROOT_PATH + srcBlobPropertyPathStr.substring( + sourcePathStr.length())); + } + + /** Get information of the path. + * + * @param path path for which the path information is to be fetched + * @param tracingContext object of tracingContext used for the tracing of the + * server calls. + * + * @return object containing the path information + * @throws AzureBlobFileSystemException if server call fails + */ + private PathInformation getPathInformation(Path path, + TracingContext tracingContext) + throws AzureBlobFileSystemException { + try { + AbfsRestOperation op = getAbfsClient().getPathStatus(path.toString(), + tracingContext, null, true); + + return new PathInformation(true, + getAbfsClient().checkIsDir(op.getResult()), + extractEtagHeader(op.getResult()), + op.getResult() instanceof AbfsHttpOperation.AbfsHttpOperationWithFixedResultForGetFileStatus); + } catch (AbfsRestOperationException e) { + if (e.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { + return new PathInformation(false, false, null, false); + } + throw e; + } + } + + @VisibleForTesting + public List getLeases() { + return leases; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListActionTaker.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListActionTaker.java new file mode 100644 index 0000000000000..0bfd25073b8b2 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListActionTaker.java @@ -0,0 +1,269 @@ +/** + * 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.services; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; +import org.apache.hadoop.fs.azurebfs.contracts.services.BlobListResultSchema; +import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema; +import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ROOT_PATH; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS; + +/** + * ListActionTaker is an abstract class that provides a way to list the paths + * recursively and take action on each path. The implementations of this class + * should provide the action to be taken on each listed path. + */ +public abstract class ListActionTaker { + + private static final Logger LOG = LoggerFactory.getLogger(ListActionTaker.class); + + private final Path path; + + private final AbfsBlobClient abfsClient; + + private final TracingContext tracingContext; + + private final ExecutorService executorService; + + private final AtomicBoolean producerThreadToBeStopped = new AtomicBoolean( + false); + + /** Constructor. + * + * @param path the path to list recursively. + * @param abfsClient the AbfsBlobClient to use for listing. + * @param tracingContext the tracing context to use for listing. + */ + public ListActionTaker(Path path, + AbfsBlobClient abfsClient, + TracingContext tracingContext) { + this.path = path; + this.abfsClient = abfsClient; + this.tracingContext = tracingContext; + executorService = Executors.newFixedThreadPool( + getMaxConsumptionParallelism()); + } + + public AbfsBlobClient getAbfsClient() { + return abfsClient; + } + + /** Get the maximum number of parallelism for consumption. + * + * @return the maximum number of parallelism for consumption. + */ + abstract int getMaxConsumptionParallelism(); + + /** Take action on a path. + * + * @param path the path to take action on. + * @return true if the action is successful. + * @throws AzureBlobFileSystemException if the action fails. + */ + abstract boolean takeAction(Path path) throws AzureBlobFileSystemException; + + private boolean takeAction(List paths) + throws AzureBlobFileSystemException { + List> futureList = new ArrayList<>(); + for (Path path : paths) { + Future future = executorService.submit(() -> takeAction(path)); + futureList.add(future); + } + + AzureBlobFileSystemException executionException = null; + boolean actionResult = true; + for (Future future : futureList) { + try { + Boolean result = future.get(); + if (!result) { + actionResult = false; + } + } catch (InterruptedException e) { + LOG.debug("Thread interrupted while taking action on path: {}", + path.toUri().getPath()); + } catch (ExecutionException e) { + executionException = (AzureBlobFileSystemException) e.getCause(); + } + } + if (executionException != null) { + throw executionException; + } + return actionResult; + } + + /** + * Spawns a producer thread that list the children of the path recursively and queue + * them in into {@link ListBlobQueue}. On the main thread, it dequeues the + * path and supply them to parallel thread for relevant action which is defined + * in {@link #takeAction(Path)}. + * + * @return true if the action is successful. + * @throws AzureBlobFileSystemException if the action fails. + */ + public boolean listRecursiveAndTakeAction() + throws AzureBlobFileSystemException { + AbfsConfiguration configuration = getAbfsClient().getAbfsConfiguration(); + Thread producerThread = null; + try { + ListBlobQueue listBlobQueue = createListBlobQueue(configuration); + producerThread = new Thread(() -> { + try { + produceConsumableList(listBlobQueue); + } catch (AzureBlobFileSystemException e) { + listBlobQueue.markProducerFailure(e); + } + }); + producerThread.start(); + + while (!listBlobQueue.getIsCompleted()) { + List paths = listBlobQueue.consume(); + if (paths == null) { + continue; + } + try { + boolean resultOnPartAction = takeAction(paths); + if (!resultOnPartAction) { + return false; + } + } catch (AzureBlobFileSystemException parallelConsumptionException) { + listBlobQueue.markConsumptionFailed(); + throw parallelConsumptionException; + } + } + return true; + } finally { + if (producerThread != null) { + producerThreadToBeStopped.set(true); + } + executorService.shutdownNow(); + } + } + + /** + * Create a {@link ListBlobQueue} instance. + * + * @param configuration the configuration to use. + * @return the created {@link ListBlobQueue} instance. + * @throws InvalidConfigurationValueException if the configuration is invalid. + */ + @VisibleForTesting + protected ListBlobQueue createListBlobQueue(final AbfsConfiguration configuration) + throws InvalidConfigurationValueException { + return new ListBlobQueue( + configuration.getProducerQueueMaxSize(), + getMaxConsumptionParallelism(), + configuration.getListingMaxConsumptionLag() + ); + } + + /** + * Produce the consumable list of paths. + * + * @param listBlobQueue the {@link ListBlobQueue} to enqueue the paths. + * @throws AzureBlobFileSystemException if the listing fails. + */ + private void produceConsumableList(final ListBlobQueue listBlobQueue) + throws AzureBlobFileSystemException { + String continuationToken = null; + do { + continuationToken = listAndEnqueue(listBlobQueue, continuationToken); + } while (!producerThreadToBeStopped.get() && continuationToken != null + && !listBlobQueue.getConsumptionFailed()); + listBlobQueue.complete(); + } + + /** + * List the paths and enqueue them into the {@link ListBlobQueue}. + * + * @param listBlobQueue the {@link ListBlobQueue} to enqueue the paths. + * @param continuationToken the continuation token to use for listing. + * @return the continuation token for the next listing. + * @throws AzureBlobFileSystemException if the listing fails. + */ + @VisibleForTesting + protected String listAndEnqueue(final ListBlobQueue listBlobQueue, + String continuationToken) throws AzureBlobFileSystemException { + final int queueAvailableSizeForProduction = Math.min( + DEFAULT_AZURE_LIST_MAX_RESULTS, + listBlobQueue.availableSizeForProduction()); + if (queueAvailableSizeForProduction == 0) { + return null; + } + final AbfsRestOperation op; + try { + op = getAbfsClient().listPath(path.toUri().getPath(), + true, + queueAvailableSizeForProduction, continuationToken, + tracingContext); + } catch (AzureBlobFileSystemException ex) { + throw ex; + } catch (IOException ex) { + throw new AbfsRestOperationException(-1, null, + "Unknown exception from listing: " + ex.getMessage(), ex); + } + + ListResultSchema retrievedSchema = op.getResult().getListResultSchema(); + if (retrievedSchema == null) { + return continuationToken; + } + continuationToken + = ((BlobListResultSchema) retrievedSchema).getNextMarker(); + List paths = new ArrayList<>(); + addPaths(paths, retrievedSchema); + listBlobQueue.enqueue(paths); + return continuationToken; + } + + /** + * Add the paths from the retrieved schema to the list of paths. + * + * @param paths the list of paths to add to. + * @param retrievedSchema the retrieved schema. + */ + @VisibleForTesting + protected void addPaths(final List paths, + final ListResultSchema retrievedSchema) { + for (ListResultEntrySchema entry : retrievedSchema.paths()) { + Path entryPath = new Path(ROOT_PATH, entry.name()); + if (!entryPath.equals(this.path)) { + paths.add(entryPath); + } + } + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobQueue.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobQueue.java new file mode 100644 index 0000000000000..871ae14c91f00 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobQueue.java @@ -0,0 +1,197 @@ +/** + * 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.services; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.List; +import java.util.Queue; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException; + +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CONSUMER_MAX_LAG; + +/** + * Data-structure to hold the list of paths to be processed. The paths are + * enqueued by the producer and dequeued by the consumer. The producer can + * enqueue the paths until the queue is full. The consumer can consume the paths + * until the queue is empty. The producer can mark the queue as completed once + * all the paths are enqueued and there is no more paths that can be returned from + * server. The consumer can mark the queue as failed if it encounters any exception + * while consuming the paths. + */ +class ListBlobQueue { + + private final Queue pathQueue = new ArrayDeque<>(); + + private final int maxSize; + + private final int consumeSetSize; + + private volatile boolean isCompleted = false; + + private volatile boolean isConsumptionFailed = false; + + private volatile AzureBlobFileSystemException failureFromProducer; + + /** + * Maximum number of entries in the queue allowed for letting the producer to + * produce. If the current size of the queue is greater than or equal to + * maxConsumptionLag, the producer will wait until the current size of the queue + * becomes lesser than maxConsumptionLag. This parameter is used to control the + * behavior of the producer-consumer pattern and preventing producer from + * rapidly producing very small amount of items. + *

+ * For example, let's say maxSize is 10000 and maxConsumptionLag is 5000. + * The producer will stop producing when the current size of the queue is 5000 + * and will wait until the current size of the queue becomes lesser than 5000. + * Once, the size becomes lesser than 5000, producer can produce (maxSize - currentSize) + * of items, which would make the current size of the queue to be 10000. Then again + * it will wait for 5000 items to be consumed before generating next 5000 items. + *

+ * If this is not used, the producer will keep on producing items as soon as + * the queue become available with small size. Let say, 5 items got consumed, + * producer would make a server call for only 5 items and would populate the queue. + *

+ * This mechanism would prevent producer making server calls for very small amount + * of items. + */ + private final int maxConsumptionLag; + + ListBlobQueue(int maxSize, int consumeSetSize, int maxConsumptionLag) + throws InvalidConfigurationValueException { + this.maxSize = maxSize; + this.maxConsumptionLag = maxConsumptionLag; + this.consumeSetSize = consumeSetSize; + + if (maxConsumptionLag >= maxSize) { + throw new InvalidConfigurationValueException(FS_AZURE_CONSUMER_MAX_LAG, + "maxConsumptionLag should be lesser than maxSize"); + } + } + + /** Mark the queue as failed.*/ + void markProducerFailure(AzureBlobFileSystemException failure) { + failureFromProducer = failure; + } + + /** Mark the queue as completed.*/ + void complete() { + isCompleted = true; + } + + /** Mark the consumption as failed.*/ + synchronized void markConsumptionFailed() { + isConsumptionFailed = true; + notify(); + } + + /** Check if the consumption has failed. + * + * @return true if the consumption has failed + */ + boolean getConsumptionFailed() { + return isConsumptionFailed; + } + + /** Check if the queue is completed. + * + * @return true if the queue is completed + */ + boolean getIsCompleted() { + return isCompleted && size() == 0; + } + + /** Get the exception from producer. + * + * @return exception from producer + */ + private AzureBlobFileSystemException getException() { + return failureFromProducer; + } + + /** Enqueue the paths. + * + * @param pathList list of paths to be enqueued + */ + synchronized void enqueue(List pathList) { + if (isCompleted) { + throw new IllegalStateException( + "Cannot enqueue paths as the queue is already marked as completed"); + } + pathQueue.addAll(pathList); + } + + /** Consume the paths. + * + * @return list of paths to be consumed + * @throws AzureBlobFileSystemException if the consumption fails + */ + synchronized List consume() throws AzureBlobFileSystemException { + AzureBlobFileSystemException exception = getException(); + if (exception != null) { + throw exception; + } + return dequeue(); + } + + /** Dequeue the paths. + * + * @return list of paths to be consumed + */ + private List dequeue() { + List pathListForConsumption = new ArrayList<>(); + int counter = 0; + while (counter < consumeSetSize && !pathQueue.isEmpty()) { + pathListForConsumption.add(pathQueue.poll()); + counter++; + } + if (counter > 0) { + notify(); + } + return pathListForConsumption; + } + + private synchronized int size() { + return pathQueue.size(); + } + + /** + * Returns the available size of the queue for production. This is calculated by subtracting + * the current size of the queue from its maximum size. This method waits until + * the current size of the queue becomes lesser than the maxConsumptionLag. This + * method is synchronized to prevent concurrent modifications of the queue. + * + * @return the available size of the queue. + */ + synchronized int availableSizeForProduction() { + while (size() >= maxConsumptionLag) { + if (isConsumptionFailed) { + return 0; + } + try { + wait(); + } catch (InterruptedException ignored) { + } + } + return maxSize - size(); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/PathInformation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/PathInformation.java new file mode 100644 index 0000000000000..26424582a3d83 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/PathInformation.java @@ -0,0 +1,101 @@ +/** + * 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.services; + +/** + * Information about a path. + */ +public class PathInformation { + private Boolean pathExists; + private Boolean isDirectory; + private Boolean isImplicit; + private String eTag; + + /** + * Constructor. + * @param pathExists The path exists. + * @param isDirectory Is the path a directory? + * @param eTag The ETag of the path. + * @param isImplicit Is the path implicit? + */ + public PathInformation(Boolean pathExists, Boolean isDirectory, String eTag, Boolean isImplicit) { + this.pathExists = pathExists; + this.isDirectory = isDirectory; + this.eTag = eTag; + this.isImplicit = isImplicit; + } + + public PathInformation() { + } + + /** + * Copy the path information. + * @param pathInformation The path information to copy. + */ + public void copy(PathInformation pathInformation) { + this.pathExists = pathInformation.getPathExists(); + this.isDirectory = pathInformation.getIsDirectory(); + this.eTag = pathInformation.getETag(); + this.isImplicit = pathInformation.getIsImplicit(); + } + + /** + * Get the ETag of the path. + * + * @return the etag value*/ + public String getETag() { + return eTag; + } + + /** + * Get value of pathExists. + * + * @return true if path exists, false otherwise. + */ + public Boolean getPathExists() { + return pathExists; + } + + /** + * Get value of isDirectory. + * + * @return true if path is a directory, false otherwise. + */ + public Boolean getIsDirectory() { + return isDirectory; + } + + /** + * Get value of isImplicit. + * + * @return true if path is implicit, false otherwise. + */ + public Boolean getIsImplicit() { + return isImplicit; + } + + /** + * Set the eTag value. + * + * @param eTag The eTag value to set. + */ + void setETag(String eTag) { + this.eTag = eTag; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java new file mode 100644 index 0000000000000..a702b7f84a446 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicity.java @@ -0,0 +1,337 @@ +/** + * 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.services; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; + +//import java.util.ArrayList; +//import java.util.Collections; +//import java.util.List; +import java.util.Random; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.commons.codec.binary.Base64; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsDriverException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; +import org.apache.hadoop.fs.azurebfs.contracts.services.BlobAppendRequestParameters; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +import static java.net.HttpURLConnection.HTTP_NOT_FOUND; +import static java.net.HttpURLConnection.HTTP_PRECON_FAILED; +import static org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.extractEtagHeader; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_ID_LENGTH; +//import static org.apache.hadoop.fs.azurebfs.services.AzureIngressHandler.generateBlockListXml; + +/** + * For a directory enabled for atomic-rename, before rename starts, a file with + * -RenamePending.json suffix is created. In this file, the states required for the + * rename operation are given. This file is created by {@link #preRename()} method. + * This is important in case the JVM process crashes during rename, the atomicity + * will be maintained, when the job calls {@link AzureBlobFileSystem#listStatus(Path)} + * or {@link AzureBlobFileSystem#getFileStatus(Path)}. On these API calls to filesystem, + * it will be checked if there is any RenamePending JSON file. If yes, the crashed rename + * operation would be resumed as per the file. + */ +public class RenameAtomicity { + + private final TracingContext tracingContext; + + private Path src, dst; + + private String srcEtag; + + private final AbfsBlobClient abfsClient; + + private final Path renameJsonPath; + + public static final String SUFFIX = "-RenamePending.json"; + + private int preRenameRetryCount = 0; + + private int renamePendingJsonLen; + + private final ObjectMapper objectMapper = new ObjectMapper(); + + private static final Random RANDOM = new Random(); + + /** + * Performs pre-rename operations. Creates a file with -RenamePending.json + * suffix in the source parent directory. This file contains the states + * required for the rename operation. + * + * @param src Source path + * @param dst Destination path + * @param renameJsonPath Path of the JSON file to be created + * @param tracingContext Tracing context + * @param srcEtag ETag of the source directory + * @param abfsClient AbfsClient instance + */ + public RenameAtomicity(final Path src, final Path dst, + final Path renameJsonPath, + TracingContext tracingContext, + final String srcEtag, + final AbfsClient abfsClient) { + this.src = src; + this.dst = dst; + this.abfsClient = (AbfsBlobClient) abfsClient; + this.renameJsonPath = renameJsonPath; + this.tracingContext = tracingContext; + this.srcEtag = srcEtag; + } + + /** + * Resumes the rename operation from the JSON file. + * + * @param renameJsonPath Path of the JSON file + * @param renamePendingJsonFileLen Length of the JSON file + * @param tracingContext Tracing context + * @param srcEtag ETag of the source directory + * @param abfsClient AbfsClient instance + */ + public RenameAtomicity(final Path renameJsonPath, + final int renamePendingJsonFileLen, + TracingContext tracingContext, + final String srcEtag, + final AbfsClient abfsClient) { + this.abfsClient = (AbfsBlobClient) abfsClient; + this.renameJsonPath = renameJsonPath; + this.tracingContext = tracingContext; + this.srcEtag = srcEtag; + this.renamePendingJsonLen = renamePendingJsonFileLen; + } + + /** + * Redo the rename operation from the JSON file. + * + * @throws AzureBlobFileSystemException If the redo operation fails. + */ + public void redo() throws AzureBlobFileSystemException { + byte[] buffer = readRenamePendingJson(renameJsonPath, renamePendingJsonLen); + String contents = new String(buffer, Charset.defaultCharset()); + try { + final RenamePendingJsonFormat renamePendingJsonFormatObj; + try { + renamePendingJsonFormatObj = objectMapper.readValue(contents, + RenamePendingJsonFormat.class); + } catch (JsonProcessingException e) { + return; + } + if (renamePendingJsonFormatObj != null && StringUtils.isNotEmpty( + renamePendingJsonFormatObj.getOldFolderName()) + && StringUtils.isNotEmpty( + renamePendingJsonFormatObj.getNewFolderName()) + && StringUtils.isNotEmpty(renamePendingJsonFormatObj.getETag())) { + this.src = new Path(renamePendingJsonFormatObj.getOldFolderName()); + this.dst = new Path(renamePendingJsonFormatObj.getNewFolderName()); + this.srcEtag = renamePendingJsonFormatObj.getETag(); + + BlobRenameHandler blobRenameHandler = new BlobRenameHandler( + this.src.toUri().getPath(), dst.toUri().getPath(), + abfsClient, srcEtag, true, + true, tracingContext); + + blobRenameHandler.execute(); + } + } finally { + deleteRenamePendingJson(); + } + } + + /** Read the JSON file. + * + * @param path Path of the JSON file + * @param len Length of the JSON file + * @return Contents of the JSON file + * @throws AzureBlobFileSystemException If the read operation fails. + */ + @VisibleForTesting + byte[] readRenamePendingJson(Path path, int len) + throws AzureBlobFileSystemException { + byte[] bytes = new byte[len]; + abfsClient.read(path.toUri().getPath(), 0, bytes, 0, + len, null, null, null, + tracingContext); + return bytes; + } + + /** Generate a random block ID. + * + * @return Random block ID + */ + public static String generateBlockId() { + // PutBlock on the path. + byte[] blockIdByteArray = new byte[BLOCK_ID_LENGTH]; + RANDOM.nextBytes(blockIdByteArray); + return new String(Base64.encodeBase64(blockIdByteArray), + StandardCharsets.UTF_8); + } + + /** Create the JSON file with the contents. + * + * @param path Path of the JSON file + * @param bytes Contents of the JSON file + * @throws AzureBlobFileSystemException If the create operation fails. + */ + @VisibleForTesting + void createRenamePendingJson(Path path, byte[] bytes) + throws AzureBlobFileSystemException { + // PutBlob on the path. + AbfsRestOperation putBlobOp = abfsClient.createPath(path.toUri().getPath(), + true, + true, + null, + false, + null, + null, + tracingContext); + String eTag = extractEtagHeader(putBlobOp.getResult()); + + String blockId = generateBlockId(); + AppendRequestParameters appendRequestParameters + = new AppendRequestParameters(0, 0, + bytes.length, AppendRequestParameters.Mode.APPEND_MODE, false, null, + abfsClient.getAbfsConfiguration().isExpectHeaderEnabled(), + new BlobAppendRequestParameters(blockId, eTag)); + + abfsClient.append(path.toUri().getPath(), bytes, + appendRequestParameters, null, null, tracingContext); + +// List blockIdList = new ArrayList<>(Collections.singleton(blockId)); +// String blockList = generateBlockListXml(blockIdList); + // PutBlockList on the path. + String blockList = ""; + abfsClient.flush(blockList.getBytes(StandardCharsets.UTF_8), + path.toUri().getPath(), true, null, null, eTag, null, tracingContext); + } + + /** + * Before starting the atomic rename, create a file with -RenamePending.json + * suffix in the source parent directory. This file contains the states + * required source, destination, and source-eTag for the rename operation. + *

+ * If the path that is getting renamed is a /sourcePath, then the JSON file + * will be /sourcePath-RenamePending.json. + * + * @return Length of the JSON file. + * @throws AzureBlobFileSystemException If the pre-rename operation fails. + */ + @VisibleForTesting + public int preRename() throws AzureBlobFileSystemException { + String makeRenamePendingFileContents = makeRenamePendingFileContents( + srcEtag); + + try { + createRenamePendingJson(renameJsonPath, + makeRenamePendingFileContents.getBytes(StandardCharsets.UTF_8)); + return makeRenamePendingFileContents.length(); + } catch (AzureBlobFileSystemException e) { + /* + * Scenario: file has been deleted by parallel thread before the RenameJSON + * could be written and flushed. In such case, there has to be one retry of + * preRename. + * ref: https://issues.apache.org/jira/browse/HADOOP-12678 + * On DFS endpoint, flush API is called. If file is not there, server returns + * 404. + * On blob endpoint, flush API is not there. PutBlockList is called with + * if-match header. If file is not there, the conditional header will fail, + * the server will return 412. + */ + if (isPreRenameRetriableException(e)) { + preRenameRetryCount++; + if (preRenameRetryCount == 1) { + return preRename(); + } + } + throw e; + } + } + + /** Check if the exception is retryable for pre-rename operation. + * + * @param e Exception to be checked + * @return true if the exception is retryable, false otherwise + */ + private boolean isPreRenameRetriableException(IOException e) { + AbfsRestOperationException ex; + while (e != null) { + if (e instanceof AbfsRestOperationException) { + ex = (AbfsRestOperationException) e; + return ex.getStatusCode() == HTTP_NOT_FOUND + || ex.getStatusCode() == HTTP_PRECON_FAILED; + } + e = (IOException) e.getCause(); + } + return false; + } + + /** Delete the JSON file after rename is done. + * @throws AzureBlobFileSystemException If the delete operation fails. + */ + public void postRename() throws AzureBlobFileSystemException { + deleteRenamePendingJson(); + } + + /** Delete the JSON file. + * + * @throws AzureBlobFileSystemException If the delete operation fails. + */ + private void deleteRenamePendingJson() throws AzureBlobFileSystemException { + try { + abfsClient.deleteBlobPath(renameJsonPath, null, + tracingContext); + } catch (AzureBlobFileSystemException e) { + if (e instanceof AbfsRestOperationException + && ((AbfsRestOperationException) e).getStatusCode() + == HTTP_NOT_FOUND) { + return; + } + throw e; + } + } + + /** + * Return the contents of the JSON file to represent the operations + * to be performed for a folder rename. + * + * @return JSON string which represents the operation. + */ + private String makeRenamePendingFileContents(String eTag) throws + AzureBlobFileSystemException { + + final RenamePendingJsonFormat renamePendingJsonFormat = new RenamePendingJsonFormat(); + renamePendingJsonFormat.setOldFolderName(src.toUri().getPath()); + renamePendingJsonFormat.setNewFolderName(dst.toUri().getPath()); + renamePendingJsonFormat.setETag(eTag); + try { + return objectMapper.writeValueAsString(renamePendingJsonFormat); + } catch (JsonProcessingException e) { + throw new AbfsDriverException(e); + } + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenamePendingJsonFormat.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenamePendingJsonFormat.java new file mode 100644 index 0000000000000..564998c469f7a --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenamePendingJsonFormat.java @@ -0,0 +1,57 @@ +/** + * 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.services; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class RenamePendingJsonFormat { + @JsonProperty(value = "OldFolderName") + private String oldFolderName; + + @JsonProperty(value = "NewFolderName") + private String newFolderName; + + @JsonProperty(value = "ETag") + private String eTag; + + /** Getters and Setters */ + String getOldFolderName() { + return oldFolderName; + } + + String getNewFolderName() { + return newFolderName; + } + + String getETag() { + return eTag; + } + + void setOldFolderName(String oldFolderName) { + this.oldFolderName = oldFolderName; + } + + void setNewFolderName(String newFolderName) { + this.newFolderName = newFolderName; + } + + void setETag(String eTag) { + this.eTag = eTag; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java index a349894cd4f30..1399833e74187 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java @@ -76,6 +76,8 @@ public class TracingContext { */ private String primaryRequestIdForRetry; + private Integer operatedBlobCount = null; + private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); public static final int MAX_CLIENT_CORRELATION_ID_LENGTH = 72; public static final String CLIENT_CORRELATION_ID_PATTERN = "[a-zA-Z0-9-]*"; @@ -131,6 +133,7 @@ public TracingContext(TracingContext originalTracingContext) { this.retryCount = 0; this.primaryRequestId = originalTracingContext.primaryRequestId; this.format = originalTracingContext.format; + this.operatedBlobCount = originalTracingContext.operatedBlobCount; if (originalTracingContext.listener != null) { this.listener = originalTracingContext.listener.getClone(); } @@ -194,6 +197,9 @@ public void constructHeader(AbfsHttpOperation httpOperation, String previousFail header = addFailureReasons(header, previousFailure, retryPolicyAbbreviation); header += (":" + httpOperation.getTracingContextSuffix()); metricHeader += !(metricResults.trim().isEmpty()) ? metricResults : ""; + if (operatedBlobCount != null) { + header += (":" + operatedBlobCount); + } break; case TWO_ID_FORMAT: header = clientCorrelationID + ":" + clientRequestId; @@ -249,6 +255,14 @@ private String addFailureReasons(final String header, return String.format("%s_%s", header, previousFailure); } + public void setOperatedBlobCount(Integer count) { + operatedBlobCount = count; + } + + public FSOperationType getOpType() { + return opType; + } + /** * Return header representing the request associated with the tracingContext * @return Header string set into X_MS_CLIENT_REQUEST_ID diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index fdc7e0a3dafe1..3e2c34e6ec795 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -19,6 +19,8 @@ package org.apache.hadoop.fs.azurebfs; import java.io.FileNotFoundException; +import java.io.IOException; +import java.nio.file.AccessDeniedException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.Callable; @@ -27,7 +29,6 @@ import java.util.concurrent.Future; import org.assertj.core.api.Assertions; -import org.junit.Assume; import org.junit.Test; import org.mockito.Mockito; @@ -35,6 +36,9 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter; +import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsDfsClient; import org.apache.hadoop.fs.azurebfs.contracts.services.StorageErrorResponseSchema; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil; @@ -44,21 +48,24 @@ import org.apache.hadoop.fs.azurebfs.services.TestAbfsPerfTracker; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; -import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.test.ReflectionUtils; +import org.apache.hadoop.test.LambdaTestUtils; import static java.net.HttpURLConnection.HTTP_BAD_REQUEST; import static java.net.HttpURLConnection.HTTP_NOT_FOUND; import static java.net.HttpURLConnection.HTTP_OK; +import static java.net.HttpURLConnection.HTTP_FORBIDDEN; +import static org.junit.Assume.assumeTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doCallRealMethod; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ROOT_PATH; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_DELETE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_DELETE_CONSIDERED_IDEMPOTENT; import static org.apache.hadoop.fs.azurebfs.services.AbfsRestOperationType.DeletePath; @@ -74,6 +81,8 @@ public class ITestAzureBlobFileSystemDelete extends private static final int REDUCED_RETRY_COUNT = 1; private static final int REDUCED_MAX_BACKOFF_INTERVALS_MS = 5000; + private static final int MAX_ITERATIONS = 20; + private static final int BLOB_COUNT = 11; public ITestAzureBlobFileSystemDelete() throws Exception { super(); @@ -161,7 +170,7 @@ public Void call() throws Exception { fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), fs.getFileSystemId(), FSOperationType.DELETE, false, 0)); // first try a non-recursive delete, expect failure - intercept(FileAlreadyExistsException.class, + intercept(IOException.class, () -> fs.delete(dir, false)); fs.registerListener(null); assertDeleted(fs, dir, true); @@ -171,7 +180,7 @@ public Void call() throws Exception { @Test public void testDeleteIdempotency() throws Exception { - Assume.assumeTrue(DEFAULT_DELETE_CONSIDERED_IDEMPOTENT); + assumeTrue(DEFAULT_DELETE_CONSIDERED_IDEMPOTENT); // Config to reduce the retry and maxBackoff time for test run AbfsConfiguration abfsConfig = TestAbfsConfigurationFieldsValidation.updateRetryConfigs( @@ -227,6 +236,13 @@ public void testDeleteIdempotency() throws Exception { public void testDeleteIdempotencyTriggerHttp404() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); + /* + * Delete call for a folder on DFS endpoint is one-server call and the + * orchestration of delete of paths inside the directory. For Blob + * endpoint, the orchestration would be done by the client. The idempotency + * issue would not happen for blob endpoint. + */ + assertTrue(fs.getAbfsClient() instanceof AbfsDfsClient); AbfsClient client = ITestAbfsClient.createTestClientFromCurrentContext( fs.getAbfsStore().getClient(), this.getConfiguration()); @@ -274,6 +290,23 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { doReturn(idempotencyRetOp).when(mockClient).deleteIdempotencyCheckOp(any()); TracingContext tracingContext = getTestTracingContext(fs, false); doReturn(tracingContext).when(idempotencyRetOp).createNewTracingContext(any()); + if (mockClient instanceof AbfsBlobClient) { + doCallRealMethod().when((AbfsBlobClient) mockClient) + .getBlobDeleteHandler(Mockito.nullable(String.class), + Mockito.anyBoolean(), Mockito.nullable(TracingContext.class)); + doCallRealMethod().when(mockClient) + .listPath(Mockito.nullable(String.class), Mockito.anyBoolean(), + Mockito.anyInt(), Mockito.nullable(String.class), + Mockito.nullable(TracingContext.class)); + doCallRealMethod().when((AbfsBlobClient) mockClient) + .listPath(Mockito.nullable(String.class), Mockito.anyBoolean(), + Mockito.anyInt(), Mockito.nullable(String.class), + Mockito.nullable(TracingContext.class), + Mockito.anyBoolean()); + doCallRealMethod().when((AbfsBlobClient) mockClient) + .getPathStatus(Mockito.nullable(String.class), Mockito.nullable(TracingContext.class), + Mockito.nullable(ContextEncryptionAdapter.class), Mockito.anyBoolean()); + } when(mockClient.deletePath("/NonExistingPath", false, null, tracingContext, fs.getIsNamespaceEnabled(tracingContext))) .thenCallRealMethod(); @@ -314,4 +347,281 @@ public void deleteBlobDirParallelThreadToDeleteOnDifferentTracingContext() fs.delete(new Path("/testDir"), true); fs.close(); } + + /** + * Assumes that the AzureBlobFileSystem's client is an instance of `AbfsBlobClient`. + * This assumption is used to ensure that the test runs only if the correct client type is present. + * + * @throws IOException if the file system client cannot be retrieved + */ + private void assumeBlobClient() throws IOException { + assumeTrue(getFileSystem().getAbfsClient() instanceof AbfsBlobClient); + } + + /** + * Tests deleting an implicit directory and its contents. The test verifies that after deletion, + * both the directory and its child file no longer exist. + * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testDeleteImplicitDir() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assumeBlobClient(); + fs.mkdirs(new Path("/testDir/dir1")); + fs.create(new Path("/testDir/dir1/file1")); + AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient(); + client.deleteBlobPath(new Path("/testDir/dir1"), + null, getTestTracingContext(fs, true)); + fs.delete(new Path("/testDir/dir1"), true); + Assertions.assertThat(!fs.exists(new Path("/testDir/dir1"))) + .describedAs("FileStatus of the deleted directory should not exist") + .isTrue(); + Assertions.assertThat(!fs.exists(new Path("/testDir/dir1/file1"))) + .describedAs("Child of a deleted directory should not be present") + .isTrue(); + } + + /** + * Tests deleting an implicit directory when a single list result is returned. + * The test verifies that the directory is properly deleted and no residual file status remains. + * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testDeleteImplicitDirWithSingleListResults() throws Exception { + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance( + getRawConfiguration()); + assumeBlobClient(); + AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient(); + AbfsBlobClient spiedClient = Mockito.spy(client); + fs.getAbfsStore().setClient(spiedClient); + fs.mkdirs(new Path("/testDir/dir1")); + for (int i = 0; i < 10; i++) { + fs.create(new Path("/testDir/dir1/file" + i)); + } + Mockito.doAnswer(answer -> { + String path = answer.getArgument(0); + boolean recursive = answer.getArgument(1); + String continuation = answer.getArgument(3); + TracingContext context = answer.getArgument(4); + return client.listPath(path, recursive, 1, continuation, context); + }) + .when(spiedClient) + .listPath(Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyInt(), + Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + client.deleteBlobPath(new Path("/testDir/dir1"), + null, getTestTracingContext(fs, true)); + fs.delete(new Path("/testDir/dir1"), true); + Assertions.assertThat(fs.exists(new Path("/testDir/dir1"))) + .describedAs("FileStatus of the deleted directory should not exist") + .isFalse(); + } + + /** + * Tests deleting an explicit directory within an implicit parent directory. + * It verifies that the directory and its contents are deleted, while the parent directory remains. + * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testDeleteExplicitDirInImplicitParentDir() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assumeBlobClient(); + AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient(); + fs.mkdirs(new Path("/testDir/dir1")); + fs.create(new Path("/testDir/dir1/file1")); + client.deleteBlobPath(new Path("/testDir/"), + null, getTestTracingContext(fs, true)); + fs.delete(new Path("/testDir/dir1"), true); + Assertions.assertThat(fs.exists(new Path("/testDir/dir1"))) + .describedAs("Deleted directory should not exist") + .isFalse(); + Assertions.assertThat(fs.exists(new Path("/testDir/dir1/file1"))) + .describedAs("Child of a deleted directory should not be present") + .isFalse(); + Assertions.assertThat(fs.exists(new Path("/testDir"))) + .describedAs("Parent Implicit directory should exist") + .isTrue(); + } + + /** + * Tests handling of a parallel delete operation failure when deleting multiple files in a directory. + * The test verifies that an `AccessDeniedException` is thrown when the delete operation fails + * due to an `AbfsRestOperationException`. + * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testDeleteParallelBlobFailure() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + assumeBlobClient(); + AbfsBlobClient client = Mockito.spy((AbfsBlobClient) fs.getAbfsClient()); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + store.setClient(client); + Mockito.doReturn(store).when(fs).getAbfsStore(); + fs.mkdirs(new Path("/testDir")); + fs.create(new Path("/testDir/file1")); + fs.create(new Path("/testDir/file2")); + fs.create(new Path("/testDir/file3")); + Mockito.doThrow( + new AbfsRestOperationException(HTTP_FORBIDDEN, "", "", new Exception())) + .when(client) + .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + LambdaTestUtils.intercept( + AccessDeniedException.class, + () -> { + fs.delete(new Path("/testDir"), true); + }); + } + + /** + * Tests deleting the root directory without recursion. The test verifies that + * the delete operation returns `false` since the root directory cannot be deleted + * without recursion. + * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testDeleteRootWithNonRecursion() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + fs.mkdirs(new Path("/testDir")); + Assertions.assertThat(fs.delete(new Path(ROOT_PATH), false)).isFalse(); + } + + /** + * Tests that the producer stops when a delete operation fails with an `AccessDeniedException`. + * The test simulates a failure during a delete operation and verifies that the system properly + * handles the exception by stopping further actions. + * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testProducerStopOnDeleteFailure() throws Exception { + assumeBlobClient(); + Configuration configuration = Mockito.spy(getRawConfiguration()); + AzureBlobFileSystem fs = Mockito.spy( + (AzureBlobFileSystem) FileSystem.get(configuration)); + fs.mkdirs(new Path("/src")); + ExecutorService executorService = Executors.newFixedThreadPool(10); + List futureList = new ArrayList<>(); + for (int i = 0; i < MAX_ITERATIONS; i++) { + int iter = i; + Future future = executorService.submit(() -> { + try { + fs.create(new Path("/src/file" + iter)); + } catch (IOException ex) {} + }); + futureList.add(future); + } + for (Future future : futureList) { + future.get(); + } + AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient(); + AbfsBlobClient spiedClient = Mockito.spy(client); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + store.setClient(spiedClient); + Mockito.doReturn(store).when(fs).getAbfsStore(); + final int[] deleteCallInvocation = new int[1]; + Mockito.doAnswer(answer -> { + throw new AbfsRestOperationException(HTTP_FORBIDDEN, "", "", + new Exception()); + }).when(spiedClient) + .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + AbfsClientTestUtil.mockGetDeleteBlobHandler(spiedClient, + (blobDeleteHandler) -> { + Mockito.doAnswer(answer -> { + try { + answer.callRealMethod(); + } catch (AbfsRestOperationException ex) { + if (ex.getStatusCode() == HTTP_FORBIDDEN) { + deleteCallInvocation[0]++; + } + throw ex; + } + throw new AssertionError("List Consumption should have failed"); + }) + .when(blobDeleteHandler).listRecursiveAndTakeAction(); + return null; + }); + final int[] listCallInvocation = new int[1]; + Mockito.doAnswer(answer -> { + if (listCallInvocation[0] == 1) { + while (deleteCallInvocation[0] == 0) {} + } + listCallInvocation[0]++; + return answer.callRealMethod(); + }) + .when(spiedClient) + .listPath(Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyInt(), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + intercept(AccessDeniedException.class, + () -> { + fs.delete(new Path("/src"), true); + }); + Mockito.verify(spiedClient, Mockito.times(1)) + .listPath(Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyInt(), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + } + + /** + * Test to assert that the CID in src marker delete contains the + * total number of blobs operated in the delete directory. + * Also, to assert that all operations in the delete-directory flow have same + * primaryId and opType. + * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testDeleteEmitDeletionCountInClientRequestId() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + assumeBlobClient(); + AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient(); + AbfsBlobClient spiedClient = Mockito.spy(client); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + store.setClient(spiedClient); + Mockito.doReturn(store).when(fs).getAbfsStore(); + String dirPathStr = "/testDir/dir1"; + fs.mkdirs(new Path(dirPathStr)); + ExecutorService executorService = Executors.newFixedThreadPool(5); + List futures = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + final int iter = i; + Future future = executorService.submit(() -> fs.create(new Path("/testDir/dir1/file" + iter))); + futures.add(future); + } + for (Future future : futures) { + future.get(); + } + executorService.shutdown(); + final TracingHeaderValidator tracingHeaderValidator + = new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.DELETE, false, 0); + fs.registerListener(tracingHeaderValidator); + Mockito.doAnswer(answer -> { + Mockito.doAnswer(deleteAnswer -> { + if (dirPathStr.equalsIgnoreCase( + ((Path) deleteAnswer.getArgument(0)).toUri().getPath())) { + tracingHeaderValidator.setOperatedBlobCount(BLOB_COUNT); + Object result = deleteAnswer.callRealMethod(); + tracingHeaderValidator.setOperatedBlobCount(null); + return result; + } + return deleteAnswer.callRealMethod(); + }) + .when(spiedClient) + .deleteBlobPath(Mockito.any(Path.class), + Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + return answer.callRealMethod(); + }) + .when(store) + .delete(Mockito.any(Path.class), Mockito.anyBoolean(), + Mockito.any(TracingContext.class)); + fs.delete(new Path(dirPathStr), true); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java index c48b8b0d6267d..34ffed8248a00 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java @@ -41,6 +41,7 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.INFINITE_LEASE_DURATION; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyString; @@ -343,7 +344,8 @@ public void testAcquireRetry() throws Exception { tracingContext.setListener(listener); AbfsLease lease = new AbfsLease(fs.getAbfsClient(), - testFilePath.toUri().getPath(), tracingContext); + testFilePath.toUri().getPath(), true, INFINITE_LEASE_DURATION, + null, tracingContext); Assert.assertNotNull("Did not successfully lease file", lease.getLeaseID()); listener.setOperation(FSOperationType.RELEASE_LEASE); lease.free(); @@ -355,19 +357,20 @@ public void testAcquireRetry() throws Exception { doThrow(new AbfsLease.LeaseException("failed to acquire 1")) .doThrow(new AbfsLease.LeaseException("failed to acquire 2")) .doCallRealMethod().when(mockClient) - .acquireLease(anyString(), anyInt(), any(TracingContext.class)); + .acquireLease(anyString(), anyInt(), anyString(), any(TracingContext.class)); - lease = new AbfsLease(mockClient, testFilePath.toUri().getPath(), 5, 1, tracingContext); + lease = new AbfsLease(mockClient, testFilePath.toUri().getPath(), true, 5, 1, + INFINITE_LEASE_DURATION, null, tracingContext); Assert.assertNotNull("Acquire lease should have retried", lease.getLeaseID()); lease.free(); Assert.assertEquals("Unexpected acquire retry count", 2, lease.getAcquireRetryCount()); doThrow(new AbfsLease.LeaseException("failed to acquire")).when(mockClient) - .acquireLease(anyString(), anyInt(), any(TracingContext.class)); + .acquireLease(anyString(), anyInt(), anyString(), any(TracingContext.class)); LambdaTestUtils.intercept(AzureBlobFileSystemException.class, () -> { - new AbfsLease(mockClient, testFilePath.toUri().getPath(), 5, 1, - tracingContext); + new AbfsLease(mockClient, testFilePath.toUri().getPath(), true, 5, 1, + INFINITE_LEASE_DURATION, null, tracingContext); }); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java index ea07650e90110..d16f7533fd857 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java @@ -18,22 +18,65 @@ package org.apache.hadoop.fs.azurebfs; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.nio.file.AccessDeniedException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; -import org.junit.Assert; +import org.apache.hadoop.fs.FileSystem; +import org.assertj.core.api.Assertions; import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.stubbing.Answer; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter; +import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil; +import org.apache.hadoop.fs.azurebfs.services.AbfsDfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; +import org.apache.hadoop.fs.azurebfs.services.AbfsLease; +import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; +import org.apache.hadoop.fs.azurebfs.services.BlobRenameHandler; +import org.apache.hadoop.fs.azurebfs.services.RenameAtomicity; +import org.apache.hadoop.fs.azurebfs.services.RenameAtomicityTestUtils; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; import org.apache.hadoop.fs.statistics.IOStatisticAssertions; import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.test.LambdaTestUtils; +import org.apache.hadoop.util.functional.FunctionRaisingIOE; +import static java.net.HttpURLConnection.HTTP_CONFLICT; +import static java.net.HttpURLConnection.HTTP_FORBIDDEN; +import static java.net.HttpURLConnection.HTTP_NOT_FOUND; +import static org.junit.Assume.assumeTrue; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.RENAME_PATH_ATTEMPTS; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_ABORTED; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_FAILED; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COPY_STATUS_PENDING; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.ROOT_PATH; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_LEASE_THREADS; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.COPY_BLOB_ABORTED; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.COPY_BLOB_FAILED; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND; +import static org.apache.hadoop.fs.azurebfs.services.RenameAtomicity.SUFFIX; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist; @@ -41,6 +84,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test rename operation. @@ -48,6 +92,9 @@ public class ITestAzureBlobFileSystemRename extends AbstractAbfsIntegrationTest { + private static final int MAX_ITERATIONS = 20; + private static final int BLOB_COUNT = 11; + public ITestAzureBlobFileSystemRename() throws Exception { super(); } @@ -160,7 +207,7 @@ public void testPosixRenameDirectory() throws Exception { Path testDir2 = path("testDir2"); fs.mkdirs(new Path(testDir2 + "/test1/test2/test3")); fs.mkdirs(new Path(testDir2 + "/test4")); - Assert.assertTrue(fs.rename(new Path(testDir2 + "/test1/test2/test3"), new Path(testDir2 + "/test4"))); + assertTrue(fs.rename(new Path(testDir2 + "/test1/test2/test3"), new Path(testDir2 + "/test4"))); assertPathExists(fs, "This path should exist", testDir2); assertPathExists(fs, "This path should exist", new Path(testDir2 + "/test1/test2")); @@ -192,10 +239,1417 @@ public void testRenameWithNoDestinationParentDir() throws Exception { // Verify that metadata was in an incomplete state after the rename // failure, and we retired the rename once more. IOStatistics ioStatistics = fs.getIOStatistics(); + AbfsClient client = fs.getAbfsStore().getClient(); IOStatisticAssertions.assertThatStatisticCounter(ioStatistics, RENAME_PATH_ATTEMPTS.getStatName()) - .describedAs("There should be 2 rename attempts if metadata " - + "incomplete state failure is hit") - .isEqualTo(2); + .describedAs("For Dfs endpoint: There should be 2 rename " + + "attempts if metadata incomplete state failure is hit." + + "For Blob endpoint: There would be only one rename attempt which " + + "would have a failed precheck.") + .isEqualTo(client instanceof AbfsDfsClient ? 2 : 1); + } + + /** + * Tests renaming a directory to the root directory. This test ensures that a directory can be renamed + * successfully to the root directory and that the renamed directory appears as expected. + * + * The test creates a directory (`/src1/src2`), renames it to the root (`/`), and verifies that + * the renamed directory (`/src2`) exists in the root. + * + * @throws Exception if an error occurs during test execution + */ + @Test + public void testRenameToRoot() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + fs.mkdirs(new Path("/src1/src2")); + assertTrue(fs.rename(new Path("/src1/src2"), new Path("/"))); + assertTrue(fs.exists(new Path("/src2"))); + } + + /** + * Tests renaming a non-existent file to the root directory. This test ensures that the rename + * operation returns `false` when attempting to rename a file that does not exist. + * + * The test attempts to rename a file located at `/file` (which does not exist) to the root directory `/` + * and verifies that the rename operation fails. + * + * @throws Exception if an error occurs during test execution + */ + @Test + public void testRenameNotFoundBlobToEmptyRoot() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assertFalse(fs.rename(new Path("/file"), new Path("/"))); + } + + /** + * Assumes the AzureBlobFileSystem client is an instance of `AbfsBlobClient`, ensuring that + * the test only proceeds if the underlying client is of the expected type. + * This assumption is typically used to validate that the file system is not using + * an HNS (Hierarchical Namespace) account, as it expects an `AbfsBlobClient` type client. + * + * @param fs the AzureBlobFileSystem instance to check + */ + private void assumeNonHnsAccountBlobEndpoint(final AzureBlobFileSystem fs) { + assumeTrue(fs.getAbfsStore().getClient() instanceof AbfsBlobClient); + } + + /** + * Tests renaming a source path to a destination path that contains a colon in the path. + * This verifies that the rename operation handles paths with special characters like a colon. + * + * The test creates a source directory and renames it to a destination path that includes a colon, + * ensuring that the operation succeeds without errors. + * + * @throws Exception if an error occurs during test execution + */ + @Test(expected = IOException.class) + public void testRenameBlobToDstWithColonInPath() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + fs.create(new Path("/src")); + fs.rename(new Path("/src"), new Path("/dst:file")); + } + + /** + * Tests renaming a directory within the same parent directory when there is no marker file. + * This test ensures that the rename operation succeeds even when no special marker file is present. + * + * The test creates a file in a directory, deletes the blob path using the client, and then attempts + * to rename the directory. It verifies that the rename operation completes successfully. + * + * @throws Exception if an error occurs during test execution + */ + @Test + public void testRenameBlobInSameDirectoryWithNoMarker() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsStore().getClient(); + fs.create(new Path("/srcDir/dir/file")); + client.deleteBlobPath(new Path("/srcDir/dir"), null, + getTestTracingContext(fs, true)); + assertTrue(fs.rename(new Path("/srcDir/dir"), new Path("/srcDir"))); + } + + /** + *

+   * Test to check behaviour of rename API if the destination directory is already
+   * there. The HNS call and the one for Blob endpoint should have same behaviour.
+   *
+   * /testDir2/test1/test2/test3 contains (/file)
+   * There is another path that exists: /testDir2/test4/test3
+   * On rename(/testDir2/test1/test2/test3, /testDir2/test4).
+   * 
+ * + * Expectation for HNS / Blob endpoint:
    + *
  1. Rename should fail
  2. + *
  3. No file should be transferred to destination directory
  4. + *
+ */ + @Test + public void testPosixRenameDirectoryWhereDirectoryAlreadyThereOnDestination() + throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.mkdirs(new Path("testDir2/test1/test2/test3")); + fs.create(new Path("testDir2/test1/test2/test3/file")); + fs.mkdirs(new Path("testDir2/test4/test3")); + assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file"))); + assertFalse(fs.rename(new Path("testDir2/test1/test2/test3"), + new Path("testDir2/test4"))); + assertTrue(fs.exists(new Path("testDir2"))); + assertTrue(fs.exists(new Path("testDir2/test1/test2"))); + assertTrue(fs.exists(new Path("testDir2/test4"))); + assertTrue(fs.exists(new Path("testDir2/test1/test2/test3"))); + if (getIsNamespaceEnabled(fs) + || fs.getAbfsClient() instanceof AbfsBlobClient) { + assertFalse(fs.exists(new Path("testDir2/test4/test3/file"))); + assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file"))); + } else { + assertTrue(fs.exists(new Path("testDir2/test4/test3/file"))); + assertFalse(fs.exists(new Path("testDir2/test1/test2/test3/file"))); + } + } + + /** + *
+   * Test to check behaviour of rename API if the destination directory is already
+   * there. The HNS call and the one for Blob endpoint should have same behaviour.
+   *
+   * /testDir2/test1/test2/test3 contains (/file)
+   * There is another path that exists: /testDir2/test4/test3
+   * On rename(/testDir2/test1/test2/test3, /testDir2/test4).
+   * 
+ * + * Expectation for HNS / Blob endpoint:
    + *
  1. Rename should fail
  2. + *
  3. No file should be transferred to destination directory
  4. + *
+ */ + @Test + public void testPosixRenameDirectoryWherePartAlreadyThereOnDestination() + throws Exception { + final AzureBlobFileSystem fs = this.getFileSystem(); + fs.mkdirs(new Path("testDir2/test1/test2/test3")); + fs.create(new Path("testDir2/test1/test2/test3/file")); + fs.create(new Path("testDir2/test1/test2/test3/file1")); + fs.mkdirs(new Path("testDir2/test4/")); + fs.create(new Path("testDir2/test4/file1")); + assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file"))); + assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file1"))); + assertTrue(fs.rename(new Path("testDir2/test1/test2/test3"), + new Path("testDir2/test4"))); + assertTrue(fs.exists(new Path("testDir2"))); + assertTrue(fs.exists(new Path("testDir2/test1/test2"))); + assertTrue(fs.exists(new Path("testDir2/test4"))); + assertFalse(fs.exists(new Path("testDir2/test1/test2/test3"))); + assertFalse(fs.exists(new Path("testDir2/test4/file"))); + assertTrue(fs.exists(new Path("testDir2/test4/file1"))); + assertTrue(fs.exists(new Path("testDir2/test4/test3/file"))); + assertTrue(fs.exists(new Path("testDir2/test4/test3/file1"))); + assertTrue(fs.exists(new Path("testDir2/test4/file1"))); + assertFalse(fs.exists(new Path("testDir2/test1/test2/test3/file"))); + assertFalse(fs.exists(new Path("testDir2/test1/test2/test3/file1"))); + } + + /** + * Test that after completing rename for a directory which is enabled for + * AtomicRename, the RenamePending JSON file is deleted. + */ + @Test + public void testRenamePendingJsonIsRemovedPostSuccessfulRename() + throws Exception { + final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); + assumeNonHnsAccountBlobEndpoint(fs); + fs.setWorkingDirectory(new Path("/")); + fs.mkdirs(new Path("hbase/test1/test2/test3")); + fs.create(new Path("hbase/test1/test2/test3/file")); + fs.create(new Path("hbase/test1/test2/test3/file1")); + fs.mkdirs(new Path("hbase/test4/")); + fs.create(new Path("hbase/test4/file1")); + final Integer[] correctDeletePathCount = new Integer[1]; + correctDeletePathCount[0] = 0; + Mockito.doAnswer(answer -> { + final String correctDeletePath = "/hbase/test1/test2/test3" + SUFFIX; + if (correctDeletePath.equals( + ((Path) answer.getArgument(0)).toUri().getPath())) { + correctDeletePathCount[0] = 1; + } + return null; + }) + .when(client) + .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + assertTrue(fs.rename(new Path("hbase/test1/test2/test3"), + new Path("hbase/test4"))); + assertEquals("RenamePendingJson should be deleted", + 1, + (int) correctDeletePathCount[0]); + } + + /** + * Spies on the AzureBlobFileSystem's store and client to enable mocking and verification + * of client interactions in tests. It replaces the actual store and client with mocked versions. + * + * @param fs the AzureBlobFileSystem instance + * @return the spied AbfsClient for interaction verification + */ + private AbfsClient addSpyHooksOnClient(final AzureBlobFileSystem fs) { + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + Mockito.doReturn(store).when(fs).getAbfsStore(); + AbfsClient client = Mockito.spy(store.getClient()); + Mockito.doReturn(client).when(store).getClient(); + return client; + } + + /** + * A helper method to set up the test environment and execute the common logic for handling + * failed rename operations and recovery in HBase. This method performs the necessary setup + * (creating directories and files) and then triggers the `crashRenameAndRecover` method + * with a provided recovery action. + * + * This method is used by different tests that require different recovery actions, such as + * performing `listStatus` or checking the existence of a path after a failed rename. + * + * @param fs the AzureBlobFileSystem instance to be used in the test + * @param client the AbfsBlobClient instance to be used in the test + * @param srcPath the source path for the rename operation + * @param failedCopyPath the path that simulates a failed copy during rename + * @param recoveryAction the specific recovery action to be performed after the rename failure + * (e.g., listing directory status or checking path existence) + * @throws Exception if any error occurs during setup or execution of the recovery action + */ + private void setupAndTestHBaseFailedRenameRecovery( + final AzureBlobFileSystem fs, + final AbfsBlobClient client, + final String srcPath, + final String failedCopyPath, + final FunctionRaisingIOE recoveryAction) + throws Exception { + fs.setWorkingDirectory(new Path("/")); + fs.mkdirs(new Path(srcPath)); + fs.mkdirs(new Path(srcPath, "test3")); + fs.create(new Path(srcPath + "/test3/file")); + fs.create(new Path(failedCopyPath)); + fs.mkdirs(new Path("hbase/test4/")); + fs.create(new Path("hbase/test4/file1")); + crashRenameAndRecover(fs, client, srcPath, recoveryAction); + } + + /** + * Test for a directory in /hbase directory. To simulate the crash of process, + * test will throw an exception with 403 on a copy of one of the blob.
+ * ListStatus API will be called on the directory. Expectation is that the ListStatus + * API of {@link AzureBlobFileSystem} should recover the paused rename. + */ + @Test + public void testHBaseHandlingForFailedRenameWithListRecovery() + throws Exception { + AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); + String srcPath = "hbase/test1/test2"; + final String failedCopyPath = srcPath + "/test3/file1"; + + setupAndTestHBaseFailedRenameRecovery(fs, client, srcPath, failedCopyPath, + (abfsFs) -> { + abfsFs.listStatus(new Path(srcPath).getParent()); + return null; + }); + } + + /** + * Test for a directory in /hbase directory. To simulate the crash of process, + * test will throw an exception with 403 on a copy of one of the blob. The + * source directory is a nested directory.
+ * GetFileStatus API will be called on the directory. Expectation is that the + * GetFileStatus API of {@link AzureBlobFileSystem} should recover the paused + * rename. + */ + @Test + public void testHBaseHandlingForFailedRenameWithGetFileStatusRecovery() + throws Exception { + AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); + String srcPath = "hbase/test1/test2"; + final String failedCopyPath = srcPath + "/test3/file1"; + + setupAndTestHBaseFailedRenameRecovery(fs, client, srcPath, failedCopyPath, + (abfsFs) -> { + abfsFs.exists(new Path(srcPath)); + return null; + }); + } + + + /** + * Simulates a rename failure, performs a recovery action, and verifies that the "RenamePendingJson" + * file is deleted. It checks that the rename operation is successfully completed after recovery. + * + * @param fs the AzureBlobFileSystem instance + * @param client the AbfsBlobClient instance + * @param srcPath the source path for the rename operation + * @param recoveryCallable the recovery action to perform + * @throws Exception if an error occurs during recovery or verification + */ + private void crashRenameAndRecover(final AzureBlobFileSystem fs, + AbfsBlobClient client, + final String srcPath, + final FunctionRaisingIOE recoveryCallable) + throws Exception { + crashRename(fs, client, srcPath); + AzureBlobFileSystem fs2 = Mockito.spy(getFileSystem()); + fs2.setWorkingDirectory(new Path(ROOT_PATH)); + client = (AbfsBlobClient) addSpyHooksOnClient(fs2); + int[] renameJsonDeleteCounter = new int[1]; + Mockito.doAnswer(answer -> { + if ((ROOT_PATH + srcPath + SUFFIX) + .equalsIgnoreCase(((Path) answer.getArgument(0)).toUri().getPath())) { + renameJsonDeleteCounter[0] = 1; + } + return answer.callRealMethod(); + }) + .when(client) + .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + recoveryCallable.apply(fs2); + Assertions.assertThat(renameJsonDeleteCounter[0]) + .describedAs("RenamePendingJson should be deleted") + .isEqualTo(1); + //List would complete the rename orchestration. + assertFalse(fs2.exists(new Path("hbase/test1/test2"))); + assertFalse(fs2.exists(new Path("hbase/test1/test2/test3"))); + assertTrue(fs2.exists(new Path("hbase/test4/test2/test3"))); + assertFalse(fs2.exists(new Path("hbase/test1/test2/test3/file"))); + assertTrue(fs2.exists(new Path("hbase/test4/test2/test3/file"))); + assertFalse(fs2.exists(new Path("hbase/test1/test2/test3/file1"))); + assertTrue(fs2.exists(new Path("hbase/test4/test2/test3/file1"))); + } + + /** + * Simulates a rename failure by triggering an `AbfsRestOperationException` during the rename process. + * It intercepts the exception and ensures that all leases acquired during the atomic rename are released. + * + * @param fs the AzureBlobFileSystem instance used for the rename operation + * @param client the AbfsBlobClient instance used for mocking the rename failure + * @param srcPath the source path for the rename operation + * @throws Exception if an error occurs during the simulated failure or lease release + */ + private void crashRename(final AzureBlobFileSystem fs, + final AbfsBlobClient client, + final String srcPath) throws Exception { + BlobRenameHandler[] blobRenameHandlers = new BlobRenameHandler[1]; + AbfsClientTestUtil.mockGetRenameBlobHandler(client, + blobRenameHandler -> { + blobRenameHandlers[0] = blobRenameHandler; + return null; + }); + //Fail rename orchestration on path hbase/test1/test2/test3/file1 + Mockito.doThrow(new AbfsRestOperationException(HTTP_FORBIDDEN, "", "", + new Exception())) + .when(client) + .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + LambdaTestUtils.intercept(AccessDeniedException.class, () -> { + fs.rename(new Path(srcPath), + new Path("hbase/test4")); + }); + //Release all the leases taken by atomic rename orchestration + List leases = new ArrayList<>(blobRenameHandlers[0].getLeases()); + for (AbfsLease lease : leases) { + lease.free(); + } + } + + /** + * Simulates a scenario where HMaster in Hbase starts up and executes listStatus + * API on the directory that has to be renamed by some other executor-machine. + * The scenario is that RenamePending JSON is created but before it could be + * appended, it has been opened by the HMaster. The HMaster will delete it. The + * machine doing rename would have to recreate the JSON file. + * ref: issue + */ + @Test + public void testHbaseListStatusBeforeRenamePendingFileAppendedWithIngressOnBlob() + throws Exception { + final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + fs.setWorkingDirectory(new Path(ROOT_PATH)); + testRenamePreRenameFailureResolution(fs); + testAtomicityRedoInvalidFile(fs); + } + + /** + * Tests renaming a directory in AzureBlobFileSystem when the creation of the "RenamePendingJson" + * file fails on the first attempt. It ensures the renaming operation is retried. + * + * The test verifies that the creation of the "RenamePendingJson" file is attempted twice: + * once on failure and once on retry. + * + * @param fs the AzureBlobFileSystem instance for the test + * @throws Exception if an error occurs during the test + */ + private void testRenamePreRenameFailureResolution(final AzureBlobFileSystem fs) + throws Exception { + AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); + Path src = new Path("hbase/test1/test2"); + Path dest = new Path("hbase/test4"); + fs.mkdirs(src); + fs.mkdirs(new Path(src, "test3")); + final int[] renamePendingJsonWriteCounter = new int[1]; + /* + * Fail the creation of RenamePendingJson file on the first attempt. + */ + Answer renamePendingJsonCreateAns = createAnswer -> { + Path path = createAnswer.getArgument(0); + Mockito.doAnswer(clientFlushAns -> { + if (renamePendingJsonWriteCounter[0]++ == 0) { + fs.delete(path, true); + } + return clientFlushAns.callRealMethod(); + }) + .when(client) + .flush(Mockito.any(byte[].class), Mockito.anyString(), + Mockito.anyBoolean(), Mockito.nullable(String.class), + Mockito.nullable(String.class), Mockito.anyString(), + Mockito.nullable(ContextEncryptionAdapter.class), + Mockito.any(TracingContext.class)); + return createAnswer.callRealMethod(); + }; + RenameAtomicityTestUtils.addCreatePathMock(client, + renamePendingJsonCreateAns); + fs.rename(src, dest); + Assertions.assertThat(renamePendingJsonWriteCounter[0]) + .describedAs("Creation of RenamePendingJson should be attempted twice") + .isEqualTo(2); + } + + /** + * Tests the behavior of the redo operation when an invalid "RenamePendingJson" file exists. + * It verifies that the file is deleted and that no copy operation is performed. + * + * The test simulates a scenario where the "RenamePendingJson" file is partially written and + * ensures that the `redo` method correctly deletes the file and does not trigger a copy operation. + * + * @param fs the AzureBlobFileSystem instance for the test + * @throws Exception if an error occurs during the test + */ + private void testAtomicityRedoInvalidFile(final AzureBlobFileSystem fs) + throws Exception { + AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); + Path path = new Path("/hbase/test1/test2"); + fs.mkdirs(new Path(path, "test3")); + Path renameJson = new Path(path.getParent(), path.getName() + SUFFIX); + OutputStream os = fs.create(renameJson); + os.write("{".getBytes(StandardCharsets.UTF_8)); + os.close(); + int[] renameJsonDeleteCounter = new int[1]; + Mockito.doAnswer(deleteAnswer -> { + Path ansPath = deleteAnswer.getArgument(0); + if (renameJson.toUri() + .getPath() + .equalsIgnoreCase(ansPath.toUri().getPath())) { + renameJsonDeleteCounter[0]++; + } + return deleteAnswer.callRealMethod(); + }) + .when(client) + .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + new RenameAtomicity(renameJson, 1, + getTestTracingContext(fs, true), null, client).redo(); + Assertions.assertThat(renameJsonDeleteCounter[0]) + .describedAs("RenamePendingJson should be deleted") + .isEqualTo(1); + Mockito.verify(client, Mockito.times(0)).copyBlob(Mockito.any(Path.class), + Mockito.any(Path.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + } + + /** + * Test to check the atomicity of rename operation. The rename operation should + * be atomic and should not leave any intermediate state. + */ + @Test + public void testRenameJsonDeletedBeforeRenameAtomicityCanDelete() + throws Exception { + final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); + fs.setWorkingDirectory(new Path(ROOT_PATH)); + Path path = new Path("/hbase/test1/test2"); + fs.mkdirs(new Path(path, "test3")); + Path renameJson = new Path(path.getParent(), path.getName() + SUFFIX); + OutputStream os = fs.create(renameJson); + os.write("{}".getBytes(StandardCharsets.UTF_8)); + os.close(); + int[] renameJsonDeleteCounter = new int[1]; + Mockito.doAnswer(deleteAnswer -> { + Path ansPath = deleteAnswer.getArgument(0); + if (renameJson.toUri() + .getPath() + .equalsIgnoreCase(ansPath.toUri().getPath())) { + renameJsonDeleteCounter[0]++; + } + getFileSystem().delete(ansPath, true); + return deleteAnswer.callRealMethod(); + }) + .when(client) + .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + new RenameAtomicity(renameJson, 2, + getTestTracingContext(fs, true), null, client); + } + + /** + * Tests the scenario where the rename operation is complete before the redo + * operation for atomicity, leading to a failure. This test verifies that the + * system correctly handles the case when a rename operation is attempted after + * the source path has already been deleted, which should result in an error. + *

+ * The test simulates a situation where a `renameJson` file is created for the + * rename operation, and the source path is deleted during the read process in + * the redo operation. The `redoRenameAtomicity` is then executed, and it is + * expected to fail with a `404` error, indicating that the source path no longer exists. + *

+ * The test ensures that the system can handle this error condition and return + * the correct response, preventing a potentially invalid or inconsistent state. + * + * @throws Exception If an error occurs during file system operations. + */ + @Test + public void testRenameCompleteBeforeRenameAtomicityRedo() throws Exception { + final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); + fs.setWorkingDirectory(new Path(ROOT_PATH)); + Path path = new Path("/hbase/test1/test2"); + fs.mkdirs(new Path(path, "test3")); + Path renameJson = new Path(path.getParent(), path.getName() + SUFFIX); + /* + * Create renameJson file. + */ + AzureBlobFileSystemStore.VersionedFileStatus fileStatus + = (AzureBlobFileSystemStore.VersionedFileStatus) fs.getFileStatus(path); + int jsonLen = new RenameAtomicity(path, + new Path("/hbase/test4"), renameJson, + getTestTracingContext(fs, true), fileStatus.getEtag(), client).preRename(); + RenameAtomicity redoRenameAtomicity = Mockito.spy( + new RenameAtomicity(renameJson, jsonLen, + getTestTracingContext(fs, true), null, client)); + RenameAtomicityTestUtils.addReadPathMock(redoRenameAtomicity, + readCallbackAnswer -> { + byte[] bytes = (byte[]) readCallbackAnswer.callRealMethod(); + fs.delete(path, true); + return bytes; + }); + AbfsRestOperationException ex = intercept(AbfsRestOperationException.class, + redoRenameAtomicity::redo); + Assertions.assertThat(ex.getStatusCode()) + .describedAs("RenameAtomicity redo should fail with 404") + .isEqualTo(SOURCE_PATH_NOT_FOUND.getStatusCode()); + Assertions.assertThat(ex.getErrorCode()) + .describedAs("RenameAtomicity redo should fail with 404") + .isEqualTo(SOURCE_PATH_NOT_FOUND); + } + + /** + * Tests the idempotency of the copyBlob operation during a rename when the + * destination already exists. This test simulates a scenario where the source + * blob is copied to the destination before the actual rename operation is invoked. + * It ensures that the copyBlob operation can handle idempotency issues and perform + * the rename successfully even when the destination is pre-created. + *

+ * The test verifies that the rename operation successfully copies the blob from + * the source to the destination, and the source is deleted, leaving only the + * destination file. This ensures that the system behaves correctly in scenarios + * where the destination path already contains the blob. + * + * @throws Exception If an error occurs during file system operations. + */ + @Test + public void testCopyBlobIdempotency() throws Exception { + final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); + fs.setWorkingDirectory(new Path(ROOT_PATH)); + Path src = new Path("/srcDir/src"); + Path dst = new Path("/dst"); + fs.create(src); + Mockito.doAnswer(answer -> { + Path srcCopy = answer.getArgument(0); + Path dstCopy = answer.getArgument(1); + String leaseId = answer.getArgument(2); + TracingContext tracingContext = answer.getArgument(3); + /* + * To fail copyBlob with idempotency issue, making a copy of the source to destination + * before the invoked copy + */ + ((AbfsBlobClient) getFileSystem().getAbfsClient()).copyBlob(srcCopy, + dstCopy, leaseId, tracingContext); + return answer.callRealMethod(); + }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + Assertions.assertThat(fs.rename(src, dst)) + .describedAs("Rename should be successful and copyBlob should" + + "be able to handle idempotency issue") + .isTrue(); + Assertions.assertThat(fs.exists(src)) + .describedAs("Source should not exist after rename") + .isFalse(); + Assertions.assertThat(fs.exists(dst)) + .describedAs("Destination should exist after rename") + .isTrue(); + } + + /** + * Tests the idempotency of the rename operation when the destination path is + * created by some other process before the rename operation. This test simulates + * the scenario where a source blob is renamed, and the destination path already + * exists due to actions from another process. It ensures that the rename operation + * behaves idempotently and correctly handles the case where the destination is + * pre-created. + *

+ * The test verifies that the rename operation fails (since the destination already + * exists), but the source path remains intact, and the blob copy operation is able + * to handle the idempotency issue. + * + * @throws IOException If an error occurs during file system operations. + */ + @Test + public void testRenameBlobIdempotencyWhereDstIsCreatedFromSomeOtherProcess() + throws IOException { + final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); + fs.setWorkingDirectory(new Path(ROOT_PATH)); + Path src = new Path("/src"); + Path dst = new Path("/dst"); + fs.create(src); + Mockito.doAnswer(answer -> { + Path dstCopy = answer.getArgument(1); + fs.create(dstCopy); + return answer.callRealMethod(); + }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + Assertions.assertThat(fs.rename(src, dst)) + .describedAs("Rename should be successful and copyBlob should" + + "be able to handle idempotency issue") + .isFalse(); + Assertions.assertThat(fs.exists(src)) + .describedAs("Source should exist after rename failure") + .isTrue(); + } + + /** + * Tests renaming a directory when the destination directory is missing a marker blob. + * This test involves creating multiple directories and files, deleting a blob (marker) in the + * destination directory, and renaming the source directory to the destination. + * It then verifies that the renamed directory exists at the expected destination path. + * + * @throws Exception If an error occurs during the file system operations or assertions. + */ + @Test + public void testRenameDirWhenMarkerBlobIsAbsentOnDstDir() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + fs.mkdirs(new Path("/test1")); + fs.mkdirs(new Path("/test1/test2")); + fs.mkdirs(new Path("/test1/test2/test3")); + fs.create(new Path("/test1/test2/test3/file")); + ((AbfsBlobClient) fs.getAbfsClient()) + .deleteBlobPath(new Path("/test1/test2"), + null, getTestTracingContext(fs, true)); + fs.mkdirs(new Path("/test4/test5")); + fs.rename(new Path("/test4"), new Path("/test1/test2")); + assertTrue(fs.exists(new Path("/test1/test2/test4/test5"))); + } + + /** + * Tests the renaming of a directory when the source directory does not have a marker file. + * This test creates a file within a source directory, deletes the source directory from the blob storage, + * creates a new target directory, and renames the source directory to the target location. + * It verifies that the renamed source directory exists in the target path. + * + * @throws Exception If an error occurs during the file system operations or assertions. + */ + @Test + public void testBlobRenameSrcDirHasNoMarker() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + fs.create(new Path("/test1/test2/file1")); + ((AbfsBlobClient) fs.getAbfsStore().getClient()) + .deleteBlobPath(new Path("/test1"), null, + getTestTracingContext(fs, true)); + fs.mkdirs(new Path("/test2")); + fs.rename(new Path("/test1"), new Path("/test2")); + assertTrue(fs.exists(new Path("/test2/test1"))); + } + + /** + * Mocks the progress status for a copy blob operation. + * This method simulates a copy operation that is pending and not yet completed. + * It intercepts the `copyBlob` method and modifies its response to return a "COPY_STATUS_PENDING" + * status for the copy operation. + * + * @param spiedClient The {@link AbfsBlobClient} instance that is being spied on. + * @throws AzureBlobFileSystemException if the mock setup fails. + */ + private void addMockForProgressStatusOnCopyOperation(final AbfsBlobClient spiedClient) + throws AzureBlobFileSystemException { + Mockito.doAnswer(answer -> { + AbfsRestOperation op = Mockito.spy( + (AbfsRestOperation) answer.callRealMethod()); + AbfsHttpOperation httpOp = Mockito.spy(op.getResult()); + Mockito.doReturn(COPY_STATUS_PENDING).when(httpOp).getResponseHeader( + HttpHeaderConfigurations.X_MS_COPY_STATUS); + Mockito.doReturn(httpOp).when(op).getResult(); + return op; + }) + .when(spiedClient) + .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + } + + /** + * Verifies the behavior of a blob copy operation that takes time to complete. + * The test ensures the following: + *

    + *
  • A file is created and a rename operation is initiated.
  • + *
  • The copy operation progress is mocked to simulate a time-consuming process.
  • + *
  • The rename operation triggers a call to handle the copy progress.
  • + *
  • The test checks that the file exists after the rename and that the + * `handleCopyInProgress` method is called exactly once.
  • + *
+ * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testCopyBlobTakeTime() throws Exception { + AzureBlobFileSystem fileSystem = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fileSystem); + AbfsBlobClient spiedClient = (AbfsBlobClient) addSpyHooksOnClient( + fileSystem); + addMockForProgressStatusOnCopyOperation(spiedClient); + fileSystem.create(new Path("/test1/file")); + BlobRenameHandler[] blobRenameHandlers = new BlobRenameHandler[1]; + AbfsClientTestUtil.mockGetRenameBlobHandler(spiedClient, + blobRenameHandler -> { + blobRenameHandlers[0] = blobRenameHandler; + return null; + }); + fileSystem.rename(new Path("/test1/file"), new Path("/test1/file2")); + assertTrue(fileSystem.exists(new Path("/test1/file2"))); + Mockito.verify(blobRenameHandlers[0], Mockito.times(1)) + .handleCopyInProgress(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.any(String.class)); + } + + /** + * Mocks the final status of a blob copy operation. + * This method ensures that when checking the status of a copy operation in progress, + * it returns the specified final status (e.g., success, failure, aborted). + * + * @param spiedClient The mocked Azure Blob client to apply the mock behavior. + * @param requiredCopyFinalStatus The final status of the copy operation to be returned + * (e.g., COPY_STATUS_FAILED, COPY_STATUS_ABORTED). + */ + private void addMockForCopyOperationFinalStatus(final AbfsBlobClient spiedClient, + final String requiredCopyFinalStatus) { + AbfsClientTestUtil.mockGetRenameBlobHandler(spiedClient, + blobRenameHandler -> { + Mockito.doAnswer(onHandleCopyInProgress -> { + Path handlePath = onHandleCopyInProgress.getArgument(0); + TracingContext tracingContext = onHandleCopyInProgress.getArgument( + 1); + Mockito.doAnswer(onStatusCheck -> { + AbfsRestOperation op = Mockito.spy( + (AbfsRestOperation) onStatusCheck.callRealMethod()); + AbfsHttpOperation httpOp = Mockito.spy(op.getResult()); + Mockito.doReturn(requiredCopyFinalStatus) + .when(httpOp) + .getResponseHeader( + HttpHeaderConfigurations.X_MS_COPY_STATUS); + Mockito.doReturn(httpOp).when(op).getResult(); + return op; + }) + .when(spiedClient) + .getPathStatus(handlePath.toUri().getPath(), + tracingContext, null, false); + return onHandleCopyInProgress.callRealMethod(); + }) + .when(blobRenameHandler) + .handleCopyInProgress(Mockito.any(Path.class), + Mockito.any(TracingContext.class), Mockito.any(String.class)); + return null; + }); + } + + /** + * Verifies the behavior when a blob copy operation takes time and eventually fails. + * The test ensures the following: + *
    + *
  • A file is created and a copy operation is initiated.
  • + *
  • The copy operation is mocked to eventually fail.
  • + *
  • The rename operation triggers an exception due to the failed copy.
  • + *
  • The test checks that the appropriate 'COPY_FAILED' error code and status code are returned.
  • + *
+ * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testCopyBlobTakeTimeAndEventuallyFail() throws Exception { + AzureBlobFileSystem fileSystem = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fileSystem); + AbfsBlobClient spiedClient = (AbfsBlobClient) addSpyHooksOnClient( + fileSystem); + addMockForProgressStatusOnCopyOperation(spiedClient); + fileSystem.create(new Path("/test1/file")); + addMockForCopyOperationFinalStatus(spiedClient, COPY_STATUS_FAILED); + AbfsRestOperationException ex = intercept(AbfsRestOperationException.class, + () -> { + fileSystem.rename(new Path("/test1/file"), new Path("/test1/file2")); + }); + Assertions.assertThat(ex.getStatusCode()) + .describedAs("Expecting COPY_FAILED status code") + .isEqualTo(COPY_BLOB_FAILED.getStatusCode()); + Assertions.assertThat(ex.getErrorCode()) + .describedAs("Expecting COPY_FAILED error code") + .isEqualTo(COPY_BLOB_FAILED); + } + + /** + * Verifies the behavior when a blob copy operation takes time and is eventually aborted. + * The test ensures the following: + *
    + *
  • A file is created and a copy operation is initiated.
  • + *
  • The copy operation is mocked to eventually be aborted.
  • + *
  • The rename operation triggers an exception due to the aborted copy.
  • + *
  • The test checks that the appropriate 'COPY_ABORTED' error code and status code are returned.
  • + *
+ * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testCopyBlobTakeTimeAndEventuallyAborted() throws Exception { + AzureBlobFileSystem fileSystem = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fileSystem); + AbfsBlobClient spiedClient = (AbfsBlobClient) addSpyHooksOnClient( + fileSystem); + addMockForProgressStatusOnCopyOperation(spiedClient); + fileSystem.create(new Path("/test1/file")); + addMockForCopyOperationFinalStatus(spiedClient, COPY_STATUS_ABORTED); + AbfsRestOperationException ex = intercept(AbfsRestOperationException.class, + () -> { + fileSystem.rename(new Path("/test1/file"), new Path("/test1/file2")); + }); + Assertions.assertThat(ex.getStatusCode()) + .describedAs("Expecting COPY_ABORTED status code") + .isEqualTo(COPY_BLOB_ABORTED.getStatusCode()); + Assertions.assertThat(ex.getErrorCode()) + .describedAs("Expecting COPY_ABORTED error code") + .isEqualTo(COPY_BLOB_ABORTED); + } + + /** + * Verifies the behavior when a blob copy operation takes time and the destination blob + * is deleted during the process. The test ensures the following: + *
    + *
  • A source file is created and a copy operation is initiated.
  • + *
  • During the copy process, the destination file is deleted.
  • + *
  • The copy operation returns a pending status.
  • + *
  • The test checks that the destination file does not exist after the copy operation is interrupted.
  • + *
+ * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testCopyBlobTakeTimeAndBlobIsDeleted() throws Exception { + AzureBlobFileSystem fileSystem = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fileSystem); + AbfsBlobClient spiedClient = (AbfsBlobClient) addSpyHooksOnClient( + fileSystem); + String srcFile = "/test1/file"; + String dstFile = "/test1/file2"; + Mockito.doAnswer(answer -> { + AbfsRestOperation op = Mockito.spy( + (AbfsRestOperation) answer.callRealMethod()); + fileSystem.delete(new Path(dstFile), false); + AbfsHttpOperation httpOp = Mockito.spy(op.getResult()); + Mockito.doReturn(COPY_STATUS_PENDING).when(httpOp).getResponseHeader( + HttpHeaderConfigurations.X_MS_COPY_STATUS); + Mockito.doReturn(httpOp).when(op).getResult(); + return op; + }) + .when(spiedClient) + .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + fileSystem.create(new Path(srcFile)); + assertFalse(fileSystem.rename(new Path(srcFile), new Path(dstFile))); + assertFalse(fileSystem.exists(new Path(dstFile))); + } + + /** + * Verifies the behavior when attempting to copy a blob after the source has been deleted + * in the Azure Blob FileSystem. The test ensures the following: + *
    + *
  • A source blob is created and then deleted.
  • + *
  • An attempt to copy the deleted source blob results in a 'not found' error.
  • + *
  • The test checks that the correct HTTP error (404 Not Found) is returned when copying a non-existent source.
  • + *
+ * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testCopyAfterSourceHasBeenDeleted() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient(); + fs.create(new Path("/src")); + TracingContext tracingContext = new TracingContext("clientCorrelationId", + "fileSystemId", FSOperationType.TEST_OP, + getConfiguration().getTracingHeaderFormat(), + null); + client.deleteBlobPath(new Path("/src"), null, + getTestTracingContext(fs, true)); + Boolean srcBlobNotFoundExReceived = false; + AbfsRestOperationException ex = intercept(AbfsRestOperationException.class, + () -> { + client.copyBlob(new Path("/src"), new Path("/dst"), + null, getTestTracingContext(fs, true)); + }); + Assertions.assertThat(ex.getStatusCode()) + .describedAs("Source has to be not found at copy") + .isEqualTo(HTTP_NOT_FOUND); + } + + /** + * Verifies that parallel rename operations in the Azure Blob FileSystem fail when + * trying to perform an atomic rename with lease acquisition. The test ensures the following: + *
    + *
  • A directory is created and a rename operation is attempted.
  • + *
  • A parallel thread attempts to rename the directory while the lease is being acquired.
  • + *
  • The parallel rename operation should fail due to a lease conflict, triggering an exception.
  • + *
  • The test verifies that the expected conflict exception is thrown when attempting a parallel rename.
  • + *
+ * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testParallelRenameForAtomicRenameShouldFail() throws Exception { + Configuration config = getRawConfiguration(); + config.set(FS_AZURE_LEASE_THREADS, "2"); + AzureBlobFileSystem fs = Mockito.spy( + (AzureBlobFileSystem) FileSystem.newInstance(config)); + assumeNonHnsAccountBlobEndpoint(fs); + fs.setWorkingDirectory(new Path(ROOT_PATH)); + AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); + Path src = new Path("/hbase/src"); + Path dst = new Path("/hbase/dst"); + fs.mkdirs(src); + AtomicBoolean leaseAcquired = new AtomicBoolean(false); + AtomicBoolean exceptionOnParallelRename = new AtomicBoolean(false); + AtomicBoolean parallelThreadDone = new AtomicBoolean(false); + Mockito.doAnswer(answer -> { + AbfsRestOperation op = (AbfsRestOperation) answer.callRealMethod(); + leaseAcquired.set(true); + while (!parallelThreadDone.get()) {} + return op; + }) + .when(client) + .acquireLease(Mockito.anyString(), Mockito.anyInt(), + Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + new Thread(() -> { + while (!leaseAcquired.get()) {} + try { + fs.rename(src, dst); + } catch (Exception e) { + if (e.getCause() instanceof AbfsLease.LeaseException + && e.getCause().getCause() instanceof AbfsRestOperationException + && ((AbfsRestOperationException) e.getCause() + .getCause()).getStatusCode() == HTTP_CONFLICT) { + exceptionOnParallelRename.set(true); + } + } finally { + parallelThreadDone.set(true); + } + }).start(); + fs.rename(src, dst); + while (!parallelThreadDone.get()) {} + Assertions.assertThat(exceptionOnParallelRename.get()) + .describedAs("Parallel rename should fail") + .isTrue(); + } + + /** + * Verifies the behavior of appending data to a blob during a rename operation in the + * Azure Blob FileSystem. The test ensures the following: + *
    + *
  • A file is created and data is appended to it while a rename operation is in progress.
  • + *
  • The append operation should fail due to the rename operation in progress.
  • + *
  • The test checks that the append operation is properly interrupted and fails as expected.
  • + *
+ * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testAppendAtomicBlobDuringRename() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); + Path src = new Path("/hbase/src"); + Path dst = new Path("/hbase/dst"); + FSDataOutputStream os = fs.create(src); + AtomicBoolean copyInProgress = new AtomicBoolean(false); + AtomicBoolean outputStreamClosed = new AtomicBoolean(false); + AtomicBoolean appendFailed = new AtomicBoolean(false); + Mockito.doAnswer(answer -> { + copyInProgress.set(true); + while (!outputStreamClosed.get()) {} + return answer.callRealMethod(); + }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + new Thread(() -> { + while (!copyInProgress.get()) {} + try { + os.write(1); + os.close(); + } catch (IOException e) { + appendFailed.set(true); + } finally { + outputStreamClosed.set(true); + } + }).start(); + fs.rename(src, dst); + Assertions.assertThat(appendFailed.get()) + .describedAs("Append should fail") + .isTrue(); + } + + /** + * Verifies the behavior of renaming a directory in the Azure Blob FileSystem when + * there is a neighboring directory with the same prefix. The test ensures the following: + *
    + *
  • Two directories with similar prefixes are created, along with files inside them.
  • + *
  • The rename operation moves one directory to a new location.
  • + *
  • Files in the renamed directory are moved, while files in the neighboring directory with the same prefix remain unaffected.
  • + *
  • Correct existence checks are performed to confirm the renamed directory and its files are moved, and the original directory is deleted.
  • + *
+ * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testBlobRenameOfDirectoryHavingNeighborWithSamePrefix() + throws Exception { + AzureBlobFileSystem fs = getFileSystem(); + assumeNonHnsAccountBlobEndpoint(fs); + fs.mkdirs(new Path("/testDir/dir")); + fs.mkdirs(new Path("/testDir/dirSamePrefix")); + fs.create(new Path("/testDir/dir/file1")); + fs.create(new Path("/testDir/dir/file2")); + fs.create(new Path("/testDir/dirSamePrefix/file1")); + fs.create(new Path("/testDir/dirSamePrefix/file2")); + fs.rename(new Path("/testDir/dir"), new Path("/testDir/dir2")); + Assertions.assertThat(fs.exists(new Path("/testDir/dirSamePrefix/file1"))) + .isTrue(); + Assertions.assertThat(fs.exists(new Path("/testDir/dir/file1"))) + .isFalse(); + Assertions.assertThat(fs.exists(new Path("/testDir/dir/file2"))) + .isFalse(); + Assertions.assertThat(fs.exists(new Path("/testDir/dir/"))) + .isFalse(); + } + + /** + * Verifies the behavior of renaming a directory in the Azure Blob FileSystem when + * the `listPath` operation returns paginated results with one object per list. + * The test ensures the following: + *
    + *
  • A directory and its files are created.
  • + *
  • The `listPath` operation is mocked to return one file at a time in each paginated result.
  • + *
  • The rename operation successfully moves the directory and its files to a new location.
  • + *
  • All files are verified to exist in the new location after the rename.
  • + *
+ * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testBlobRenameWithListGivingPaginatedResultWithOneObjectPerList() + throws Exception { + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + AbfsBlobClient spiedClient = (AbfsBlobClient) addSpyHooksOnClient(fs); + fs.mkdirs(new Path("/testDir/dir1")); + for (int i = 0; i < 10; i++) { + fs.create(new Path("/testDir/dir1/file" + i)); + } + Mockito.doAnswer(answer -> { + String path = answer.getArgument(0); + boolean recursive = answer.getArgument(1); + String continuation = answer.getArgument(3); + TracingContext context = answer.getArgument(4); + return getFileSystem().getAbfsClient() + .listPath(path, recursive, 1, continuation, context); + }) + .when(spiedClient) + .listPath(Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyInt(), + Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + fs.rename(new Path("/testDir/dir1"), new Path("/testDir/dir2")); + for (int i = 0; i < 10; i++) { + Assertions.assertThat(fs.exists(new Path("/testDir/dir2/file" + i))) + .describedAs("File " + i + " should exist in /testDir/dir2") + .isTrue(); + } + } + + /** + * Verifies that the producer stops on a rename failure due to an access denial + * (HTTP_FORBIDDEN error) in the Azure Blob FileSystem. The test ensures the following: + *
    + *
  • Multiple file creation tasks are submitted concurrently.
  • + *
  • The rename operation is attempted but fails with an access denied exception.
  • + *
  • On failure, the list operation for the source directory is invoked at most twice.
  • + *
+ * The test simulates a failure scenario where the rename operation encounters an access + * denied error, and the list operation should stop after the failure. + * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testProducerStopOnRenameFailure() throws Exception { + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + fs.mkdirs(new Path("/src")); + ExecutorService executorService = Executors.newFixedThreadPool(10); + List futureList = new ArrayList<>(); + for (int i = 0; i < MAX_ITERATIONS; i++) { + int iter = i; + Future future = executorService.submit(() -> { + try { + fs.create(new Path("/src/file" + iter)); + } catch (IOException ex) {} + }); + futureList.add(future); + } + for (Future future : futureList) { + future.get(); + } + AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsClient(); + AbfsBlobClient spiedClient = Mockito.spy(client); + AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore()); + store.setClient(spiedClient); + Mockito.doReturn(store).when(fs).getAbfsStore(); + final int[] copyCallInvocation = new int[1]; + Mockito.doAnswer(answer -> { + throw new AbfsRestOperationException(HTTP_FORBIDDEN, "", "", + new Exception()); + }).when(spiedClient) + .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + AbfsClientTestUtil.mockGetRenameBlobHandler(spiedClient, + (blobRenameHandler) -> { + Mockito.doAnswer(answer -> { + try { + answer.callRealMethod(); + } catch (AbfsRestOperationException ex) { + if (ex.getStatusCode() == HTTP_FORBIDDEN) { + copyCallInvocation[0]++; + } + throw ex; + } + throw new AssertionError("List Consumption should have failed"); + }) + .when(blobRenameHandler).listRecursiveAndTakeAction(); + return null; + }); + final int[] listCallInvocation = new int[1]; + Mockito.doAnswer(answer -> { + if (answer.getArgument(0).equals("/src")) { + if (listCallInvocation[0] == 1) { + while (copyCallInvocation[0] == 0) {} + } + listCallInvocation[0]++; + return getFileSystem().getAbfsClient().listPath(answer.getArgument(0), + answer.getArgument(1), 1, + answer.getArgument(3), answer.getArgument(4)); + } + return answer.callRealMethod(); + }) + .when(spiedClient) + .listPath(Mockito.anyString(), Mockito.anyBoolean(), Mockito.anyInt(), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + intercept(AccessDeniedException.class, + () -> { + fs.rename(new Path("/src"), new Path("/dst")); + }); + Assertions.assertThat(listCallInvocation[0]) + .describedAs("List on src should have been invoked at-most twice." + + "One before consumption and the other after consumption has starting." + + "Once consumption fails, listing would be stopped.") + .isLessThanOrEqualTo(2); + } + + /** + * Verifies the behavior of renaming a directory through the Azure Blob FileSystem + * when the source directory is deleted just before the rename operation is resumed. + * It ensures that: + *
    + *
  • No blobs are copied during the resume operation.
  • + *
+ * The test simulates a crash, deletes the source directory, and checks for the expected result. + * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testRenameResumeThroughListStatusWithSrcDirDeletedJustBeforeResume() + throws Exception { + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); + fs.setWorkingDirectory(new Path(ROOT_PATH)); + Path srcPath = new Path("hbase/test1/"); + Path failurePath = new Path(srcPath, "file"); + fs.mkdirs(srcPath); + fs.create(failurePath); + crashRename(fs, client, srcPath.toUri().getPath()); + fs.delete(srcPath, true); + AtomicInteger copiedBlobs = new AtomicInteger(0); + Mockito.doAnswer(answer -> { + copiedBlobs.incrementAndGet(); + return answer.callRealMethod(); + }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + fs.listStatus(new Path("hbase")); + Assertions.assertThat(copiedBlobs.get()) + .describedAs("No Copy on resume") + .isEqualTo(0); + } + + /** + * Verifies the behavior of renaming a directory through the Azure Blob FileSystem + * when the source directory's ETag changes just before the rename operation is resumed. + * It ensures that: + *
    + *
  • No blobs are copied during the resume operation.
  • + *
  • The pending rename JSON file is deleted.
  • + *
+ * The test simulates a crash, retries the operation, and checks for the expected results. + * + * @throws Exception if an error occurs during the test execution + */ + @Test + public void testRenameResumeThroughListStatusWithSrcDirETagChangedJustBeforeResume() + throws Exception { + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); + fs.setWorkingDirectory(new Path(ROOT_PATH)); + Path srcPath = new Path("hbase/test1/"); + Path failurePath = new Path(srcPath, "file"); + fs.mkdirs(srcPath); + fs.create(failurePath); + crashRename(fs, client, srcPath.toUri().getPath() + ); + fs.delete(srcPath, true); + fs.mkdirs(srcPath); + AtomicInteger copiedBlobs = new AtomicInteger(0); + Mockito.doAnswer(answer -> { + copiedBlobs.incrementAndGet(); + return answer.callRealMethod(); + }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + AtomicInteger pendingJsonDeleted = new AtomicInteger(0); + Mockito.doAnswer(listAnswer -> { + Path path = listAnswer.getArgument(0); + if (path.toUri().getPath().endsWith(SUFFIX)) { + pendingJsonDeleted.incrementAndGet(); + } + return listAnswer.callRealMethod(); + }) + .when(client) + .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + fs.listStatus(new Path("/hbase")); + Assertions.assertThat(copiedBlobs.get()) + .describedAs("No Copy on resume") + .isEqualTo(0); + Assertions.assertThat(pendingJsonDeleted.get()) + .describedAs("RenamePendingJson should be deleted") + .isEqualTo(1); + } + + /** + * Test case to verify the behavior of renaming a directory through the Azure Blob + * FileSystem when the source directory's ETag changes just before the rename operation + * is resumed. This test specifically checks the following: + * + * @throws Exception if any errors occur during the test execution + */ + @Test + public void testRenameResumeThroughGetStatusWithSrcDirETagChangedJustBeforeResume() + throws Exception { + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); + fs.setWorkingDirectory(new Path(ROOT_PATH)); + Path srcPath = new Path("hbase/test1/"); + Path failurePath = new Path(srcPath, "file"); + fs.mkdirs(srcPath); + fs.create(failurePath); + crashRename(fs, client, srcPath.toUri().getPath() + ); + fs.delete(srcPath, true); + fs.mkdirs(srcPath); + AtomicInteger copiedBlobs = new AtomicInteger(0); + Mockito.doAnswer(answer -> { + copiedBlobs.incrementAndGet(); + return answer.callRealMethod(); + }).when(client).copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), Mockito.any(TracingContext.class)); + AtomicInteger pendingJsonDeleted = new AtomicInteger(0); + Mockito.doAnswer(listAnswer -> { + Path path = listAnswer.getArgument(0); + if (path.toUri().getPath().endsWith(SUFFIX)) { + pendingJsonDeleted.incrementAndGet(); + } + return listAnswer.callRealMethod(); + }) + .when(client) + .deleteBlobPath(Mockito.any(Path.class), Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + Assertions.assertThat(fs.exists(srcPath)) + .describedAs("Source should exist") + .isTrue(); + Assertions.assertThat(copiedBlobs.get()) + .describedAs("No Copy on resume") + .isEqualTo(0); + Assertions.assertThat(pendingJsonDeleted.get()) + .describedAs("RenamePendingJson should be deleted") + .isEqualTo(1); + } + + /** + * Test to assert that the CID in src marker blob copy and delete contains the + * total number of blobs operated in the rename directory. + * Also, to assert that all operations in the rename-directory flow have same + * primaryId and opType. + */ + @Test + public void testRenameSrcDirDeleteEmitDeletionCountInClientRequestId() + throws Exception { + AzureBlobFileSystem fs = Mockito.spy(getFileSystem()); + assumeNonHnsAccountBlobEndpoint(fs); + AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs); + String dirPathStr = "/testDir/dir1"; + fs.mkdirs(new Path(dirPathStr)); + ExecutorService executorService = Executors.newFixedThreadPool(5); + List futures = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + final int iter = i; + Future future = executorService.submit(() -> + fs.create(new Path("/testDir/dir1/file" + iter))); + futures.add(future); + } + for (Future future : futures) { + future.get(); + } + executorService.shutdown(); + final TracingHeaderValidator tracingHeaderValidator + = new TracingHeaderValidator( + fs.getAbfsStore().getAbfsConfiguration().getClientCorrelationId(), + fs.getFileSystemId(), FSOperationType.RENAME, true, 0); + fs.registerListener(tracingHeaderValidator); + Mockito.doAnswer(copyAnswer -> { + if (dirPathStr.equalsIgnoreCase( + ((Path) copyAnswer.getArgument(0)).toUri().getPath())) { + tracingHeaderValidator.setOperatedBlobCount(BLOB_COUNT); + return copyAnswer.callRealMethod(); + } + return copyAnswer.callRealMethod(); + }) + .when(client) + .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class), + Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + Mockito.doAnswer(deleteAnswer -> { + if (dirPathStr.equalsIgnoreCase( + ((Path) deleteAnswer.getArgument(0)).toUri().getPath())) { + Object result = deleteAnswer.callRealMethod(); + tracingHeaderValidator.setOperatedBlobCount(null); + return result; + } + return deleteAnswer.callRealMethod(); + }) + .when(client) + .deleteBlobPath(Mockito.any(Path.class), + Mockito.nullable(String.class), + Mockito.any(TracingContext.class)); + fs.rename(new Path(dirPathStr), new Path("/dst/")); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java index f913da7b15ed0..52a5acf109e96 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRenameUnicode.java @@ -26,13 +26,19 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.services.AbfsBlobClient; +import static java.net.HttpURLConnection.HTTP_BAD_REQUEST; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COLON; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Parameterized test of rename operations of unicode paths. @@ -84,6 +90,17 @@ public void testRenameFileUsingUnicode() throws Exception { assertIsFile(fs, filePath); Path folderPath2 = new Path(destDir); + if (getFileSystem().getAbfsClient() instanceof AbfsBlobClient + && destDir.contains(COLON)) { + AbfsRestOperationException ex = intercept( + AbfsRestOperationException.class, () -> { + fs.rename(folderPath1, folderPath2); + return null; + }); + assertTrue(ex.getCause() instanceof PathIOException); + assertEquals(HTTP_BAD_REQUEST, ex.getStatusCode()); + return; + } assertRenameOutcome(fs, folderPath1, folderPath2, true); assertPathDoesNotExist(fs, "renamed", folderPath1); assertIsDirectory(fs, folderPath2); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java index a153d9f3027ea..b81cf9ba00832 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java @@ -201,4 +201,53 @@ public Object answer(final InvocationOnMock invocationOnMock) Mockito.anyString(), Mockito.any(URL.class), Mockito.anyList(), Mockito.nullable(String.class)); } + + /** + * Mocks the `getBlobDeleteHandler` method of `AbfsBlobClient` to apply a custom handler + * for the delete operation. This allows for controlling the behavior of the delete + * process during testing. + * + * @param blobClient the `AbfsBlobClient` instance to mock + * @param functionRaisingIOE the function to apply to the mocked `BlobDeleteHandler` + */ + public static void mockGetDeleteBlobHandler(AbfsBlobClient blobClient, + FunctionRaisingIOE functionRaisingIOE) { + Mockito.doAnswer(answer -> { + BlobDeleteHandler blobDeleteHandler = Mockito.spy( + (BlobDeleteHandler) answer.callRealMethod()); + Mockito.doAnswer(answer1 -> { + functionRaisingIOE.apply(blobDeleteHandler); + return answer1.callRealMethod(); + }).when(blobDeleteHandler).execute(); + return blobDeleteHandler; + }) + .when(blobClient) + .getBlobDeleteHandler(Mockito.anyString(), Mockito.anyBoolean(), + Mockito.any(TracingContext.class)); + } + + /** + * Mocks the `getBlobRenameHandler` method of `AbfsBlobClient` to apply a custom handler + * for the rename operation. This allows for controlling the behavior of the rename + * process during testing. + * + * @param blobClient the `AbfsBlobClient` instance to mock + * @param functionRaisingIOE the function to apply to the mocked `BlobRenameHandler` + */ + public static void mockGetRenameBlobHandler(AbfsBlobClient blobClient, + FunctionRaisingIOE functionRaisingIOE) { + Mockito.doAnswer(answer -> { + BlobRenameHandler blobRenameHandler = Mockito.spy( + (BlobRenameHandler) answer.callRealMethod()); + Mockito.doAnswer(answer1 -> { + functionRaisingIOE.apply(blobRenameHandler); + return answer1.callRealMethod(); + }).when(blobRenameHandler).execute(); + return blobRenameHandler; + }) + .when(blobClient) + .getBlobRenameHandler(Mockito.anyString(), Mockito.anyString(), + Mockito.nullable(String.class), Mockito.anyBoolean(), + Mockito.any(TracingContext.class)); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java index 7a05bd4129d58..f706395e69e6a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java @@ -452,7 +452,6 @@ public static AbfsClient getMockAbfsClient(AbfsClient baseAbfsClientInstance, (currentAuthType == AuthType.SharedKey) || (currentAuthType == AuthType.OAuth)); - // TODO : [FnsOverBlob][HADOOP-19234] Update to work with Blob Endpoint as well when Fns Over Blob is ready. AbfsClient client = mock(AbfsDfsClient.class); AbfsPerfTracker tracker = new AbfsPerfTracker( "test", diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsPaginatedDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsPaginatedDelete.java index 8c1fcee5f6fa8..eb06b81621898 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsPaginatedDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsPaginatedDelete.java @@ -142,6 +142,7 @@ private void setTestUserConf(Configuration conf, String key, String value) { */ @Test public void testRecursiveDeleteWithPagination() throws Exception { + assertTrue(getFileSystem().getAbfsStore().getClient() instanceof AbfsDfsClient); testRecursiveDeleteWithPaginationInternal(false, true, AbfsHttpConstants.ApiVersion.DEC_12_2019); testRecursiveDeleteWithPaginationInternal(false, true, @@ -164,6 +165,7 @@ public void testRecursiveDeleteWithPagination() throws Exception { */ @Test public void testNonRecursiveDeleteWithPagination() throws Exception { + assertTrue(getFileSystem().getAbfsStore().getClient() instanceof AbfsDfsClient); testNonRecursiveDeleteWithPaginationInternal(true); testNonRecursiveDeleteWithPaginationInternal(false); } @@ -174,6 +176,7 @@ public void testNonRecursiveDeleteWithPagination() throws Exception { */ @Test public void testRecursiveDeleteWithInvalidCT() throws Exception { + assertTrue(getFileSystem().getAbfsStore().getClient() instanceof AbfsDfsClient); testRecursiveDeleteWithInvalidCTInternal(true); testRecursiveDeleteWithInvalidCTInternal(false); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicityTestUtils.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicityTestUtils.java new file mode 100644 index 0000000000000..e02d9a00bda89 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicityTestUtils.java @@ -0,0 +1,79 @@ +/** + * 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.services; + +import org.mockito.Mockito; +import org.mockito.stubbing.Answer; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; + +public final class RenameAtomicityTestUtils { + + private RenameAtomicityTestUtils() { + } + + /** + * Creates a spied object of {@link BlobRenameHandler} and {@link RenameAtomicity} + * and adds mocked behavior to {@link RenameAtomicity#createRenamePendingJson(Path, byte[])}. + * + * @param client client that would supply BlobRenameHandler and RenameAtomicity. + * @param answer mocked behavior for {@link RenameAtomicity#createRenamePendingJson(Path, byte[])}. + */ + public static void addCreatePathMock(AbfsBlobClient client, Answer answer) { + Mockito.doAnswer(clientHandlerAns -> { + BlobRenameHandler renameHandler = Mockito.spy( + (BlobRenameHandler) clientHandlerAns.callRealMethod()); + Mockito.doAnswer(getRenameAtomicityAns -> { + RenameAtomicity renameAtomicity = Mockito.spy( + (RenameAtomicity) getRenameAtomicityAns.callRealMethod()); + Mockito.doAnswer(answer) + .when(renameAtomicity) + .createRenamePendingJson(Mockito.any( + Path.class), Mockito.any(byte[].class)); + return renameAtomicity; + }) + .when(renameHandler) + .getRenameAtomicity(Mockito.any(PathInformation.class)); + return renameHandler; + }) + .when(client) + .getBlobRenameHandler(Mockito.anyString(), Mockito.anyString(), + Mockito.nullable(String.class), Mockito.anyBoolean(), Mockito.any( + TracingContext.class)); + } + + + /** + * Adds mocked behavior to {@link RenameAtomicity#readRenamePendingJson(Path, int)}. + * + * @param redoRenameAtomicity {@link RenameAtomicity} to be spied. + * @param answer mocked behavior for {@link RenameAtomicity#readRenamePendingJson(Path, int)}. + * + * @throws AzureBlobFileSystemException server error or error from mocked behavior. + */ + public static void addReadPathMock(RenameAtomicity redoRenameAtomicity, + Answer answer) + throws AzureBlobFileSystemException { + Mockito.doAnswer(answer) + .when(redoRenameAtomicity) + .readRenamePendingJson(Mockito.any(Path.class), Mockito.anyInt()); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java index 741459254d400..1d8f12842ab99 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java @@ -434,6 +434,9 @@ public void testExistingPathCorrectlyRejected() throws Exception { @Test public void testRenameRecoveryUnsupportedForFlatNamespace() throws Exception { Assume.assumeTrue(!isNamespaceEnabled); + // In DFS endpoint, renamePath is O(1) API call and idempotency issue can happen. + // For blob endpoint, client orchestrates the rename operation. + assertTrue(getFileSystem().getAbfsStore().getClient() instanceof AbfsDfsClient); AzureBlobFileSystem fs = getFileSystem(); AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); TracingContext testTracingContext = getTestTracingContext(fs, false); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java index 27a84e4978ad2..a8211e33f9f8a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java @@ -39,6 +39,8 @@ public class TracingHeaderValidator implements Listener { private static final String GUID_PATTERN = "^[0-9a-fA-F]{8}-([0-9a-fA-F]{4}-){3}[0-9a-fA-F]{12}$"; + private Integer operatedBlobCount = null; + @Override public void callTracingHeaderValidator(String tracingContextHeader, TracingHeaderFormat format) { @@ -52,6 +54,7 @@ public TracingHeaderValidator getClone() { clientCorrelationId, fileSystemId, operation, needsPrimaryRequestId, retryNum, streamID); tracingHeaderValidator.primaryRequestId = primaryRequestId; + tracingHeaderValidator.operatedBlobCount = operatedBlobCount; return tracingHeaderValidator; } @@ -78,6 +81,13 @@ private void validateTracingHeader(String tracingContextHeader) { if (format != TracingHeaderFormat.ALL_ID_FORMAT) { return; } + if (idList.length >= 8) { + if (operatedBlobCount != null) { + Assertions.assertThat(Integer.parseInt(idList[7])) + .describedAs("OperatedBlobCount is incorrect") + .isEqualTo(operatedBlobCount); + } + } if (!primaryRequestId.isEmpty() && !idList[3].isEmpty()) { Assertions.assertThat(idList[3]) .describedAs("PrimaryReqID should be common for these requests") @@ -152,4 +162,8 @@ public void setOperation(FSOperationType operation) { public void updatePrimaryRequestID(String primaryRequestId) { this.primaryRequestId = primaryRequestId; } + + public void setOperatedBlobCount(Integer operatedBlobCount) { + this.operatedBlobCount = operatedBlobCount; + } }