+ */
+ public static final Integer HTTP_STATUS_CATEGORY_QUOTIENT = 100;
+
+ public static final String BLOB = "Blob";
+ private static final String PREFIX = "Prefix";
+ public static final String NEXT_MARKER = "NextMarker";
+ public static final String METADATA = "Metadata";
+ public static final String PROPERTIES = "Properties";
+ public static final String BLOB_PREFIX = "BlobPrefix";
+ public static final String CONTENT_LEN = "Content-Length";
+ public static final String RESOURCE_TYPE = "ResourceType";
+ public static final String INVALID_XML = "Invalid XML";
+ 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 HDI_ISFOLDER = "hdi_isfolder";
+ public static final Integer BLOB_LEASE_ONE_MINUTE_DURATION = 60;
+ public static final String ETAG = "Etag";
+ public static final String LAST_MODIFIED_TIME = "Last-Modified";
+ public static final String CREATION_TIME = "Creation-Time";
+ public static final String OWNER = "Owner";
+ public static final String GROUP = "Group";
+ public static final String PERMISSIONS = "Permissions";
+ public static final String ACL = "Acl";
+ public static final String COPY_ID = "CopyId";
+ public static final String COPY_STATUS = "CopyStatus";
+ public static final String COPY_SOURCE = "CopySource";
+ public static final String COPY_PROGRESS = "CopyProgress";
+ public static final String COPY_COMPLETION_TIME = "CopyCompletionTime";
+ public static final String COPY_STATUS_DESCRIPTION = "CopyStatusDescription";
+ public static final String BLOB_ERROR_CODE_START_XML = "";
+ public static final String BLOB_ERROR_CODE_END_XML = "";
+ public static final String BLOB_ERROR_MESSAGE_START_XML = "";
+ public static final String BLOB_ERROR_MESSAGE_END_XML = "";
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 12beb5a9bbabe8..7cf95261b9dfdc 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
@@ -35,9 +35,15 @@ public final class ConfigurationKeys {
* path to determine HNS status.
*/
public static final String FS_AZURE_ACCOUNT_IS_HNS_ENABLED = "fs.azure.account.hns.enabled";
+ /**
+ * Enable or disable expect hundred continue header.
+ * Value: {@value}.
+ */
+ public static final String FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED = "fs.azure.account.expect.header.enabled";
public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME = "fs.azure.account.key";
public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX = "fs\\.azure\\.account\\.key\\.(.*)";
public static final String FS_AZURE_SECURE_MODE = "fs.azure.secure.mode";
+ public static final String FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED = "fs.azure.account.throttling.enabled";
// Retry strategy defined by the user
public static final String AZURE_MIN_BACKOFF_INTERVAL = "fs.azure.io.retry.min.backoff.interval";
@@ -79,7 +85,7 @@ public final class ConfigurationKeys {
/**
* What data block buffer to use.
*
- * Options include: "disk"(Default), "array", and "bytebuffer".
+ * Options include: "disk", "array", and "bytebuffer"(Default).
*
* Default is {@link FileSystemConfigurations#DATA_BLOCKS_BUFFER_DEFAULT}.
* Value: {@value}
@@ -116,6 +122,8 @@ public final class ConfigurationKeys {
public static final String AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION = "fs.azure.createRemoteFileSystemDuringInitialization";
public static final String AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION = "fs.azure.skipUserGroupMetadataDuringInitialization";
public static final String FS_AZURE_ENABLE_AUTOTHROTTLING = "fs.azure.enable.autothrottling";
+ public static final String FS_AZURE_ACCOUNT_OPERATION_IDLE_TIMEOUT = "fs.azure.account.operation.idle.timeout";
+ public static final String FS_AZURE_ANALYSIS_PERIOD = "fs.azure.analysis.period";
public static final String FS_AZURE_ALWAYS_USE_HTTPS = "fs.azure.always.use.https";
public static final String FS_AZURE_ATOMIC_RENAME_KEY = "fs.azure.atomic.rename.key";
/** This config ensures that during create overwrite an existing file will be
@@ -123,6 +131,7 @@ public final class ConfigurationKeys {
*/
public static final String FS_AZURE_ENABLE_CONDITIONAL_CREATE_OVERWRITE = "fs.azure.enable.conditional.create.overwrite";
public static final String FS_AZURE_ENABLE_MKDIR_OVERWRITE = "fs.azure.enable.mkdir.overwrite";
+ public static final String FS_AZURE_ENABLE_BLOB_MKDIR_OVERWRITE = "fs.azure.enable.blob.mkdir.overwrite";
/** Provides a config to provide comma separated path prefixes on which Appendblob based files are created
* Default is empty. **/
public static final String FS_AZURE_APPEND_BLOB_KEY = "fs.azure.appendblob.directories";
@@ -186,6 +195,13 @@ public final class ConfigurationKeys {
public static final String FS_AZURE_SKIP_SUPER_USER_REPLACEMENT = "fs.azure.identity.transformer.skip.superuser.replacement";
public static final String AZURE_KEY_ACCOUNT_KEYPROVIDER = "fs.azure.account.keyprovider";
public static final String AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT = "fs.azure.shellkeyprovider.script";
+
+ /**
+ * Enable or disable readahead buffer in AbfsInputStream.
+ * Value: {@value}.
+ */
+ public static final String FS_AZURE_ENABLE_READAHEAD = "fs.azure.enable.readahead";
+
/** Setting this true will make the driver use it's own RemoteIterator implementation */
public static final String FS_AZURE_ENABLE_ABFS_LIST_ITERATOR = "fs.azure.enable.abfslistiterator";
/** Server side encryption key */
@@ -229,6 +245,8 @@ public static String accountProperty(String property, String account) {
/** Key for SAS token provider **/
public static final String FS_AZURE_SAS_TOKEN_PROVIDER_TYPE = "fs.azure.sas.token.provider.type";
+ /** Key for fixed SAS token **/
+ public static final String FS_AZURE_SAS_FIXED_TOKEN = "fs.azure.sas.fixed.token";
/** For performance, AbfsInputStream/AbfsOutputStream re-use SAS tokens until the expiry is within this number of seconds. **/
public static final String FS_AZURE_SAS_TOKEN_RENEW_PERIOD_FOR_STREAMS = "fs.azure.sas.token.renew.period.for.streams";
@@ -246,5 +264,19 @@ public static String accountProperty(String property, String account) {
* @see FileSystem#openFile(org.apache.hadoop.fs.Path)
*/
public static final String FS_AZURE_BUFFERED_PREAD_DISABLE = "fs.azure.buffered.pread.disable";
+ public static final String FS_AZURE_BLOB_DIR_RENAME_MAX_THREAD = "fs.azure.blob.dir.rename.max.thread";
+ public static final String FS_AZURE_BLOB_DIR_DELETE_MAX_THREAD = "fs.azure.blob.dir.delete.max.thread";
+ public static final String FS_AZURE_BLOB_COPY_PROGRESS_POLL_WAIT_MILLIS = "fs.azure.blob.copy.progress.poll.wait.millis";
+
+ public static final String FS_AZURE_ENABLE_BLOB_ENDPOINT = "fs.azure.enable.blob.endpoint";
+ public static final String FS_AZURE_MKDIRS_FALLBACK_TO_DFS = "fs.azure.mkdirs.fallback.to.dfs";
+ public static final String FS_AZURE_INGRESS_FALLBACK_TO_DFS = "fs.azure.ingress.fallback.to.dfs";
+ public static final String FS_AZURE_READ_FALLBACK_TO_DFS = "fs.azure.read.fallback.to.dfs";
+
+ public static final String FS_AZURE_REDIRECT_DELETE = "fs.azure.redirect.delete";
+ public static final String FS_AZURE_REDIRECT_RENAME = "fs.azure.redirect.rename";
+ public static final String FS_AZURE_PRODUCER_QUEUE_MAX_SIZE = "fs.azure.blob.dir.list.producer.queue.max.size";
+ public static final String FS_AZURE_LEASE_CREATE_NON_RECURSIVE = "fs.azure.lease.create.non.recursive";
+
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 f58c61e8908a69..8b96a7a6acd931 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
@@ -32,7 +32,7 @@
public final class FileSystemConfigurations {
public static final String DEFAULT_FS_AZURE_ACCOUNT_IS_HNS_ENABLED = "";
-
+ 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 * 1000;
@@ -78,8 +78,10 @@ public final class FileSystemConfigurations {
public static final boolean DEFAULT_AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION = false;
public static final String DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES = "/hbase";
+ public static final String HBASE_ROOT = "/hbase";
public static final boolean DEFAULT_FS_AZURE_ENABLE_CONDITIONAL_CREATE_OVERWRITE = true;
public static final boolean DEFAULT_FS_AZURE_ENABLE_MKDIR_OVERWRITE = true;
+ public static final boolean DEFAULT_FS_AZURE_BLOB_ENABLE_MKDIR_OVERWRITE = false;
public static final String DEFAULT_FS_AZURE_APPEND_BLOB_DIRECTORIES = "";
public static final String DEFAULT_FS_AZURE_INFINITE_LEASE_DIRECTORIES = "";
public static final int DEFAULT_LEASE_THREADS = 0;
@@ -91,9 +93,14 @@ public final class FileSystemConfigurations {
public static final int DEFAULT_READ_AHEAD_QUEUE_DEPTH = 2;
+ public static final String IS_FOLDER_METADATA_KEY = "hdi_isfolder";
+
public static final boolean DEFAULT_ENABLE_FLUSH = true;
public static final boolean DEFAULT_DISABLE_OUTPUTSTREAM_FLUSH = true;
public static final boolean DEFAULT_ENABLE_AUTOTHROTTLING = true;
+ public static final boolean DEFAULT_FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED = true;
+ public static final int DEFAULT_ACCOUNT_OPERATION_IDLE_TIMEOUT_MS = 60_000;
+ public static final int DEFAULT_ANALYSIS_PERIOD_MS = 10_000;
public static final DelegatingSSLSocketFactory.SSLChannelMode DEFAULT_FS_AZURE_SSL_CHANNEL_MODE
= DelegatingSSLSocketFactory.SSLChannelMode.Default;
@@ -106,6 +113,7 @@ public final class FileSystemConfigurations {
public static final boolean DEFAULT_ABFS_LATENCY_TRACK = false;
public static final long DEFAULT_SAS_TOKEN_RENEW_PERIOD_FOR_STREAMS_IN_SECONDS = 120;
+ public static final boolean DEFAULT_ENABLE_READAHEAD = true;
public static final String DEFAULT_FS_AZURE_USER_AGENT_PREFIX = EMPTY_STRING;
public static final String DEFAULT_VALUE_UNKNOWN = "UNKNOWN";
@@ -115,6 +123,19 @@ public final class FileSystemConfigurations {
public static final int STREAM_ID_LEN = 12;
public static final boolean DEFAULT_ENABLE_ABFS_LIST_ITERATOR = true;
+ public static final boolean DEFAULT_FS_AZURE_ENABLE_BLOBENDPOINT = false;
+ public static final boolean DEFAULT_FS_AZURE_MKDIRS_FALLBACK_TO_DFS = false;
+ public static final boolean DEFAULT_FS_AZURE_INGRESS_FALLBACK_TO_DFS = false;
+ public static final boolean DEFAULT_AZURE_READ_FALLBACK_TO_DFS = false;
+
+ // To have functionality similar to drop1 delete is going to wasb by default for now.
+ public static final boolean DEFAULT_FS_AZURE_REDIRECT_RENAME = false;
+ public static final boolean DEFAULT_FS_AZURE_REDIRECT_DELETE = false;
+ public static final int DEFAULT_FS_AZURE_PRODUCER_QUEUE_MAX_SIZE = 10000;
+ public static final boolean DEFAULT_FS_AZURE_LEASE_CREATE_NON_RECURSIVE = false;
+ public static final int DEFAULT_FS_AZURE_BLOB_RENAME_THREAD = 5;
+ public static final int DEFAULT_FS_AZURE_BLOB_DELETE_THREAD = 5;
+
/**
* Limit of queued block upload operations before writes
* block for an OutputStream. Value: {@value}
@@ -127,11 +148,16 @@ public final class FileSystemConfigurations {
*/
public static final String DATA_BLOCKS_BUFFER_DISK = "disk";
+ public static final String DATA_BLOCKS_BYTEBUFFER = "bytebuffer";
+
/**
* Default buffer option: {@value}.
*/
public static final String DATA_BLOCKS_BUFFER_DEFAULT =
- DATA_BLOCKS_BUFFER_DISK;
+ DATA_BLOCKS_BYTEBUFFER;
+
+ /** The blockId of each block should be of the same length. */
+ 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/FileSystemUriSchemes.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemUriSchemes.java
index c7a0cdad605ab2..573c819fc29f2f 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemUriSchemes.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemUriSchemes.java
@@ -29,14 +29,14 @@
public final class FileSystemUriSchemes {
public static final String ABFS_SCHEME = "abfs";
public static final String ABFS_SECURE_SCHEME = "abfss";
- public static final String ABFS_DNS_PREFIX = "dfs";
+ public static final String ABFS_DNS_PREFIX = ".dfs.";
public static final String HTTP_SCHEME = "http";
public static final String HTTPS_SCHEME = "https";
public static final String WASB_SCHEME = "wasb";
public static final String WASB_SECURE_SCHEME = "wasbs";
- public static final String WASB_DNS_PREFIX = "blob";
+ public static final String WASB_DNS_PREFIX = ".blob.";
private FileSystemUriSchemes() {}
-}
\ No newline at end of file
+}
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 d4065ac2836d02..cd3c321b4395d0 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
@@ -68,8 +68,17 @@ public final class HttpHeaderConfigurations {
public static final String X_MS_LEASE_ACTION = "x-ms-lease-action";
public static final String X_MS_LEASE_DURATION = "x-ms-lease-duration";
public static final String X_MS_LEASE_ID = "x-ms-lease-id";
+ public static final String X_MS_SOURCE_LEASE_ID = "x-ms-source-lease-id";
public static final String X_MS_PROPOSED_LEASE_ID = "x-ms-proposed-lease-id";
public static final String X_MS_LEASE_BREAK_PERIOD = "x-ms-lease-break-period";
+ public static final String X_MS_BLOB_TYPE = "x-ms-blob-type";
+ public static final String X_MS_META_HDI_ISFOLDER = "x-ms-meta-hdi_isfolder";
+ public static final String X_MS_COPY_ID = "x-ms-copy-id";
+ public static final String X_MS_COPY_SOURCE = "x-ms-copy-source";
+ public static final String X_MS_COPY_STATUS_DESCRIPTION = "x-ms-copy-status-description";
+ public static final String X_MS_COPY_STATUS = "x-ms-copy-status";
+ public static final String EXPECT = "Expect";
+ public static final String X_MS_METADATA_PREFIX = "x-ms-meta-";
private HttpHeaderConfigurations() {}
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java
index e9bb95cad21cd6..1b493e590b9b76 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java
@@ -27,6 +27,14 @@
@InterfaceStability.Evolving
public final class HttpQueryParams {
public static final String QUERY_PARAM_RESOURCE = "resource";
+ public static final String QUERY_PARAM_RESTYPE = "restype";
+ public static final String QUERY_PARAM_COMP = "comp";
+ public static final String QUERY_PARAM_COMP_LEASE_VALUE = "lease";
+ public static final String QUERY_PARAM_COMP_VALUE_LIST = "list";
+ public static final String QUERY_PARAM_PREFIX = "prefix";
+ public static final String QUERY_PARAM_MARKER = "marker";
+ public static final String QUERY_PARAM_MAXRESULT = "maxresults";
+ public static final String QUERY_PARAM_BLOCKLISTTYPE = "blocklisttype";
public static final String QUERY_PARAM_DIRECTORY = "directory";
public static final String QUERY_PARAM_CONTINUATION = "continuation";
public static final String QUERY_PARAM_RECURSIVE = "recursive";
@@ -40,12 +48,17 @@ public final class HttpQueryParams {
public static final String QUERY_PARAM_CLOSE = "close";
public static final String QUERY_PARAM_UPN = "upn";
public static final String QUERY_PARAM_BLOBTYPE = "blobtype";
+ public static final String QUERY_PARAM_BLOCKID = "blockid";
+ public static final String QUERY_PARAM_DELIMITER = "delimiter";
//query params for SAS
public static final String QUERY_PARAM_SAOID = "saoid";
public static final String QUERY_PARAM_SKOID = "skoid";
public static final String QUERY_PARAM_SUOID = "suoid";
public static final String QUERY_PARAM_SIGNATURE = "sig";
+ public static final String QUERY_PARAM_INCLUDE = "include";
+ public static final String QUERY_PARAM_INCLUDE_VALUE_METADATA = "metadata";
+ public static final String CONTAINER = "container";
private HttpQueryParams() {}
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/InternalConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/InternalConstants.java
new file mode 100644
index 00000000000000..12d4f14d92a00b
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/InternalConstants.java
@@ -0,0 +1,46 @@
+/*
+ * 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.constants;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Constants which are used internally and which don't fit into the other
+ * classes.
+ * For use within the {@code hadoop-azure} module only.
+ */
+@InterfaceAudience.Private
+public final class InternalConstants {
+
+ private InternalConstants() {
+ }
+
+ /**
+ * Does this version of the store have safe readahead?
+ * Possible combinations of this and the probe
+ * {@code "fs.capability.etags.available"}.
+ *
+ *
{@value}: store is safe
+ *
!etags: store is safe
+ *
etags && !{@value}: store is UNSAFE
+ *
+ */
+ public static final String CAPABILITY_SAFE_READAHEAD =
+ "fs.azure.capability.readahead.safe";
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsRestOperationException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsRestOperationException.java
index 6c53762363840a..2ac325ba3181e8 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsRestOperationException.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsRestOperationException.java
@@ -84,18 +84,20 @@ private static String formatMessage(final AbfsHttpOperation abfsHttpOperation) {
// HEAD request response doesn't have StorageErrorCode, StorageErrorMessage.
if (abfsHttpOperation.getMethod().equals("HEAD")) {
return String.format(
- "Operation failed: \"%1$s\", %2$s, HEAD, %3$s",
+ "Operation failed: \"%1$s\", %2$s, HEAD, %3$s, rId: %4$s",
abfsHttpOperation.getStatusDescription(),
abfsHttpOperation.getStatusCode(),
- abfsHttpOperation.getMaskedUrl());
+ abfsHttpOperation.getMaskedUrl(),
+ abfsHttpOperation.getRequestId());
}
return String.format(
- "Operation failed: \"%1$s\", %2$s, %3$s, %4$s, %5$s, \"%6$s\"",
+ "Operation failed: \"%1$s\", %2$s, %3$s, %4$s, rId: %5$s, %6$s, \"%7$s\"",
abfsHttpOperation.getStatusDescription(),
abfsHttpOperation.getStatusCode(),
abfsHttpOperation.getMethod(),
abfsHttpOperation.getMaskedUrl(),
+ abfsHttpOperation.getRequestId(),
abfsHttpOperation.getStorageErrorCode(),
// Remove break line to ensure the request id and timestamp can be shown in console.
abfsHttpOperation.getStorageErrorMessage().replaceAll("\\n", " "));
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java
index aba1d8c1efa2b3..147cb6d83cb0ba 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java
@@ -29,12 +29,33 @@
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class InvalidAbfsRestOperationException extends AbfsRestOperationException {
+
+ private static final String ERROR_MESSAGE = "InvalidAbfsRestOperationException";
+
public InvalidAbfsRestOperationException(
final Exception innerException) {
super(
AzureServiceErrorCode.UNKNOWN.getStatusCode(),
AzureServiceErrorCode.UNKNOWN.getErrorCode(),
- "InvalidAbfsRestOperationException",
+ innerException != null
+ ? innerException.toString()
+ : ERROR_MESSAGE,
innerException);
}
+
+ /**
+ * Adds the retry count along with the exception.
+ * @param innerException The inner exception which is originally caught.
+ * @param retryCount The retry count when the exception was thrown.
+ */
+ public InvalidAbfsRestOperationException(
+ final Exception innerException, int retryCount) {
+ super(
+ AzureServiceErrorCode.UNKNOWN.getStatusCode(),
+ AzureServiceErrorCode.UNKNOWN.getErrorCode(),
+ innerException != null
+ ? innerException.toString()
+ : ERROR_MESSAGE + " RetryCount: " + retryCount,
+ innerException);
+ }
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java
index 7369bfaf56422c..170a331df74f15 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AppendRequestParameters.java
@@ -34,19 +34,24 @@ public enum Mode {
private final Mode mode;
private final boolean isAppendBlob;
private final String leaseId;
+ private boolean isExpectHeaderEnabled;
+ private boolean isRetryDueToExpect;
public AppendRequestParameters(final long position,
final int offset,
final int length,
final Mode mode,
final boolean isAppendBlob,
- final String leaseId) {
+ final String leaseId,
+ final boolean isExpectHeaderEnabled) {
this.position = position;
this.offset = offset;
this.length = length;
this.mode = mode;
this.isAppendBlob = isAppendBlob;
this.leaseId = leaseId;
+ this.isExpectHeaderEnabled = isExpectHeaderEnabled;
+ this.isRetryDueToExpect = false;
}
public long getPosition() {
@@ -72,4 +77,20 @@ public boolean isAppendBlob() {
public String getLeaseId() {
return this.leaseId;
}
+
+ public boolean isExpectHeaderEnabled() {
+ return isExpectHeaderEnabled;
+ }
+
+ public boolean isRetryDueToExpect() {
+ return isRetryDueToExpect;
+ }
+
+ public void setExpectHeaderEnabled(boolean expectHeaderEnabled) {
+ isExpectHeaderEnabled = expectHeaderEnabled;
+ }
+
+ public void setRetryDueToExpect(boolean retryDueToExpect) {
+ isRetryDueToExpect = retryDueToExpect;
+ }
}
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 8bc31c4f92b2a3..ef4f7dc1773ae3 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
@@ -47,6 +47,8 @@ public enum AzureServiceErrorCode {
INVALID_QUERY_PARAMETER_VALUE("InvalidQueryParameterValue", HttpURLConnection.HTTP_BAD_REQUEST, null),
AUTHORIZATION_PERMISSION_MISS_MATCH("AuthorizationPermissionMismatch", HttpURLConnection.HTTP_FORBIDDEN, null),
ACCOUNT_REQUIRES_HTTPS("AccountRequiresHttps", HttpURLConnection.HTTP_BAD_REQUEST, null),
+ COPY_BLOB_FAILED("COPY_BLOB_FAILED", HttpURLConnection.HTTP_INTERNAL_ERROR, null),
+ COPY_BLOB_ABORTED("COPY_BLOB_ABORTED", HttpURLConnection.HTTP_INTERNAL_ERROR, null),
UNKNOWN(null, -1, null);
private final String errorCode;
@@ -66,6 +68,10 @@ public String getErrorCode() {
return this.errorCode;
}
+ public String getErrorMessage() {
+ return this.errorMessage;
+ }
+
public static List getAzureServiceCode(int httpStatusCode) {
List errorCodes = new ArrayList<>();
if (httpStatusCode == UNKNOWN.httpStatusCode) {
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 00000000000000..16ab2a0ac02628
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/enums/BlobCopyProgress.java
@@ -0,0 +1,8 @@
+package org.apache.hadoop.fs.azurebfs.enums;
+
+public enum BlobCopyProgress {
+ SUCCESS,
+ FAILURE,
+ ABORTED,
+ PENDING;
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/SASTokenProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/SASTokenProvider.java
index a2cd292b0b2309..98846e7bb8cfd9 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/SASTokenProvider.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/SASTokenProvider.java
@@ -36,11 +36,22 @@ public interface SASTokenProvider {
String CREATE_DIRECTORY_OPERATION = "create-directory";
String CREATE_FILE_OPERATION = "create-file";
String DELETE_OPERATION = "delete";
+ String DELETE_BLOB_OPERATION = "delete-blob";
+ String DELETE_CONTAINER_OPERATION = "delete-container";
+ String CREATE_CONTAINER_OPERATION = "create-container";
String DELETE_RECURSIVE_OPERATION = "delete-recursive";
String GET_ACL_OPERATION = "get-acl";
String GET_STATUS_OPERATION = "get-status";
+ String GET_BLOCK_LIST = "get-block-list";
String GET_PROPERTIES_OPERATION = "get-properties";
+ String GET_BLOB_PROPERTIES_OPERATION = "get-blob-properties";
+ String GET_CONTAINER_PROPERTIES_OPERATION = "get-container-properties";
+ String GET_BLOB_METADATA_OPERATION = "get-blob-metadata";
+ String GET_CONTAINER_METADATA_OPERATION = "get-container-metadata";
String LIST_OPERATION = "list";
+ String LIST_BLOB_OPERATION = "list-blob";
+ String COPY_BLOB_DESTINATION = "copy-blob-dst";
+ String COPY_BLOB_SOURCE = "copy-blob-src";
String READ_OPERATION = "read";
String RENAME_SOURCE_OPERATION = "rename-source";
String RENAME_DESTINATION_OPERATION = "rename-destination";
@@ -48,8 +59,12 @@ public interface SASTokenProvider {
String SET_OWNER_OPERATION = "set-owner";
String SET_PERMISSION_OPERATION = "set-permission";
String SET_PROPERTIES_OPERATION = "set-properties";
+ String SET_BLOB_METADATA_OPERATION = "set-blob-metadata";
+ String SET_CONTAINER_METADATA_OPERATION = "set-container-metadata";
String WRITE_OPERATION = "write";
+ String LEASE_OPERATION = "lease";
+
/**
* Initialize authorizer for Azure Blob File System.
* @param configuration Configuration object
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java
index 49f90feb22e95c..fa77d09f546c05 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java
@@ -212,7 +212,7 @@ public String getRequestId() {
return this.requestId;
}
- protected HttpException(
+ public HttpException(
final int httpErrorCode,
final String requestId,
final String message,
@@ -340,7 +340,7 @@ private static boolean isRecoverableFailure(IOException e) {
|| e instanceof FileNotFoundException);
}
- private static AzureADToken getTokenSingleCall(String authEndpoint,
+ public static AzureADToken getTokenSingleCall(String authEndpoint,
String payload, Hashtable headers, String httpMethod,
boolean isMsi)
throws IOException {
@@ -386,6 +386,7 @@ private static AzureADToken getTokenSingleCall(String authEndpoint,
long responseContentLength = conn.getHeaderFieldLong("Content-Length", 0);
requestId = requestId == null ? "" : requestId;
+ LOG.debug("The res " + responseContentType);
if (httpResponseCode == HttpURLConnection.HTTP_OK
&& responseContentType.startsWith("application/json") && responseContentLength > 0) {
InputStream httpResponseStream = conn.getInputStream();
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADToken.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADToken.java
index daa5a93bf6cdce..a3f15ca27c3d33 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADToken.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADToken.java
@@ -18,6 +18,10 @@
package org.apache.hadoop.fs.azurebfs.oauth2;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
import java.util.Date;
@@ -25,10 +29,15 @@
* Object representing the AAD access token to use when making HTTP requests to Azure Data Lake Storage.
*/
public class AzureADToken {
+ private static final Logger LOG = LoggerFactory.getLogger(AzureADAuthenticator.class);
private String accessToken;
private Date expiry;
- public String getAccessToken() {
+ public String getAccessToken() throws IOException {
+ if (accessToken == null || accessToken.length() == 0) {
+ LOG.debug("The access token value obtained is empty");
+ throw new IOException("The token value obtained is empty");
+ }
return this.accessToken;
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdapter.java
index 14914101e5cca5..737ccacbfd4fc7 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdapter.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdapter.java
@@ -72,6 +72,10 @@ protected AzureADToken refreshToken() throws IOException {
ex = null;
try {
accessToken = adaptee.getAccessToken();
+ if (accessToken == null || accessToken.length() == 0) {
+ LOG.debug("CustomTokenProvider Access token value obtained is empty");
+ throw new IOException("The CustomTokenProvider Access token is empty");
+ }
LOG.trace("CustomTokenProvider Access token fetch was successful with retry count {}",
(fetchTokenRetryCount - retryCount));
} catch (Exception e) {
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobLease.java
new file mode 100644
index 00000000000000..e254ff98f6444c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlobLease.java
@@ -0,0 +1,69 @@
+/**
+ * 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.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_LEASE_ID;
+
+public class AbfsBlobLease extends AbfsLease {
+
+ public AbfsBlobLease(final AbfsClient client,
+ final String path,
+ final Integer leaseDuration,
+ final TracingContext tracingContext) throws AzureBlobFileSystemException {
+ super(client, path, leaseDuration, tracingContext);
+ }
+
+ public AbfsBlobLease(final AbfsClient client,
+ final String path,
+ final int acquireMaxRetries,
+ final int acquireRetryInterval,
+ final Integer leaseDuration,
+ final TracingContext tracingContext) throws AzureBlobFileSystemException {
+ super(client, path, acquireMaxRetries, acquireRetryInterval, leaseDuration,
+ tracingContext);
+ }
+
+ @Override
+ String callRenewLeaseAPI(final String path,
+ final String leaseId,
+ final TracingContext tracingContext) throws AzureBlobFileSystemException {
+ return extractLeaseInfo(client.renewBlobLease(path, leaseId, tracingContext));
+ }
+
+ @Override
+ AbfsRestOperation callAcquireLeaseAPI(final String path,
+ final Integer leaseDuration,
+ final TracingContext tracingContext) throws AzureBlobFileSystemException {
+ return client.acquireBlobLease(path, leaseDuration, tracingContext);
+ }
+
+ @Override
+ void callReleaseLeaseAPI(final String path,
+ final String leaseID,
+ final TracingContext tracingContext) throws AzureBlobFileSystemException {
+ client.releaseBlobLease(path, leaseID, tracingContext);
+ }
+
+ private String extractLeaseInfo(final AbfsRestOperation op) {
+ return op.getResult().getResponseHeader(X_MS_LEASE_ID);
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlock.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlock.java
new file mode 100644
index 00000000000000..025dfd4f76c69e
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsBlock.java
@@ -0,0 +1,129 @@
+/**
+ * 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.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.fs.store.DataBlocks;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.BLOCK_ID_LENGTH;
+
+/**
+ * Return activeBlock with blockId.
+ */
+public class AbfsBlock {
+
+ DataBlocks.DataBlock activeBlock;
+ String blockId;
+ AbfsOutputStream outputStream;
+
+ /**
+ * Gets the activeBlock and the blockId.
+ * @param outputStream AbfsOutputStream Instance.
+ * @param offset Used to generate blockId based on offset.
+ * @throws IOException
+ */
+ AbfsBlock(AbfsOutputStream outputStream, long offset) throws IOException {
+ this.outputStream = outputStream;
+ DataBlocks.BlockFactory blockFactory = outputStream.getBlockFactory();
+ long blockCount = outputStream.getBlockCount();
+ int blockSize = outputStream.getBlockSize();
+ AbfsOutputStreamStatistics outputStreamStatistics = outputStream.getOutputStreamStatistics();
+ this.activeBlock = blockFactory.create(blockCount, blockSize, outputStreamStatistics);
+ this.blockId = generateBlockId(offset);
+ }
+
+ /**
+ * Helper method that generates blockId.
+ * @param position The offset needed to generate blockId.
+ * @return String representing the block ID generated.
+ */
+ private String generateBlockId(long position) {
+ String streamId = this.outputStream.getStreamID();
+ String streamIdHash = Integer.toString(streamId.hashCode());
+ String blockId = String.format("%d_%s", position, streamIdHash);
+ byte[] blockIdByteArray = new byte[BLOCK_ID_LENGTH];
+ System.arraycopy(blockId.getBytes(), 0, blockIdByteArray, 0, Math.min(BLOCK_ID_LENGTH, blockId.length()));
+ return new String(Base64.encodeBase64(blockIdByteArray), StandardCharsets.UTF_8);
+ }
+
+ /**
+ * Returns activeBlock.
+ * @return activeBlock.
+ */
+ public DataBlocks.DataBlock getActiveBlock() {
+ return activeBlock;
+ }
+
+ /**
+ * Returns blockId for the block.
+ * @return blockId.
+ */
+ public String getBlockId() {
+ return blockId;
+ }
+
+ /**
+ * Returns datasize for the block.
+ * @return datasize.
+ */
+ public int dataSize() {
+ return activeBlock.dataSize();
+ }
+
+ /**
+ * Return instance of BlockUploadData.
+ * @return instance of BlockUploadData.
+ * @throws IOException
+ */
+ public DataBlocks.BlockUploadData startUpload() throws IOException {
+ return activeBlock.startUpload();
+ }
+
+ /**
+ * Return the block has data or not.
+ * @return block has data or not.
+ */
+ public boolean hasData() {
+ return activeBlock.hasData();
+ }
+
+ /**
+ * Write a series of bytes from the buffer, from the offset. Returns the number of bytes written.
+ * Only valid in the state Writing. Base class verifies the state but does no writing.
+ * @param buffer buffer.
+ * @param offset offset.
+ * @param length length.
+ * @return number of bytes written.
+ * @throws IOException
+ */
+ public int write(byte[] buffer, int offset, int length) throws IOException {
+ return activeBlock.write(buffer, offset, length);
+ }
+
+ /**
+ * Returns remainingCapacity.
+ * @return remainingCapacity.
+ */
+ public int remainingCapacity() {
+ return activeBlock.remainingCapacity();
+ }
+}
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 69ef0d01c7823f..8bdca8c3b5a155 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
@@ -30,14 +30,18 @@
import java.security.NoSuchAlgorithmException;
import java.util.ArrayList;
import java.util.Base64;
+import java.util.HashMap;
import java.util.List;
import java.util.Locale;
+import java.util.Map;
import java.util.UUID;
import java.util.concurrent.Callable;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
-import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.Path;
+
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.thirdparty.com.google.common.base.Strings;
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback;
@@ -67,18 +71,23 @@
import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import static java.net.HttpURLConnection.HTTP_CONFLICT;
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_DELETE_CONSIDERED_IDEMPOTENT;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.SERVER_SIDE_ENCRYPTION_ALGORITHM;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DNS_PREFIX;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX;
import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.*;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
/**
* AbfsClient.
*/
public class AbfsClient implements Closeable {
public static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class);
+ public static final String HUNDRED_CONTINUE_USER_AGENT = SINGLE_WHITE_SPACE + HUNDRED_CONTINUE + SEMICOLON;
private final URL baseUrl;
private final SharedKeyCredentials sharedKeyCredentials;
@@ -96,6 +105,7 @@ public class AbfsClient implements Closeable {
private AccessTokenProvider tokenProvider;
private SASTokenProvider sasTokenProvider;
private final AbfsCounters abfsCounters;
+ private final AbfsThrottlingIntercept intercept;
private final ListeningScheduledExecutorService executorService;
@@ -111,6 +121,7 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden
this.retryPolicy = abfsClientContext.getExponentialRetryPolicy();
this.accountName = abfsConfiguration.getAccountName().substring(0, abfsConfiguration.getAccountName().indexOf(AbfsHttpConstants.DOT));
this.authType = abfsConfiguration.getAuthType(accountName);
+ this.intercept = AbfsThrottlingInterceptFactory.getInstance(accountName, abfsConfiguration);
String encryptionKey = this.abfsConfiguration
.getClientProvidedEncryptionKey();
@@ -175,6 +186,28 @@ private byte[] getSHA256Hash(String key) throws IOException {
}
}
+ private URL changePrefixFromBlobtoDfs(URL url) throws InvalidUriException {
+ try {
+ url = new URL(url.toString().replace(WASB_DNS_PREFIX, ABFS_DNS_PREFIX));
+ } catch (MalformedURLException ex) {
+ throw new InvalidUriException(url.toString());
+ }
+ return url;
+ }
+
+ private URL changePrefixFromDfsToBlob(URL url) throws InvalidUriException {
+ if (url.toString().contains(WASB_DNS_PREFIX)
+ || getAbfsConfiguration().getPrefixMode() == PrefixMode.DFS) {
+ return url;
+ }
+ try {
+ url = new URL(url.toString().replace(ABFS_DNS_PREFIX, WASB_DNS_PREFIX));
+ } catch (MalformedURLException ex) {
+ throw new InvalidUriException(url.toString());
+ }
+ return url;
+ }
+
private String getBase64EncodedString(String key) {
return getBase64EncodedString(key.getBytes(StandardCharsets.UTF_8));
}
@@ -207,11 +240,16 @@ SharedKeyCredentials getSharedKeyCredentials() {
return sharedKeyCredentials;
}
+ AbfsThrottlingIntercept getIntercept() {
+ return intercept;
+ }
+
List createDefaultHeaders() {
final List requestHeaders = new ArrayList();
requestHeaders.add(new AbfsHttpHeader(X_MS_VERSION, xMsVersion));
requestHeaders.add(new AbfsHttpHeader(ACCEPT, APPLICATION_JSON
- + COMMA + SINGLE_WHITE_SPACE + APPLICATION_OCTET_STREAM));
+ + COMMA + SINGLE_WHITE_SPACE + APPLICATION_OCTET_STREAM
+ + COMMA + SINGLE_WHITE_SPACE + APPLICATION_XML));
requestHeaders.add(new AbfsHttpHeader(ACCEPT_CHARSET,
UTF_8));
requestHeaders.add(new AbfsHttpHeader(CONTENT_TYPE, EMPTY_STRING));
@@ -243,13 +281,37 @@ public AbfsRestOperation createFilesystem(TracingContext tracingContext) throws
final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
- final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.CreateFileSystem,
- this,
- HTTP_METHOD_PUT,
- url,
- requestHeaders);
+ URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.CreateFileSystem, HTTP_METHOD_PUT, url,
+ requestHeaders);
+ op.execute(tracingContext);
+ return op;
+ }
+
+ /**
+ * Caller of
+ * @param tracingContext
+ * @return Creates the Container acting as current filesystem
+ * @throws AzureBlobFileSystemException
+ */
+ public AbfsRestOperation createContainer(TracingContext tracingContext)
+ throws AzureBlobFileSystemException {
+ final List requestHeaders = createDefaultHeaders();
+
+ final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder();
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESTYPE, CONTAINER);
+
+ appendSASTokenToQuery("",
+ SASTokenProvider.CREATE_CONTAINER_OPERATION, abfsUriQueryBuilder);
+ final URL url = createBlobRequestUrl(abfsUriQueryBuilder);
+
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.CreateContainer, HTTP_METHOD_PUT, url,
+ requestHeaders);
op.execute(tracingContext);
return op;
}
@@ -267,13 +329,13 @@ public AbfsRestOperation setFilesystemProperties(final String properties, Tracin
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
- final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.SetFileSystemProperties,
- this,
- HTTP_METHOD_PUT,
- url,
- requestHeaders);
+ URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.SetFileSystemProperties, HTTP_METHOD_PUT, url,
+ requestHeaders);
op.execute(tracingContext);
return op;
}
@@ -292,13 +354,12 @@ public AbfsRestOperation listPath(final String relativePath, final boolean recur
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN, String.valueOf(abfsConfiguration.isUpnUsed()));
appendSASTokenToQuery(relativePath, SASTokenProvider.LIST_OPERATION, abfsUriQueryBuilder);
- final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.ListPaths,
- this,
- HTTP_METHOD_GET,
- url,
- requestHeaders);
+ URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.ListPaths, HTTP_METHOD_GET, url, requestHeaders);
op.execute(tracingContext);
return op;
}
@@ -309,13 +370,13 @@ public AbfsRestOperation getFilesystemProperties(TracingContext tracingContext)
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
- final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.GetFileSystemProperties,
- this,
- HTTP_METHOD_HEAD,
- url,
- requestHeaders);
+ URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.GetFileSystemProperties, HTTP_METHOD_HEAD, url,
+ requestHeaders);
op.execute(tracingContext);
return op;
}
@@ -326,13 +387,37 @@ public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
- final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.DeleteFileSystem,
- this,
- HTTP_METHOD_DELETE,
- url,
- requestHeaders);
+ URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.DeleteFileSystem, HTTP_METHOD_DELETE, url,
+ requestHeaders);
+ op.execute(tracingContext);
+ return op;
+ }
+
+ /**
+ * Caller of
+ * @param tracingContext
+ * @return Deletes the Container acting as current filesystem
+ * @throws AzureBlobFileSystemException
+ */
+ public AbfsRestOperation deleteContainer(TracingContext tracingContext)
+ throws AzureBlobFileSystemException {
+ final List requestHeaders = createDefaultHeaders();
+
+ final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESTYPE, CONTAINER);
+
+ appendSASTokenToQuery("",
+ SASTokenProvider.DELETE_CONTAINER_OPERATION, abfsUriQueryBuilder);
+ final URL url = createBlobRequestUrl(abfsUriQueryBuilder);
+
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.DeleteContainer, HTTP_METHOD_DELETE, url,
+ requestHeaders);
op.execute(tracingContext);
return op;
}
@@ -372,13 +457,12 @@ public AbfsRestOperation createPath(final String path, final boolean isFile, fin
: SASTokenProvider.CREATE_DIRECTORY_OPERATION;
appendSASTokenToQuery(path, operation, abfsUriQueryBuilder);
- final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.CreatePath,
- this,
- HTTP_METHOD_PUT,
- url,
- requestHeaders);
+ URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.CreatePath, HTTP_METHOD_PUT, url, requestHeaders);
try {
op.execute(tracingContext);
} catch (AzureBlobFileSystemException ex) {
@@ -386,7 +470,7 @@ public AbfsRestOperation createPath(final String path, final boolean isFile, fin
if (!op.hasResult()) {
throw ex;
}
- if (!isFile && op.getResult().getStatusCode() == HttpURLConnection.HTTP_CONFLICT) {
+ if (!isFile && op.getResult().getStatusCode() == HTTP_CONFLICT) {
String existingResource =
op.getResult().getResponseHeader(X_MS_EXISTING_RESOURCE_TYPE);
if (existingResource != null && existingResource.equals(DIRECTORY)) {
@@ -398,6 +482,56 @@ public AbfsRestOperation createPath(final String path, final boolean isFile, fin
return op;
}
+ public AbfsRestOperation createPathBlob(final String path, final boolean isFile, final boolean overwrite,
+ final HashMap metadata,
+ final String eTag,
+ TracingContext tracingContext)
+ throws AzureBlobFileSystemException {
+ final List requestHeaders = createDefaultHeaders();
+ if (!overwrite) {
+ requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, AbfsHttpConstants.STAR));
+ }
+ if (eTag != null && !eTag.isEmpty()) {
+ requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_MATCH, eTag));
+ }
+
+ final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+
+ String operation = SASTokenProvider.CREATE_FILE_OPERATION;
+ appendSASTokenToQuery(path, operation, abfsUriQueryBuilder);
+
+ final URL url = createBlobRequestUrl(path, abfsUriQueryBuilder);
+ if (metadata != null && !metadata.isEmpty()) {
+ for (Map.Entry entry : metadata.entrySet()) {
+ requestHeaders.add(new AbfsHttpHeader(entry.getKey(), entry.getValue()));
+ }
+ }
+ requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, ZERO));
+ requestHeaders.add(new AbfsHttpHeader(X_MS_BLOB_TYPE, BLOCK_BLOB_TYPE));
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.PutBlob, HTTP_METHOD_PUT, url, requestHeaders);
+ try {
+ op.execute(tracingContext);
+ } catch (AzureBlobFileSystemException ex) {
+ // If we have no HTTP response, throw the original exception.
+ if (!op.hasResult()) {
+ throw ex;
+ }
+ if (!isFile && op.getResult().getStatusCode() == HTTP_CONFLICT) {
+ // This ensures that we don't throw ex only for existing directory but if a blob exists we throw exception.
+ tracingContext.setFallbackDFSAppend(tracingContext.getFallbackDFSAppend() + "M");
+ AbfsRestOperation blobProperty = getBlobProperty(new Path(path), tracingContext);
+ final AbfsHttpOperation opResult = blobProperty.getResult();
+ boolean isDirectory = (opResult.getResponseHeader(X_MS_META_HDI_ISFOLDER) != null);
+ if (isDirectory) {
+ return op;
+ }
+ }
+ throw ex;
+ }
+ return op;
+ }
+
public AbfsRestOperation acquireLease(final String path, int duration, TracingContext tracingContext) throws AzureBlobFileSystemException {
final List requestHeaders = createDefaultHeaders();
@@ -407,13 +541,30 @@ public AbfsRestOperation acquireLease(final String path, int duration, TracingCo
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
- final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.LeasePath,
- this,
- HTTP_METHOD_POST,
- url,
- requestHeaders);
+ URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.LeasePath, HTTP_METHOD_POST, url, requestHeaders);
+ op.execute(tracingContext);
+ return op;
+ }
+
+ public AbfsRestOperation acquireBlobLease(final String path, final int duration, final 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()));
+ final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, QUERY_PARAM_COMP_LEASE_VALUE);
+ appendSASTokenToQuery(path, SASTokenProvider.LEASE_OPERATION, abfsUriQueryBuilder);
+
+
+ final URL url = createBlobRequestUrl(path, abfsUriQueryBuilder);
+
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.LeaseBlob, HTTP_METHOD_PUT, url, requestHeaders);
op.execute(tracingContext);
return op;
}
@@ -427,13 +578,31 @@ public AbfsRestOperation renewLease(final String path, final String leaseId,
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
- final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.LeasePath,
- this,
- HTTP_METHOD_POST,
- url,
- requestHeaders);
+ URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.LeasePath, HTTP_METHOD_POST, url, requestHeaders);
+ op.execute(tracingContext);
+ return op;
+ }
+
+ public AbfsRestOperation renewBlobLease(final String path, final String leaseId,
+ TracingContext tracingContext) throws AzureBlobFileSystemException {
+ final List requestHeaders = createDefaultHeaders();
+
+ requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, RENEW_LEASE_ACTION));
+ requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId));
+
+ final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, QUERY_PARAM_COMP_LEASE_VALUE);
+ appendSASTokenToQuery(path, SASTokenProvider.LEASE_OPERATION, abfsUriQueryBuilder);
+
+ final URL url = createBlobRequestUrl(path, abfsUriQueryBuilder);
+
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.LeaseBlob, HTTP_METHOD_PUT, url, requestHeaders);
op.execute(tracingContext);
return op;
}
@@ -447,13 +616,31 @@ public AbfsRestOperation releaseLease(final String path,
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
- final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.LeasePath,
- this,
- HTTP_METHOD_POST,
- url,
- requestHeaders);
+ URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.LeasePath, HTTP_METHOD_POST, url, requestHeaders);
+ op.execute(tracingContext);
+ return op;
+ }
+
+ public AbfsRestOperation releaseBlobLease(final String path,
+ final String leaseId, TracingContext tracingContext) throws AzureBlobFileSystemException {
+ final List requestHeaders = createDefaultHeaders();
+
+ requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ACTION, RELEASE_LEASE_ACTION));
+ requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId));
+
+ final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, QUERY_PARAM_COMP_LEASE_VALUE);
+ appendSASTokenToQuery(path, SASTokenProvider.LEASE_OPERATION, abfsUriQueryBuilder);
+
+ final URL url = createBlobRequestUrl(path, abfsUriQueryBuilder);
+
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.LeasePath, HTTP_METHOD_PUT, url, requestHeaders);
op.execute(tracingContext);
return op;
}
@@ -467,13 +654,12 @@ public AbfsRestOperation breakLease(final String path,
final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
- final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.LeasePath,
- this,
- HTTP_METHOD_POST,
- url,
- requestHeaders);
+ URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.LeasePath, HTTP_METHOD_POST, url, requestHeaders);
op.execute(tracingContext);
return op;
}
@@ -498,13 +684,12 @@ public AbfsRestOperation renamePath(String source, final String destination,
abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation);
appendSASTokenToQuery(destination, SASTokenProvider.RENAME_DESTINATION_OPERATION, abfsUriQueryBuilder);
- final URL url = createRequestUrl(destination, abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.RenamePath,
- this,
- HTTP_METHOD_PUT,
- url,
- requestHeaders);
+ URL url = createRequestUrl(destination, abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.RenamePath, HTTP_METHOD_PUT, url, requestHeaders);
// no attempt at recovery using timestamps as it was not reliable.
op.execute(tracingContext);
return op;
@@ -515,6 +700,9 @@ public AbfsRestOperation append(final String path, final byte[] buffer,
throws AzureBlobFileSystemException {
final List requestHeaders = createDefaultHeaders();
addCustomerProvidedKeyHeaders(requestHeaders);
+ if (reqParams.isExpectHeaderEnabled()) {
+ requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE));
+ }
// JDK7 does not support PATCH, so to workaround the issue we will use
// PUT and specify the real method in the X-Http-Method-Override header.
requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
@@ -534,25 +722,49 @@ public AbfsRestOperation append(final String path, final byte[] buffer,
abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, TRUE);
}
}
+ if (reqParams.isRetryDueToExpect()) {
+ String userAgentRetry = userAgent;
+ userAgentRetry = userAgentRetry.replace(HUNDRED_CONTINUE_USER_AGENT, EMPTY_STRING);
+ requestHeaders.removeIf(header -> header.getName().equalsIgnoreCase(USER_AGENT));
+ requestHeaders.add(new AbfsHttpHeader(USER_AGENT, userAgentRetry));
+ }
// AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance
String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.WRITE_OPERATION,
abfsUriQueryBuilder, cachedSasToken);
- final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.Append,
- this,
- HTTP_METHOD_PUT,
- url,
- requestHeaders,
- buffer,
- reqParams.getoffset(),
- reqParams.getLength(),
- sasTokenForReuse);
+ URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperationForAppend(AbfsRestOperationType.Append,
+ HTTP_METHOD_PUT,
+ url,
+ requestHeaders,
+ buffer,
+ reqParams.getoffset(),
+ reqParams.getLength(),
+ sasTokenForReuse);
try {
op.execute(tracingContext);
} catch (AzureBlobFileSystemException e) {
+ /*
+ If the http response code indicates a user error we retry
+ the same append request with expect header being disabled.
+ When "100-continue" header is enabled but a non Http 100 response comes,
+ the response message might not get set correctly by the server.
+ So, this handling is to avoid breaking of backward compatibility
+ if someone has taken dependency on the exception message,
+ which is created using the error string present in the response header.
+ */
+ int responseStatusCode = ((AbfsRestOperationException) e).getStatusCode();
+ if (checkUserError(responseStatusCode) && reqParams.isExpectHeaderEnabled()) {
+ LOG.debug("User error, retrying without 100 continue enabled for the given path {}", path);
+ reqParams.setExpectHeaderEnabled(false);
+ reqParams.setRetryDueToExpect(true);
+ return this.append(path, buffer, reqParams, cachedSasToken,
+ tracingContext);
+ }
// If we have no HTTP response, throw the original exception.
if (!op.hasResult()) {
throw e;
@@ -560,16 +772,15 @@ public AbfsRestOperation append(final String path, final byte[] buffer,
if (reqParams.isAppendBlob()
&& appendSuccessCheckOp(op, path,
(reqParams.getPosition() + reqParams.getLength()), tracingContext)) {
- final AbfsRestOperation successOp = new AbfsRestOperation(
- AbfsRestOperationType.Append,
- this,
- HTTP_METHOD_PUT,
- url,
- requestHeaders,
- buffer,
- reqParams.getoffset(),
- reqParams.getLength(),
- sasTokenForReuse);
+ final AbfsRestOperation successOp = getAbfsRestOperationForAppend(
+ AbfsRestOperationType.Append,
+ HTTP_METHOD_PUT,
+ url,
+ requestHeaders,
+ buffer,
+ reqParams.getoffset(),
+ reqParams.getLength(),
+ sasTokenForReuse);
successOp.hardSetResult(HttpURLConnection.HTTP_OK);
return successOp;
}
@@ -579,6 +790,184 @@ && appendSuccessCheckOp(op, path,
return op;
}
+ /**
+ * Append operation for blob endpoint which takes block id as a param.
+ * @param blockId The blockId of the block to be appended.
+ * @param path The path at which the block is to be appended.
+ * @param buffer The buffer which has the data to be appended.
+ * @param reqParams The request params.
+ * @param cachedSasToken The cachedSasToken if available.
+ * @param tracingContext Tracing context of the operation.
+ * @param eTag Etag of the blob to prevent parallel writer situations.
+ * @return AbfsRestOperation op.
+ * @throws AzureBlobFileSystemException
+ */
+ public AbfsRestOperation append(final String blockId, final String path, final byte[] buffer,
+ AppendRequestParameters reqParams, final String cachedSasToken,
+ TracingContext tracingContext, String eTag)
+ throws AzureBlobFileSystemException {
+ final List requestHeaders = createDefaultHeaders();
+ if (reqParams.getLeaseId() != null) {
+ requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, reqParams.getLeaseId()));
+ }
+ if (reqParams.isExpectHeaderEnabled()) {
+ requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE));
+ }
+ final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCK);
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_BLOCKID, blockId);
+ requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, String.valueOf(buffer.length)));
+ requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
+
+ if (reqParams.isRetryDueToExpect()) {
+ String userAgentRetry = userAgent;
+ userAgentRetry = userAgentRetry.replace(HUNDRED_CONTINUE_USER_AGENT, EMPTY_STRING);
+ requestHeaders.removeIf(header -> header.getName().equalsIgnoreCase(USER_AGENT));
+ requestHeaders.add(new AbfsHttpHeader(USER_AGENT, userAgentRetry));
+ }
+
+ // AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance
+ String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.WRITE_OPERATION,
+ abfsUriQueryBuilder, cachedSasToken);
+
+ final URL url = createBlobRequestUrl(path, abfsUriQueryBuilder);
+ final AbfsRestOperation op = getPutBlockOperation(buffer, reqParams, requestHeaders,
+ sasTokenForReuse, url);
+ try {
+ op.execute(tracingContext);
+ } catch (AzureBlobFileSystemException e) {
+ /*
+ If the http response code indicates a user error we retry
+ the same append request with expect header being disabled.
+ When "100-continue" header is enabled but a non Http 100 response comes,
+ the response message might not get set correctly by the server.
+ So, this handling is to avoid breaking of backward compatibility
+ if someone has taken dependency on the exception message,
+ which is created using the error string present in the response header.
+ */
+ int responseStatusCode = ((AbfsRestOperationException) e).getStatusCode();
+ if (checkUserErrorBlob(responseStatusCode) && reqParams.isExpectHeaderEnabled()) {
+ LOG.debug("User error, retrying without 100 continue enabled for the given path {}", path);
+ reqParams.setExpectHeaderEnabled(false);
+ reqParams.setRetryDueToExpect(true);
+ return this.append(blockId, path, buffer, reqParams, cachedSasToken,
+ tracingContext, eTag);
+ }
+ else {
+ throw e;
+ }
+ }
+ return op;
+ }
+
+ @VisibleForTesting
+ AbfsRestOperation getPutBlockOperation(final byte[] buffer,
+ final AppendRequestParameters reqParams,
+ final List requestHeaders,
+ final String sasTokenForReuse,
+ final URL url) {
+ return getAbfsRestOperation(AbfsRestOperationType.PutBlock, HTTP_METHOD_PUT,
+ url,
+ requestHeaders, buffer, reqParams.getoffset(), reqParams.getLength(),
+ sasTokenForReuse);
+ }
+
+ /**
+ * The flush operation to commit the blocks.
+ * @param buffer This has the xml in byte format with the blockIds to be flushed.
+ * @param path The path to flush the data to.
+ * @param isClose True when the stream is closed.
+ * @param cachedSasToken The cachedSasToken if available.
+ * @param leaseId The leaseId of the blob if available.
+ * @param eTag The etag of the blob.
+ * @param tracingContext Tracing context for the operation.
+ * @return AbfsRestOperation op.
+ * @throws IOException
+ */
+ public AbfsRestOperation flush(byte[] buffer, final String path, boolean isClose,
+ final String cachedSasToken, final String leaseId, String eTag,
+ TracingContext tracingContext) throws IOException {
+ final List requestHeaders = createDefaultHeaders();
+ if (leaseId != null) {
+ requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId));
+ }
+
+ final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST);
+ requestHeaders.add(new AbfsHttpHeader(CONTENT_LENGTH, String.valueOf(buffer.length)));
+ requestHeaders.add(new AbfsHttpHeader(CONTENT_TYPE, APPLICATION_XML));
+ requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_CLOSE, String.valueOf(isClose));
+ // AbfsInputStream/AbfsOutputStream reuse SAS tokens for better performance
+ String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.WRITE_OPERATION,
+ abfsUriQueryBuilder, cachedSasToken);
+
+ final URL url = createBlobRequestUrl(path, abfsUriQueryBuilder);
+ final AbfsRestOperation op = getPutBlockListOperation(buffer,
+ requestHeaders, sasTokenForReuse,
+ url);
+ op.execute(tracingContext);
+ return op;
+ }
+
+ @VisibleForTesting
+ AbfsRestOperation getPutBlockListOperation(final byte[] buffer,
+ final List requestHeaders,
+ final String sasTokenForReuse,
+ final URL url) {
+ return getAbfsRestOperation(AbfsRestOperationType.PutBlockList,
+ HTTP_METHOD_PUT, url, requestHeaders, buffer, 0, buffer.length,
+ sasTokenForReuse);
+ }
+
+ /*
+ * Returns the rest operation for append.
+ * @param operationType The AbfsRestOperationType.
+ * @param httpMethod specifies the httpMethod.
+ * @param url specifies the url.
+ * @param requestHeaders This includes the list of request headers.
+ * @param buffer The buffer to write into.
+ * @param bufferOffset The buffer offset.
+ * @param bufferLength The buffer Length.
+ * @param sasTokenForReuse The sasToken.
+ * @return AbfsRestOperation op.
+ */
+ @VisibleForTesting
+ AbfsRestOperation getAbfsRestOperationForAppend(final AbfsRestOperationType operationType,
+ final String httpMethod,
+ final URL url,
+ final List requestHeaders,
+ final byte[] buffer,
+ final int bufferOffset,
+ final int bufferLength,
+ final String sasTokenForReuse) {
+ return getAbfsRestOperation(operationType, httpMethod, url, requestHeaders,
+ buffer, bufferOffset, bufferLength, sasTokenForReuse);
+ }
+
+ /**
+ * Returns true if the status code lies in the range of user error.
+ * @param responseStatusCode http response status code.
+ * @return True or False.
+ */
+ private boolean checkUserError(int responseStatusCode) {
+ return (responseStatusCode >= HttpURLConnection.HTTP_BAD_REQUEST
+ && responseStatusCode < HttpURLConnection.HTTP_INTERNAL_ERROR);
+ }
+
+ /**
+ * Returns true if the status code lies in the range of user error.
+ * In the case of HTTP_CONFLICT for PutBlockList we fallback to DFS and hence
+ * this retry handling is not needed.
+ * @param responseStatusCode http response status code.
+ * @return True or False.
+ */
+ private boolean checkUserErrorBlob(int responseStatusCode) {
+ return (responseStatusCode >= HttpURLConnection.HTTP_BAD_REQUEST
+ && responseStatusCode < HttpURLConnection.HTTP_INTERNAL_ERROR
+ && responseStatusCode != HttpURLConnection.HTTP_CONFLICT);
+ }
+
// For AppendBlob its possible that the append succeeded in the backend but the request failed.
// However a retry would fail with an InvalidQueryParameterValue
// (as the current offset would be unacceptable).
@@ -624,13 +1013,13 @@ public AbfsRestOperation flush(final String path, final long position,
String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.WRITE_OPERATION,
abfsUriQueryBuilder, cachedSasToken);
- final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.Flush,
- this,
- HTTP_METHOD_PUT,
- url,
- requestHeaders, sasTokenForReuse);
+ URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.Flush, HTTP_METHOD_PUT, url, requestHeaders,
+ sasTokenForReuse);
op.execute(tracingContext);
return op;
}
@@ -651,13 +1040,13 @@ public AbfsRestOperation setPathProperties(final String path, final String prope
abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, SET_PROPERTIES_ACTION);
appendSASTokenToQuery(path, SASTokenProvider.SET_PROPERTIES_OPERATION, abfsUriQueryBuilder);
- final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.SetPathProperties,
- this,
- HTTP_METHOD_PUT,
- url,
- requestHeaders);
+ URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.SetPathProperties, HTTP_METHOD_PUT, url,
+ requestHeaders);
op.execute(tracingContext);
return op;
}
@@ -680,13 +1069,38 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN, String.valueOf(abfsConfiguration.isUpnUsed()));
appendSASTokenToQuery(path, operation, abfsUriQueryBuilder);
- final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.GetPathStatus,
- this,
- HTTP_METHOD_HEAD,
- url,
- requestHeaders);
+ URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.GetPathStatus, HTTP_METHOD_HEAD, url,
+ requestHeaders);
+ op.execute(tracingContext);
+ return op;
+ }
+
+ /**
+ * GetBlockList call to the backend to get the list of committed blockId's.
+ * @param path The path to get the list of blockId's.
+ * @param tracingContext The tracing context for the operation.
+ * @return AbfsRestOperation op.
+ * @throws AzureBlobFileSystemException
+ */
+ public AbfsRestOperation getBlockList(final String path, TracingContext tracingContext) throws AzureBlobFileSystemException {
+ final List requestHeaders = createDefaultHeaders();
+
+ final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+ String operation = SASTokenProvider.GET_BLOCK_LIST;
+ appendSASTokenToQuery(path, operation, abfsUriQueryBuilder);
+
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, BLOCKLIST);
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_BLOCKLISTTYPE, COMMITTED);
+ final URL url = createBlobRequestUrl(path, abfsUriQueryBuilder);
+
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.GetBlockList, HTTP_METHOD_GET, url,
+ requestHeaders);
op.execute(tracingContext);
return op;
}
@@ -705,16 +1119,22 @@ public AbfsRestOperation read(final String path, final long position, final byte
String sasTokenForReuse = appendSASTokenToQuery(path, SASTokenProvider.READ_OPERATION,
abfsUriQueryBuilder, cachedSasToken);
- final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.ReadFile,
- this,
- HTTP_METHOD_GET,
- url,
- requestHeaders,
- buffer,
- bufferOffset,
- bufferLength, sasTokenForReuse);
+ URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+ final AbfsRestOperationType opType;
+ if (!OperativeEndpoint.isReadEnabledOnDFS(getAbfsConfiguration())) {
+ LOG.debug("Read over Blob for read config value {} for path {} ",
+ abfsConfiguration.shouldReadFallbackToDfs(), path);
+ opType = AbfsRestOperationType.GetBlob;
+ url = changePrefixFromDfsToBlob(url);
+ } else {
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ opType = AbfsRestOperationType.ReadFile;
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(opType, HTTP_METHOD_GET,
+ url, requestHeaders, buffer, bufferOffset, bufferLength,
+ sasTokenForReuse);
op.execute(tracingContext);
return op;
@@ -731,13 +1151,13 @@ public AbfsRestOperation deletePath(final String path, final boolean recursive,
String operation = recursive ? SASTokenProvider.DELETE_RECURSIVE_OPERATION : SASTokenProvider.DELETE_OPERATION;
appendSASTokenToQuery(path, operation, abfsUriQueryBuilder);
- final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.DeletePath,
- this,
- HTTP_METHOD_DELETE,
- url,
- requestHeaders);
+ URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.DeletePath, HTTP_METHOD_DELETE, url,
+ requestHeaders);
try {
op.execute(tracingContext);
} catch (AzureBlobFileSystemException e) {
@@ -781,11 +1201,8 @@ public AbfsRestOperation deleteIdempotencyCheckOp(final AbfsRestOperation op) {
&& DEFAULT_DELETE_CONSIDERED_IDEMPOTENT) {
// Server has returned HTTP 404, which means path no longer
// exists. Assuming delete result to be idempotent, return success.
- final AbfsRestOperation successOp = new AbfsRestOperation(
- AbfsRestOperationType.DeletePath,
- this,
- HTTP_METHOD_DELETE,
- op.getUrl(),
+ final AbfsRestOperation successOp = getAbfsRestOperation(
+ AbfsRestOperationType.DeletePath, HTTP_METHOD_DELETE, op.getUrl(),
op.getRequestHeaders());
successOp.hardSetResult(HttpURLConnection.HTTP_OK);
LOG.debug("Returning success response from delete idempotency logic");
@@ -815,12 +1232,12 @@ public AbfsRestOperation setOwner(final String path, final String owner, final S
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_ACCESS_CONTROL);
appendSASTokenToQuery(path, SASTokenProvider.SET_OWNER_OPERATION, abfsUriQueryBuilder);
- final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.SetOwner,
- this,
- AbfsHttpConstants.HTTP_METHOD_PUT,
- url,
+ URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.SetOwner, AbfsHttpConstants.HTTP_METHOD_PUT, url,
requestHeaders);
op.execute(tracingContext);
return op;
@@ -841,13 +1258,13 @@ public AbfsRestOperation setPermission(final String path, final String permissio
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_ACCESS_CONTROL);
appendSASTokenToQuery(path, SASTokenProvider.SET_PERMISSION_OPERATION, abfsUriQueryBuilder);
- final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.SetPermissions,
- this,
- AbfsHttpConstants.HTTP_METHOD_PUT,
- url,
- requestHeaders);
+ URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.SetPermissions, AbfsHttpConstants.HTTP_METHOD_PUT,
+ url, requestHeaders);
op.execute(tracingContext);
return op;
}
@@ -876,12 +1293,12 @@ public AbfsRestOperation setAcl(final String path, final String aclSpecString, f
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_ACCESS_CONTROL);
appendSASTokenToQuery(path, SASTokenProvider.SET_ACL_OPERATION, abfsUriQueryBuilder);
- final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.SetAcl,
- this,
- AbfsHttpConstants.HTTP_METHOD_PUT,
- url,
+ URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.SetAcl, AbfsHttpConstants.HTTP_METHOD_PUT, url,
requestHeaders);
op.execute(tracingContext);
return op;
@@ -901,12 +1318,12 @@ public AbfsRestOperation getAclStatus(final String path, final boolean useUPN,
abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN, String.valueOf(useUPN));
appendSASTokenToQuery(path, SASTokenProvider.GET_ACL_OPERATION, abfsUriQueryBuilder);
- final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
- final AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.GetAcl,
- this,
- AbfsHttpConstants.HTTP_METHOD_HEAD,
- url,
+ URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.GetAcl, AbfsHttpConstants.HTTP_METHOD_HEAD, url,
requestHeaders);
op.execute(tracingContext);
return op;
@@ -928,14 +1345,365 @@ public AbfsRestOperation checkAccess(String path, String rwx, TracingContext tra
abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, CHECK_ACCESS);
abfsUriQueryBuilder.addQuery(QUERY_FS_ACTION, rwx);
appendSASTokenToQuery(path, SASTokenProvider.CHECK_ACCESS_OPERATION, abfsUriQueryBuilder);
+
URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
- AbfsRestOperation op = new AbfsRestOperation(
- AbfsRestOperationType.CheckAccess, this,
- AbfsHttpConstants.HTTP_METHOD_HEAD, url, createDefaultHeaders());
+ if (url.toString().contains(WASB_DNS_PREFIX)) {
+ url = changePrefixFromBlobtoDfs(url);
+ }
+
+ AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.CheckAccess, AbfsHttpConstants.HTTP_METHOD_HEAD,
+ url, createDefaultHeaders());
op.execute(tracingContext);
return op;
}
+ /**
+ * Caller of
+ * copyBlob API. This is an asynchronous API, it returns copyId and expects client
+ * to poll the server on the destination and check the copy-progress.
+ *
+ * @param sourceBlobPath path of source to be copied
+ * @param destinationBlobPath path of the destination
+ * @param srcLeaseId
+ * @param tracingContext tracingContext object
+ *
+ * @return AbfsRestOperation abfsRestOperation which contains the response from the server.
+ * This method owns the logic of triggereing copyBlob API. The caller of this method have
+ * to own the logic of polling the destination with the copyId returned in the response from
+ * this method.
+ *
+ * @throws AzureBlobFileSystemException exception recevied while making server call.
+ */
+ public AbfsRestOperation copyBlob(Path sourceBlobPath,
+ Path destinationBlobPath,
+ final String srcLeaseId, TracingContext tracingContext) throws AzureBlobFileSystemException {
+ AbfsUriQueryBuilder abfsUriQueryBuilderDst = createDefaultUriQueryBuilder();
+ AbfsUriQueryBuilder abfsUriQueryBuilderSrc = new AbfsUriQueryBuilder();
+ String dstBlobRelativePath = destinationBlobPath.toUri().getPath();
+ String srcBlobRelativePath = sourceBlobPath.toUri().getPath();
+ appendSASTokenToQuery(dstBlobRelativePath,
+ SASTokenProvider.COPY_BLOB_DESTINATION, abfsUriQueryBuilderDst);
+ appendSASTokenToQuery(srcBlobRelativePath,
+ SASTokenProvider.COPY_BLOB_SOURCE, abfsUriQueryBuilderSrc);
+ final URL url = createBlobRequestUrl(dstBlobRelativePath,
+ abfsUriQueryBuilderDst);
+ final String sourcePathUrl = createBlobRequestUrl(srcBlobRelativePath,
+ abfsUriQueryBuilderSrc).toString();
+ List requestHeaders = createDefaultHeaders();
+ if (srcLeaseId != null) {
+ requestHeaders.add(new AbfsHttpHeader(X_MS_SOURCE_LEASE_ID, srcLeaseId));
+ }
+ requestHeaders.add(new AbfsHttpHeader(X_MS_COPY_SOURCE, sourcePathUrl));
+ requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, STAR));
+
+ final AbfsRestOperation op = getCopyBlobOperation(url, requestHeaders);
+ op.execute(tracingContext);
+
+ return op;
+ }
+
+ @VisibleForTesting
+ AbfsRestOperation getCopyBlobOperation(final URL url,
+ final List requestHeaders) {
+ return getAbfsRestOperation(AbfsRestOperationType.CopyBlob, HTTP_METHOD_PUT,
+ url, requestHeaders);
+ }
+
+ @VisibleForTesting
+ AbfsRestOperation getListBlobOperation(final URL url,
+ final List requestHeaders) {
+ return getAbfsRestOperation(AbfsRestOperationType.GetListBlobProperties, HTTP_METHOD_GET,
+ url, requestHeaders);
+ }
+
+ /**
+ * @return the blob properties returned from server.
+ * @throws AzureBlobFileSystemException in case it is not a 404 error or some other exception
+ * which was not able to be retried.
+ * */
+ public AbfsRestOperation getBlobProperty(Path blobPath,
+ TracingContext tracingContext) throws AzureBlobFileSystemException {
+ AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+ String blobRelativePath = blobPath.toUri().getPath();
+ appendSASTokenToQuery(blobRelativePath,
+ SASTokenProvider.GET_BLOB_PROPERTIES_OPERATION, abfsUriQueryBuilder);
+ final URL url = createBlobRequestUrl(blobRelativePath, abfsUriQueryBuilder);
+ final List requestHeaders = createDefaultHeaders();
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.GetBlobProperties, HTTP_METHOD_HEAD, url,
+ requestHeaders);
+ op.execute(tracingContext);
+ return op;
+ }
+
+ /**
+ * Caller of
+ * @return the container properties returned from server.
+ * @throws AzureBlobFileSystemException in case it is not a 404 error or some other exception
+ * which was not able to be retried.
+ * */
+ public AbfsRestOperation getContainerProperty(TracingContext tracingContext) throws AzureBlobFileSystemException {
+ final List requestHeaders = createDefaultHeaders();
+
+ final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESTYPE, CONTAINER);
+
+ appendSASTokenToQuery("",
+ SASTokenProvider.GET_CONTAINER_PROPERTIES_OPERATION, abfsUriQueryBuilder);
+
+ final URL url = createBlobRequestUrl(abfsUriQueryBuilder);
+
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.GetContainerProperties, HTTP_METHOD_HEAD, url,
+ requestHeaders);
+ op.execute(tracingContext);
+ return op;
+ }
+
+ /**
+ * Gets user-defined properties(metadata) of the blob over blob endpoint.
+ * @param blobPath
+ * @param tracingContext
+ * @return the user-defined properties on blob path
+ * @throws AzureBlobFileSystemException
+ */
+ public AbfsRestOperation getBlobMetadata(Path blobPath,
+ TracingContext tracingContext) throws AzureBlobFileSystemException {
+ final List requestHeaders = createDefaultHeaders();
+ AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, QUERY_PARAM_INCLUDE_VALUE_METADATA);
+
+ String blobRelativePath = blobPath.toUri().getPath();
+ appendSASTokenToQuery(blobRelativePath,
+ SASTokenProvider.GET_BLOB_METADATA_OPERATION, abfsUriQueryBuilder);
+
+ final URL url = createBlobRequestUrl(blobRelativePath, abfsUriQueryBuilder);
+
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.GetBlobMetadata, HTTP_METHOD_HEAD, url,
+ requestHeaders);
+ op.execute(tracingContext);
+ return op;
+ }
+
+ /**
+ * Sets user-defined properties(metadata) of the blob over blob endpoint.
+ * @param blobPath
+ * @param metadataRequestHeaders
+ * @param tracingContext
+ * @throws AzureBlobFileSystemException
+ */
+ public AbfsRestOperation setBlobMetadata(Path blobPath, List metadataRequestHeaders,
+ TracingContext tracingContext) throws AzureBlobFileSystemException {
+ // Request Header for this call will also contain metadata headers
+ final List defaultRequestHeaders = createDefaultHeaders();
+ final List requestHeaders = new ArrayList();
+ requestHeaders.addAll(defaultRequestHeaders);
+ requestHeaders.addAll(metadataRequestHeaders);
+
+ AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, QUERY_PARAM_INCLUDE_VALUE_METADATA);
+
+ String blobRelativePath = blobPath.toUri().getPath();
+ appendSASTokenToQuery(blobRelativePath,
+ SASTokenProvider.SET_BLOB_METADATA_OPERATION, abfsUriQueryBuilder);
+
+ final URL url = createBlobRequestUrl(blobRelativePath, abfsUriQueryBuilder);
+
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.SetBlobMetadata, HTTP_METHOD_PUT, url,
+ requestHeaders);
+ op.execute(tracingContext);
+ return op;
+ }
+
+ /**
+ * Caller of
+ * Gets user-defined properties(metadata) of the container(filesystem) over blob endpoint.
+ * @param tracingContext
+ * @return the user-defined properties on container path
+ * @throws AzureBlobFileSystemException
+ */
+ public AbfsRestOperation getContainerMetadata(TracingContext tracingContext)
+ throws AzureBlobFileSystemException {
+ final List requestHeaders = createDefaultHeaders();
+
+ AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESTYPE, CONTAINER);
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, QUERY_PARAM_INCLUDE_VALUE_METADATA);
+
+ appendSASTokenToQuery("",
+ SASTokenProvider.GET_CONTAINER_METADATA_OPERATION, abfsUriQueryBuilder);
+
+ final URL url = createBlobRequestUrl(abfsUriQueryBuilder);
+
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.GetContainerMetadata, HTTP_METHOD_HEAD, url,
+ requestHeaders);
+ op.execute(tracingContext);
+ return op;
+ }
+
+ /**
+ * Caller of
+ * Sets user-defined properties(metadata) of the container(filesystem) over blob endpoint.
+ * @param metadataRequestHeaders
+ * @param tracingContext
+ * @throws AzureBlobFileSystemException
+ */
+ public AbfsRestOperation setContainerMetadata(List metadataRequestHeaders,
+ TracingContext tracingContext) throws AzureBlobFileSystemException {
+ // Request Header for this call will also contain metadata headers
+ final List defaultRequestHeaders = createDefaultHeaders();
+ final List requestHeaders = new ArrayList();
+ requestHeaders.addAll(defaultRequestHeaders);
+ requestHeaders.addAll(metadataRequestHeaders);
+
+ AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESTYPE, CONTAINER);
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, QUERY_PARAM_INCLUDE_VALUE_METADATA);
+
+ appendSASTokenToQuery("",
+ SASTokenProvider.SET_CONTAINER_METADATA_OPERATION, abfsUriQueryBuilder);
+
+ final URL url = createBlobRequestUrl(abfsUriQueryBuilder);
+
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.SetContainerMetadata, HTTP_METHOD_PUT, url,
+ requestHeaders);
+ op.execute(tracingContext);
+ return op;
+ }
+
+ /**
+ * Call server API BlobList.
+ *
+ * @param marker optional value. To be sent in case this method call in a non-first
+ * iteration to the blobList API. Value has to be equal to the field NextMarker in the response
+ * of previous iteration for the same operation.
+ * @param prefix optional value. Filters the results to return only blobs
+ * with names that begin with the specified prefix
+ * @param delimiter Optional. When the request includes this parameter,
+ * the operation returns a BlobPrefix element in the response body.
+ * This element acts as a placeholder for all blobs with names that begin
+ * with the same substring, up to the appearance of the delimiter character.
+ * @param maxResult define how many blobs can client handle in server response.
+ * In case maxResult <= 5000, server sends number of blobs equal to the value. In
+ * case maxResult > 5000, server sends maximum 5000 blobs.
+ * @param tracingContext object of {@link TracingContext}
+ *
+ * @return abfsRestOperation which contain list of {@link BlobProperty}
+ * via {@link AbfsRestOperation#getResult()}.{@link AbfsHttpOperation#getBlobList()}
+ *
+ * @throws AzureBlobFileSystemException thrown from server-call / xml-parsing
+ */
+ public AbfsRestOperation getListBlobs(String marker,
+ String prefix,
+ String delimiter,
+ Integer maxResult,
+ TracingContext tracingContext)
+ throws AzureBlobFileSystemException {
+ AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESTYPE, CONTAINER);
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_COMP, QUERY_PARAM_COMP_VALUE_LIST);
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_DELIMITER, delimiter);
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_INCLUDE,
+ QUERY_PARAM_INCLUDE_VALUE_METADATA);
+ prefix = getDirectoryQueryParameter(prefix);
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_PREFIX, prefix);
+ if (marker != null) {
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_MARKER, marker);
+ }
+ if (maxResult != null) {
+ abfsUriQueryBuilder.addQuery(QUERY_PARAM_MAXRESULT, maxResult.toString());
+ }
+ appendSASTokenToQuery(null, SASTokenProvider.LIST_BLOB_OPERATION,
+ abfsUriQueryBuilder);
+ final URL url = createBlobRequestUrl(abfsUriQueryBuilder);
+ final List requestHeaders = createDefaultHeaders();
+ final AbfsRestOperation op = getListBlobOperation(url, requestHeaders);
+
+ op.execute(tracingContext);
+ return op;
+ }
+
+ /**
+ * Deletes the blob for which the path is given.
+ *
+ * @param blobPath path on which blob has to be deleted.
+ * @param leaseId
+ * @param tracingContext tracingContext object for tracing the server calls.
+ *
+ * @return abfsRestOpertion
+ *
+ * @throws AzureBlobFileSystemException exception thrown from server or due to
+ * network issue.
+ */
+ public AbfsRestOperation deleteBlobPath(final Path blobPath,
+ final String leaseId, final TracingContext tracingContext) throws AzureBlobFileSystemException {
+ AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+ String blobRelativePath = blobPath.toUri().getPath();
+ appendSASTokenToQuery(blobRelativePath,
+ SASTokenProvider.DELETE_BLOB_OPERATION, abfsUriQueryBuilder);
+ final URL url = createBlobRequestUrl(blobRelativePath, abfsUriQueryBuilder);
+ final List requestHeaders = createDefaultHeaders();
+ if(leaseId != null) {
+ requestHeaders.add(new AbfsHttpHeader(X_MS_LEASE_ID, leaseId));
+ }
+ final AbfsRestOperation op = getAbfsRestOperation(
+ AbfsRestOperationType.DeleteBlob, HTTP_METHOD_DELETE, url,
+ requestHeaders);
+ op.execute(tracingContext);
+ return op;
+ }
+
+ AbfsRestOperation getAbfsRestOperation(final AbfsRestOperationType operationType,
+ final String httpMethod,
+ final URL url,
+ final List requestHeaders,
+ final byte[] buffer,
+ final int bufferOffset,
+ final int bufferLength,
+ final String sasTokenForReuse) {
+ return new AbfsRestOperation(
+ operationType,
+ this,
+ httpMethod,
+ url,
+ requestHeaders,
+ buffer,
+ bufferOffset,
+ bufferLength,
+ sasTokenForReuse);
+ }
+
+ AbfsRestOperation getAbfsRestOperation(final AbfsRestOperationType operationType,
+ final String httpMethod,
+ final URL url,
+ final List requestHeaders) {
+ return new AbfsRestOperation(
+ operationType,
+ this,
+ httpMethod,
+ url,
+ requestHeaders
+ );
+ }
+
+ AbfsRestOperation getAbfsRestOperation(final AbfsRestOperationType operationType,
+ final String httpMethod,
+ final URL url,
+ final List requestHeaders,
+ final String sasTokenForReuse) {
+ return new AbfsRestOperation(
+ operationType,
+ this,
+ httpMethod,
+ url,
+ requestHeaders, sasTokenForReuse);
+ }
+
/**
* Get the directory query parameter used by the List Paths REST API and used
* as the path in the continuation token. If the input path is null or the
@@ -1007,6 +1775,19 @@ private String appendSASTokenToQuery(String path,
return sasToken;
}
+ private URL createBlobRequestUrl(final AbfsUriQueryBuilder abfsUriQueryBuilder)
+ throws AzureBlobFileSystemException {
+ return changePrefixFromDfsToBlob(
+ createRequestUrl(abfsUriQueryBuilder.toString()));
+ }
+
+ private URL createBlobRequestUrl(final String path,
+ final AbfsUriQueryBuilder abfsUriQueryBuilder)
+ throws AzureBlobFileSystemException {
+ return changePrefixFromDfsToBlob(
+ createRequestUrl(path, abfsUriQueryBuilder.toString()));
+ }
+
private URL createRequestUrl(final String query) throws AzureBlobFileSystemException {
return createRequestUrl(EMPTY_STRING, query);
}
@@ -1096,6 +1877,12 @@ String initializeUserAgent(final AbfsConfiguration abfsConfiguration,
appendIfNotEmpty(sb,
ExtensionHelper.getUserAgentSuffix(tokenProvider, EMPTY_STRING), true);
+ if (abfsConfiguration.isExpectHeaderEnabled()) {
+ sb.append(SINGLE_WHITE_SPACE);
+ sb.append(HUNDRED_CONTINUE);
+ sb.append(SEMICOLON);
+ }
+
sb.append(SINGLE_WHITE_SPACE);
sb.append(abfsConfiguration.getClusterName());
sb.append(FORWARD_SLASH);
@@ -1154,4 +1941,13 @@ public ListenableFuture> submit(Runnable runnable) {
public void addCallback(ListenableFuture future, FutureCallback callback) {
Futures.addCallback(future, callback, executorService);
}
+
+ AbfsConfiguration getAbfsConfiguration() {
+ return abfsConfiguration;
+ }
+
+ @VisibleForTesting
+ protected AccessTokenProvider getTokenProvider() {
+ return tokenProvider;
+ }
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingAnalyzer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingAnalyzer.java
index a55c924dd81524..2060de6f14a97e 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingAnalyzer.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingAnalyzer.java
@@ -20,27 +20,30 @@
import java.util.Timer;
import java.util.TimerTask;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
-import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.util.Preconditions;
import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import static org.apache.hadoop.util.Time.now;
+
class AbfsClientThrottlingAnalyzer {
private static final Logger LOG = LoggerFactory.getLogger(
- AbfsClientThrottlingAnalyzer.class);
- private static final int DEFAULT_ANALYSIS_PERIOD_MS = 10 * 1000;
+ AbfsClientThrottlingAnalyzer.class);
private static final int MIN_ANALYSIS_PERIOD_MS = 1000;
private static final int MAX_ANALYSIS_PERIOD_MS = 30000;
private static final double MIN_ACCEPTABLE_ERROR_PERCENTAGE = .1;
private static final double MAX_EQUILIBRIUM_ERROR_PERCENTAGE = 1;
private static final double RAPID_SLEEP_DECREASE_FACTOR = .75;
private static final double RAPID_SLEEP_DECREASE_TRANSITION_PERIOD_MS = 150
- * 1000;
+ * 1000;
private static final double SLEEP_DECREASE_FACTOR = .975;
private static final double SLEEP_INCREASE_FACTOR = 1.05;
private int analysisPeriodMs;
@@ -50,49 +53,86 @@ class AbfsClientThrottlingAnalyzer {
private String name = null;
private Timer timer = null;
private AtomicReference blobMetrics = null;
+ private AtomicLong lastExecutionTime = null;
+ private final AtomicBoolean isOperationOnAccountIdle = new AtomicBoolean(false);
+ private AbfsConfiguration abfsConfiguration = null;
+ private boolean accountLevelThrottlingEnabled = true;
private AbfsClientThrottlingAnalyzer() {
// hide default constructor
}
- /**
- * Creates an instance of the AbfsClientThrottlingAnalyzer class with
- * the specified name.
- *
- * @param name a name used to identify this instance.
- * @throws IllegalArgumentException if name is null or empty.
- */
- AbfsClientThrottlingAnalyzer(String name) throws IllegalArgumentException {
- this(name, DEFAULT_ANALYSIS_PERIOD_MS);
- }
-
/**
* Creates an instance of the AbfsClientThrottlingAnalyzer class with
* the specified name and period.
*
* @param name A name used to identify this instance.
- * @param period The frequency, in milliseconds, at which metrics are
- * analyzed.
+ * @param abfsConfiguration The configuration set.
* @throws IllegalArgumentException If name is null or empty.
* If period is less than 1000 or greater than 30000 milliseconds.
*/
- AbfsClientThrottlingAnalyzer(String name, int period)
- throws IllegalArgumentException {
+ AbfsClientThrottlingAnalyzer(String name, AbfsConfiguration abfsConfiguration)
+ throws IllegalArgumentException {
Preconditions.checkArgument(
- StringUtils.isNotEmpty(name),
- "The argument 'name' cannot be null or empty.");
+ StringUtils.isNotEmpty(name),
+ "The argument 'name' cannot be null or empty.");
+ int period = abfsConfiguration.getAnalysisPeriod();
Preconditions.checkArgument(
- period >= MIN_ANALYSIS_PERIOD_MS && period <= MAX_ANALYSIS_PERIOD_MS,
- "The argument 'period' must be between 1000 and 30000.");
+ period >= MIN_ANALYSIS_PERIOD_MS && period <= MAX_ANALYSIS_PERIOD_MS,
+ "The argument 'period' must be between 1000 and 30000.");
this.name = name;
- this.analysisPeriodMs = period;
+ this.abfsConfiguration = abfsConfiguration;
+ this.accountLevelThrottlingEnabled = abfsConfiguration.accountThrottlingEnabled();
+ this.analysisPeriodMs = abfsConfiguration.getAnalysisPeriod();
+ this.lastExecutionTime = new AtomicLong(now());
this.blobMetrics = new AtomicReference(
- new AbfsOperationMetrics(System.currentTimeMillis()));
+ new AbfsOperationMetrics(System.currentTimeMillis()));
this.timer = new Timer(
- String.format("abfs-timer-client-throttling-analyzer-%s", name), true);
+ String.format("abfs-timer-client-throttling-analyzer-%s", name), true);
this.timer.schedule(new TimerTaskImpl(),
- analysisPeriodMs,
- analysisPeriodMs);
+ analysisPeriodMs,
+ analysisPeriodMs);
+ }
+
+ /**
+ * Resumes the timer if it was stopped.
+ */
+ private void resumeTimer() {
+ blobMetrics = new AtomicReference(
+ new AbfsOperationMetrics(System.currentTimeMillis()));
+ timer.schedule(new TimerTaskImpl(),
+ analysisPeriodMs,
+ analysisPeriodMs);
+ isOperationOnAccountIdle.set(false);
+ }
+
+ /**
+ * Synchronized method to suspend or resume timer.
+ * @param timerFunctionality resume or suspend.
+ * @param timerTask The timertask object.
+ * @return true or false.
+ */
+ private synchronized boolean timerOrchestrator(TimerFunctionality timerFunctionality,
+ TimerTask timerTask) {
+ switch (timerFunctionality) {
+ case RESUME:
+ if (isOperationOnAccountIdle.get()) {
+ resumeTimer();
+ }
+ break;
+ case SUSPEND:
+ if (accountLevelThrottlingEnabled && (System.currentTimeMillis()
+ - lastExecutionTime.get() >= getOperationIdleTimeout())) {
+ isOperationOnAccountIdle.set(true);
+ timerTask.cancel();
+ timer.purge();
+ return true;
+ }
+ break;
+ default:
+ break;
+ }
+ return false;
}
/**
@@ -104,12 +144,13 @@ private AbfsClientThrottlingAnalyzer() {
public void addBytesTransferred(long count, boolean isFailedOperation) {
AbfsOperationMetrics metrics = blobMetrics.get();
if (isFailedOperation) {
- metrics.bytesFailed.addAndGet(count);
- metrics.operationsFailed.incrementAndGet();
+ metrics.addBytesFailed(count);
+ metrics.incrementOperationsFailed();
} else {
- metrics.bytesSuccessful.addAndGet(count);
- metrics.operationsSuccessful.incrementAndGet();
+ metrics.addBytesSuccessful(count);
+ metrics.incrementOperationsSuccessful();
}
+ blobMetrics.set(metrics);
}
/**
@@ -117,6 +158,8 @@ public void addBytesTransferred(long count, boolean isFailedOperation) {
* @return true if Thread sleeps(Throttling occurs) else false.
*/
public boolean suspendIfNecessary() {
+ lastExecutionTime.set(now());
+ timerOrchestrator(TimerFunctionality.RESUME, null);
int duration = sleepDuration;
if (duration > 0) {
try {
@@ -134,19 +177,27 @@ int getSleepDuration() {
return sleepDuration;
}
+ int getOperationIdleTimeout() {
+ return abfsConfiguration.getAccountOperationIdleTimeout();
+ }
+
+ AtomicBoolean getIsOperationOnAccountIdle() {
+ return isOperationOnAccountIdle;
+ }
+
private int analyzeMetricsAndUpdateSleepDuration(AbfsOperationMetrics metrics,
int sleepDuration) {
final double percentageConversionFactor = 100;
- double bytesFailed = metrics.bytesFailed.get();
- double bytesSuccessful = metrics.bytesSuccessful.get();
- double operationsFailed = metrics.operationsFailed.get();
- double operationsSuccessful = metrics.operationsSuccessful.get();
+ double bytesFailed = metrics.getBytesFailed().get();
+ double bytesSuccessful = metrics.getBytesSuccessful().get();
+ double operationsFailed = metrics.getOperationsFailed().get();
+ double operationsSuccessful = metrics.getOperationsSuccessful().get();
double errorPercentage = (bytesFailed <= 0)
- ? 0
- : (percentageConversionFactor
- * bytesFailed
- / (bytesFailed + bytesSuccessful));
- long periodMs = metrics.endTime - metrics.startTime;
+ ? 0
+ : (percentageConversionFactor
+ * bytesFailed
+ / (bytesFailed + bytesSuccessful));
+ long periodMs = metrics.getEndTime() - metrics.getStartTime();
double newSleepDuration;
@@ -154,10 +205,10 @@ private int analyzeMetricsAndUpdateSleepDuration(AbfsOperationMetrics metrics,
++consecutiveNoErrorCount;
// Decrease sleepDuration in order to increase throughput.
double reductionFactor =
- (consecutiveNoErrorCount * analysisPeriodMs
- >= RAPID_SLEEP_DECREASE_TRANSITION_PERIOD_MS)
- ? RAPID_SLEEP_DECREASE_FACTOR
- : SLEEP_DECREASE_FACTOR;
+ (consecutiveNoErrorCount * analysisPeriodMs
+ >= RAPID_SLEEP_DECREASE_TRANSITION_PERIOD_MS)
+ ? RAPID_SLEEP_DECREASE_FACTOR
+ : SLEEP_DECREASE_FACTOR;
newSleepDuration = sleepDuration * reductionFactor;
} else if (errorPercentage < MAX_EQUILIBRIUM_ERROR_PERCENTAGE) {
@@ -176,15 +227,15 @@ private int analyzeMetricsAndUpdateSleepDuration(AbfsOperationMetrics metrics,
double additionalDelayNeeded = 5 * analysisPeriodMs;
if (bytesSuccessful > 0) {
additionalDelayNeeded = (bytesSuccessful + bytesFailed)
- * periodMs
- / bytesSuccessful
- - periodMs;
+ * periodMs
+ / bytesSuccessful
+ - periodMs;
}
// amortize the additional delay needed across the estimated number of
// requests during the next period
newSleepDuration = additionalDelayNeeded
- / (operationsFailed + operationsSuccessful);
+ / (operationsFailed + operationsSuccessful);
final double maxSleepDuration = analysisPeriodMs;
final double minSleepDuration = sleepDuration * SLEEP_INCREASE_FACTOR;
@@ -201,16 +252,16 @@ private int analyzeMetricsAndUpdateSleepDuration(AbfsOperationMetrics metrics,
if (LOG.isDebugEnabled()) {
LOG.debug(String.format(
- "%5.5s, %10d, %10d, %10d, %10d, %6.2f, %5d, %5d, %5d",
- name,
- (int) bytesFailed,
- (int) bytesSuccessful,
- (int) operationsFailed,
- (int) operationsSuccessful,
- errorPercentage,
- periodMs,
- (int) sleepDuration,
- (int) newSleepDuration));
+ "%5.5s, %10d, %10d, %10d, %10d, %6.2f, %5d, %5d, %5d",
+ name,
+ (int) bytesFailed,
+ (int) bytesSuccessful,
+ (int) operationsFailed,
+ (int) operationsSuccessful,
+ errorPercentage,
+ periodMs,
+ (int) sleepDuration,
+ (int) newSleepDuration));
}
return (int) newSleepDuration;
@@ -238,12 +289,15 @@ public void run() {
}
long now = System.currentTimeMillis();
- if (now - blobMetrics.get().startTime >= analysisPeriodMs) {
+ if (timerOrchestrator(TimerFunctionality.SUSPEND, this)) {
+ return;
+ }
+ if (now - blobMetrics.get().getStartTime() >= analysisPeriodMs) {
AbfsOperationMetrics oldMetrics = blobMetrics.getAndSet(
- new AbfsOperationMetrics(now));
- oldMetrics.endTime = now;
+ new AbfsOperationMetrics(now));
+ oldMetrics.setEndTime(now);
sleepDuration = analyzeMetricsAndUpdateSleepDuration(oldMetrics,
- sleepDuration);
+ sleepDuration);
}
} finally {
if (doWork) {
@@ -252,24 +306,4 @@ public void run() {
}
}
}
-
- /**
- * Stores Abfs operation metrics during each analysis period.
- */
- static class AbfsOperationMetrics {
- private AtomicLong bytesFailed;
- private AtomicLong bytesSuccessful;
- private AtomicLong operationsFailed;
- private AtomicLong operationsSuccessful;
- private long endTime;
- private long startTime;
-
- AbfsOperationMetrics(long startTime) {
- this.startTime = startTime;
- this.bytesFailed = new AtomicLong();
- this.bytesSuccessful = new AtomicLong();
- this.operationsFailed = new AtomicLong();
- this.operationsSuccessful = new AtomicLong();
- }
- }
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java
index 7303e833418db7..900c9c39fe6763 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java
@@ -19,13 +19,17 @@
package org.apache.hadoop.fs.azurebfs.services;
import java.net.HttpURLConnection;
+import java.util.concurrent.locks.ReentrantLock;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
import org.apache.hadoop.fs.azurebfs.AbfsStatistic;
import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import static java.net.HttpURLConnection.HTTP_UNAVAILABLE;
+
/**
* Throttles Azure Blob File System read and write operations to achieve maximum
* throughput by minimizing errors. The errors occur when the account ingress
@@ -38,35 +42,101 @@
* and sleeps just enough to minimize errors, allowing optimal ingress and/or
* egress throughput.
*/
-public final class AbfsClientThrottlingIntercept {
+public final class AbfsClientThrottlingIntercept implements AbfsThrottlingIntercept {
private static final Logger LOG = LoggerFactory.getLogger(
AbfsClientThrottlingIntercept.class);
private static final String RANGE_PREFIX = "bytes=";
- private static AbfsClientThrottlingIntercept singleton = null;
- private AbfsClientThrottlingAnalyzer readThrottler = null;
- private AbfsClientThrottlingAnalyzer writeThrottler = null;
- private static boolean isAutoThrottlingEnabled = false;
+ private static AbfsClientThrottlingIntercept singleton; // singleton, initialized in static initialization block
+ private static final ReentrantLock LOCK = new ReentrantLock();
+ private final AbfsClientThrottlingAnalyzer readThrottler;
+ private final AbfsClientThrottlingAnalyzer writeThrottler;
+ private final String accountName;
// Hide default constructor
- private AbfsClientThrottlingIntercept() {
- readThrottler = new AbfsClientThrottlingAnalyzer("read");
- writeThrottler = new AbfsClientThrottlingAnalyzer("write");
+ public AbfsClientThrottlingIntercept(String accountName, AbfsConfiguration abfsConfiguration) {
+ this.accountName = accountName;
+ this.readThrottler = setAnalyzer("read " + accountName, abfsConfiguration);
+ this.writeThrottler = setAnalyzer("write " + accountName, abfsConfiguration);
+ LOG.debug("Client-side throttling is enabled for the ABFS file system for the account : {}", accountName);
}
- public static synchronized void initializeSingleton(boolean enableAutoThrottling) {
- if (!enableAutoThrottling) {
- return;
- }
+ // Hide default constructor
+ private AbfsClientThrottlingIntercept(AbfsConfiguration abfsConfiguration) {
+ // Account name is kept as empty as same instance is shared across all accounts.
+ this.accountName = "";
+ this.readThrottler = setAnalyzer("read", abfsConfiguration);
+ this.writeThrottler = setAnalyzer("write", abfsConfiguration);
+ LOG.debug("Client-side throttling is enabled for the ABFS file system using singleton intercept");
+ }
+
+ /**
+ * Sets the analyzer for the intercept.
+ * @param name Name of the analyzer.
+ * @param abfsConfiguration The configuration.
+ * @return AbfsClientThrottlingAnalyzer instance.
+ */
+ private AbfsClientThrottlingAnalyzer setAnalyzer(String name, AbfsConfiguration abfsConfiguration) {
+ return new AbfsClientThrottlingAnalyzer(name, abfsConfiguration);
+ }
+
+ /**
+ * Returns the analyzer for read operations.
+ * @return AbfsClientThrottlingAnalyzer for read.
+ */
+ AbfsClientThrottlingAnalyzer getReadThrottler() {
+ return readThrottler;
+ }
+
+ /**
+ * Returns the analyzer for write operations.
+ * @return AbfsClientThrottlingAnalyzer for write.
+ */
+ AbfsClientThrottlingAnalyzer getWriteThrottler() {
+ return writeThrottler;
+ }
+
+ /**
+ * Creates a singleton object of the AbfsClientThrottlingIntercept.
+ * which is shared across all filesystem instances.
+ * @param abfsConfiguration configuration set.
+ * @return singleton object of intercept.
+ */
+ static AbfsClientThrottlingIntercept initializeSingleton(AbfsConfiguration abfsConfiguration) {
if (singleton == null) {
- singleton = new AbfsClientThrottlingIntercept();
- isAutoThrottlingEnabled = true;
- LOG.debug("Client-side throttling is enabled for the ABFS file system.");
+ LOCK.lock();
+ try {
+ if (singleton == null) {
+ singleton = new AbfsClientThrottlingIntercept(abfsConfiguration);
+ LOG.debug("Client-side throttling is enabled for the ABFS file system.");
+ }
+ } finally {
+ LOCK.unlock();
+ }
}
+ return singleton;
+ }
+
+ /**
+ * Updates the metrics for the case when response code signifies throttling
+ * but there are some expected bytes to be sent.
+ * @param isThrottledOperation returns true if status code is HTTP_UNAVAILABLE
+ * @param abfsHttpOperation Used for status code and data transferred.
+ * @return true if the operation is throttled and has some bytes to transfer.
+ */
+ private boolean updateBytesTransferred(boolean isThrottledOperation,
+ AbfsHttpOperation abfsHttpOperation) {
+ return isThrottledOperation && abfsHttpOperation.getExpectedBytesToBeSent() > 0;
}
- static void updateMetrics(AbfsRestOperationType operationType,
- AbfsHttpOperation abfsHttpOperation) {
- if (!isAutoThrottlingEnabled || abfsHttpOperation == null) {
+ /**
+ * Updates the metrics for successful and failed read and write operations.
+ * @param operationType Only applicable for read and write operations.
+ * @param abfsHttpOperation Used for status code and data transferred.
+ */
+ @Override
+ public void updateMetrics(AbfsRestOperationType operationType,
+ AbfsHttpOperation abfsHttpOperation) {
+ if (abfsHttpOperation == null) {
return;
}
@@ -78,19 +148,34 @@ static void updateMetrics(AbfsRestOperationType operationType,
boolean isFailedOperation = (status < HttpURLConnection.HTTP_OK
|| status >= HttpURLConnection.HTTP_INTERNAL_ERROR);
+ // If status code is 503, it is considered as a throttled operation.
+ boolean isThrottledOperation = (status == HTTP_UNAVAILABLE);
+
switch (operationType) {
case Append:
+ case PutBlock:
contentLength = abfsHttpOperation.getBytesSent();
+ if (contentLength == 0) {
+ /*
+ Signifies the case where we could not update the bytesSent due to
+ throttling but there were some expectedBytesToBeSent.
+ */
+ if (updateBytesTransferred(isThrottledOperation, abfsHttpOperation)) {
+ LOG.debug("Updating metrics due to throttling for path {}", abfsHttpOperation.getConnUrl().getPath());
+ contentLength = abfsHttpOperation.getExpectedBytesToBeSent();
+ }
+ }
if (contentLength > 0) {
- singleton.writeThrottler.addBytesTransferred(contentLength,
+ writeThrottler.addBytesTransferred(contentLength,
isFailedOperation);
}
break;
case ReadFile:
+ case GetBlob:
String range = abfsHttpOperation.getConnection().getRequestProperty(HttpHeaderConfigurations.RANGE);
contentLength = getContentLengthIfKnown(range);
if (contentLength > 0) {
- singleton.readThrottler.addBytesTransferred(contentLength,
+ readThrottler.addBytesTransferred(contentLength,
isFailedOperation);
}
break;
@@ -104,21 +189,20 @@ static void updateMetrics(AbfsRestOperationType operationType,
* uses this to suspend the request, if necessary, to minimize errors and
* maximize throughput.
*/
- static void sendingRequest(AbfsRestOperationType operationType,
+ @Override
+ public void sendingRequest(AbfsRestOperationType operationType,
AbfsCounters abfsCounters) {
- if (!isAutoThrottlingEnabled) {
- return;
- }
-
switch (operationType) {
case ReadFile:
- if (singleton.readThrottler.suspendIfNecessary()
+ case GetBlob:
+ if (readThrottler.suspendIfNecessary()
&& abfsCounters != null) {
abfsCounters.incrementCounter(AbfsStatistic.READ_THROTTLES, 1);
}
break;
case Append:
- if (singleton.writeThrottler.suspendIfNecessary()
+ case PutBlock:
+ if (writeThrottler.suspendIfNecessary()
&& abfsCounters != null) {
abfsCounters.incrementCounter(AbfsStatistic.WRITE_THROTTLES, 1);
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsLease.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsLease.java
new file mode 100644
index 00000000000000..f72658fb789aa5
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsDfsLease.java
@@ -0,0 +1,66 @@
+/**
+ * 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.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+
+public class AbfsDfsLease extends AbfsLease {
+
+ public AbfsDfsLease(final AbfsClient client,
+ final String path,
+ final int acquireMaxRetries,
+ final int acquireRetryInterval,
+ final Integer leaseDuration,
+ final TracingContext tracingContext) throws AzureBlobFileSystemException {
+ super(client, path, acquireMaxRetries, acquireRetryInterval, leaseDuration,
+ tracingContext);
+ }
+
+ public AbfsDfsLease(final AbfsClient client,
+ final String path,
+ final Integer leaseDuration,
+ final TracingContext tracingContext) throws AzureBlobFileSystemException {
+ super(client, path, leaseDuration, tracingContext);
+ }
+
+ @Override
+ String callRenewLeaseAPI(final String path,
+ final String leaseId,
+ final TracingContext tracingContext) throws AzureBlobFileSystemException {
+ AbfsRestOperation op = client.renewLease(path, leaseId, tracingContext);
+ return op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_LEASE_ID);
+ }
+
+ @Override
+ AbfsRestOperation callAcquireLeaseAPI(final String path, final Integer leaseDuration,
+ final TracingContext tracingContext)
+ throws AzureBlobFileSystemException {
+ return client.acquireLease(path,
+ leaseDuration, tracingContext);
+ }
+
+ @Override
+ void callReleaseLeaseAPI(final String path, final String leaseID, final TracingContext tracingContext)
+ throws AzureBlobFileSystemException {
+ client.releaseLease(path, leaseID, tracingContext);
+ }
+}
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 e15795efee68d0..1799f73f7307b2 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
@@ -29,10 +29,12 @@
@InterfaceStability.Evolving
public final class AbfsErrors {
public static final String ERR_WRITE_WITHOUT_LEASE = "Attempted to write to file without lease";
- public static final String ERR_LEASE_EXPIRED = "A lease ID was specified, but the lease for the"
- + " resource has expired";
+ public static final String ERR_LEASE_EXPIRED = "A lease ID was specified, but the lease for the blob has expired.";
+ public static final String ERR_LEASE_EXPIRED_DFS = "A lease ID was specified, but the lease for the resource has expired.";
public static final String ERR_NO_LEASE_ID_SPECIFIED = "There is currently a lease on the "
+ "resource and no lease ID was specified in the request";
+ public static final String ERR_NO_LEASE_ID_SPECIFIED_BLOB = "There is currently a lease on the "
+ + "blob and no lease ID was specified in the request";
public static final String ERR_PARALLEL_ACCESS_DETECTED = "Parallel access to the create path "
+ "detected. Failing request to honor single writer semantics";
public static final String ERR_ACQUIRING_LEASE = "Unable to acquire lease";
@@ -48,6 +50,11 @@ public final class AbfsErrors {
+ "operation";
public static final String ERR_NO_LEASE_THREADS = "Lease desired but no lease threads "
+ "configured, set " + FS_AZURE_LEASE_THREADS;
+ public static final String PATH_EXISTS = "The specified path, or an element of the path, " +
+ "exists and its resource type is invalid for this operation.";
+ public static final String BLOB_OPERATION_NOT_SUPPORTED = "Blob operation is not supported.";
+ public static final String CONDITION_NOT_MET = "The condition specified using " +
+ "HTTP conditional header(s) is not met.";
private AbfsErrors() {}
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
index 0a3d773ece5418..2d38fa29845ccb 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
@@ -23,13 +23,24 @@
import java.io.OutputStream;
import java.net.HttpURLConnection;
import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
import java.util.List;
+import java.util.Map;
import javax.net.ssl.HttpsURLConnection;
import javax.net.ssl.SSLSocketFactory;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.parsers.SAXParser;
+import javax.xml.parsers.SAXParserFactory;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.classification.VisibleForTesting;
import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
+
import org.codehaus.jackson.JsonFactory;
import org.codehaus.jackson.JsonParser;
import org.codehaus.jackson.JsonToken;
@@ -37,11 +48,31 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.xml.sax.SAXException;
import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsPerfLoggable;
import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
+import org.w3c.dom.Document;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCKLIST;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK_NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOB_ERROR_CODE_END_XML;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOB_ERROR_CODE_START_XML;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COMMITTED_BLOCKS;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EQUAL;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOB_ERROR_MESSAGE_END_XML;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOB_ERROR_MESSAGE_START_XML;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.NAME;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_COMP;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COMP_LIST;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT;
/**
* Represents an HTTP operation.
@@ -56,7 +87,6 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
private static final int ONE_THOUSAND = 1000;
private static final int ONE_MILLION = ONE_THOUSAND * ONE_THOUSAND;
-
private final String method;
private final URL url;
private String maskedUrl;
@@ -73,6 +103,7 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
// metrics
private int bytesSent;
+ private int expectedBytesToBeSent;
private long bytesReceived;
// optional trace enabled metrics
@@ -81,6 +112,24 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
private long sendRequestTimeMs;
private long recvResponseTimeMs;
private boolean shouldMask = false;
+ private List blockIdList = new ArrayList<>();
+ private BlobList blobList;
+
+ private static final ThreadLocal saxParserThreadLocal
+ = new ThreadLocal() {
+ @Override
+ public SAXParser initialValue() {
+ SAXParserFactory factory = SAXParserFactory.newInstance();
+ factory.setNamespaceAware(true);
+ try {
+ return factory.newSAXParser();
+ } catch (SAXException e) {
+ throw new RuntimeException("Unable to create SAXParser", e);
+ } catch (ParserConfigurationException e) {
+ throw new RuntimeException("Check parser configuration", e);
+ }
+ }
+ };
public static AbfsHttpOperation getAbfsHttpOperationWithFixedResult(
final URL url,
@@ -100,7 +149,7 @@ public static AbfsHttpOperation getAbfsHttpOperationWithFixedResult(
protected AbfsHttpOperation(final URL url,
final String method,
final int httpStatus) {
- this.isTraceEnabled = LOG.isTraceEnabled();
+ this.isTraceEnabled = true;
this.url = url;
this.method = method;
this.statusCode = httpStatus;
@@ -139,6 +188,10 @@ public String getClientRequestId() {
.getRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID);
}
+ public String getRequestHeaderValue(String requestHeader) {
+ return connection.getRequestProperty(requestHeader);
+ }
+
public String getExpectedAppendPos() {
return expectedAppendPos;
}
@@ -155,6 +208,10 @@ public int getBytesSent() {
return bytesSent;
}
+ public int getExpectedBytesToBeSent() {
+ return expectedBytesToBeSent;
+ }
+
public long getBytesReceived() {
return bytesReceived;
}
@@ -167,6 +224,18 @@ public String getResponseHeader(String httpHeader) {
return connection.getHeaderField(httpHeader);
}
+ public Map> getResponseHeaders() {
+ return connection.getHeaderFields();
+ }
+
+ public List getBlockIdList() {
+ return blockIdList;
+ }
+
+ public BlobList getBlobList() {
+ return blobList;
+ }
+
// Returns a trace message for the request
@Override
public String toString() {
@@ -263,7 +332,7 @@ public String getMaskedEncodedUrl() {
*/
public AbfsHttpOperation(final URL url, final String method, final List requestHeaders)
throws IOException {
- this.isTraceEnabled = LOG.isTraceEnabled();
+ this.isTraceEnabled = true;
this.url = url;
this.method = method;
@@ -282,7 +351,7 @@ public AbfsHttpOperation(final URL url, final String method, final List= HttpURLConnection.HTTP_BAD_REQUEST) {
- processStorageErrorResponse();
+ processServerErrorResponse();
if (this.isTraceEnabled) {
this.recvResponseTimeMs += elapsedTimeMs(startTime);
}
@@ -388,7 +488,13 @@ public void processResponse(final byte[] buffer, final int offset, final int len
// this is a list operation and need to retrieve the data
// need a better solution
if (AbfsHttpConstants.HTTP_METHOD_GET.equals(this.method) && buffer == null) {
- parseListFilesResponse(stream);
+ if (url.toString().contains(QUERY_PARAM_COMP + EQUAL + BLOCKLIST)) {
+ parseBlockListResponse(stream);
+ } else if (url.toString().contains(COMP_LIST)) {
+ parseListBlobResponse(stream);
+ } else {
+ parseListFilesResponse(stream);
+ }
} else {
if (buffer != null) {
while (totalBytesRead < length) {
@@ -412,6 +518,10 @@ public void processResponse(final byte[] buffer, final int offset, final int len
} catch (IOException ex) {
LOG.error("UnexpectedError: ", ex);
throw ex;
+ } catch (ParserConfigurationException e) {
+ throw new RuntimeException("Check parser configuration", e);
+ } catch (SAXException e) {
+ throw new RuntimeException("SAX parser exception", e);
} finally {
if (this.isTraceEnabled) {
this.recvResponseTimeMs += elapsedTimeMs(startTime);
@@ -421,10 +531,92 @@ public void processResponse(final byte[] buffer, final int offset, final int len
}
}
+ @VisibleForTesting
+ void processServerErrorResponse() throws IOException {
+ if (getBaseUrl().contains(WASB_DNS_PREFIX)) {
+ processBlobStorageErrorResponse();
+ } else {
+ processDfsStorageErrorResponse();
+ }
+ }
+
+ /**
+ * Parse the stream from the response and set {@link #blobList} field of this
+ * class.
+ *
+ * @param stream inputStream from the server-response.
+ */
+ private void parseListBlobResponse(final InputStream stream) {
+ try {
+ final SAXParser saxParser = saxParserThreadLocal.get();
+ saxParser.reset();
+ BlobList blobList = new BlobList();
+ saxParser.parse(stream, new BlobListXmlParser(blobList, getBaseUrl()));
+ this.blobList = blobList;
+ } catch (SAXException | IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private String getBaseUrl() {
+ String urlStr = url.toString();
+ int queryParamStart = urlStr.indexOf("?");
+ if (queryParamStart == -1) {
+ return urlStr;
+ }
+ return urlStr.substring(0, queryParamStart);
+ }
+
public void setRequestProperty(String key, String value) {
this.connection.setRequestProperty(key, value);
}
+ @VisibleForTesting
+ void setConnection(HttpURLConnection connection) {
+ this.connection = connection;
+ }
+
+ /**
+ * Parses the get block list response and returns list of committed blocks.
+ *
+ * @param stream InputStream contains the list results.
+ * @throws IOException, ParserConfigurationException, SAXException
+ */
+ private void parseBlockListResponse(final InputStream stream) throws IOException, ParserConfigurationException, SAXException {
+ if (stream == null) {
+ return;
+ }
+
+ if (blockIdList.size() != 0) {
+ // already parsed the response
+ return;
+ }
+
+ // Convert the input stream to a Document object
+ DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
+ Document doc = factory.newDocumentBuilder().parse(stream);
+
+// Find the CommittedBlocks element and extract the list of block IDs
+ NodeList committedBlocksList = doc.getElementsByTagName(COMMITTED_BLOCKS);
+ if (committedBlocksList.getLength() > 0) {
+ Node committedBlocks = committedBlocksList.item(0);
+ NodeList blockList = committedBlocks.getChildNodes();
+ for (int i = 0; i < blockList.getLength(); i++) {
+ Node block = blockList.item(i);
+ if (block.getNodeName().equals(BLOCK_NAME)) {
+ NodeList nameList = block.getChildNodes();
+ for (int j = 0; j < nameList.getLength(); j++) {
+ Node name = nameList.item(j);
+ if (name.getNodeName().equals(NAME)) {
+ String blockId = name.getTextContent();
+ blockIdList.add(blockId);
+ }
+ }
+ }
+ }
+ }
+ }
+
/**
* Open the HTTP connection.
*
@@ -459,8 +651,8 @@ private HttpURLConnection openConnection() throws IOException {
* }
*
*/
- private void processStorageErrorResponse() {
- try (InputStream stream = connection.getErrorStream()) {
+ private void processDfsStorageErrorResponse() {
+ try (InputStream stream = getConnectionErrorStream()) {
if (stream == null) {
return;
}
@@ -501,6 +693,48 @@ private void processStorageErrorResponse() {
}
}
+ /**
+ * Extract errorCode and errorMessage from errorStream populated by server.
+ * Error-message in the form of:
+ *
+ *
+ * Reference
+ */
+ private void processBlobStorageErrorResponse() throws IOException {
+ InputStream errorStream = getConnectionErrorStream();
+ if (errorStream == null) {
+ return;
+ }
+ final String data = IOUtils.toString(errorStream, StandardCharsets.UTF_8);
+
+ int codeStartFirstInstance = data.indexOf(BLOB_ERROR_CODE_START_XML);
+ int codeEndFirstInstance = data.indexOf(BLOB_ERROR_CODE_END_XML);
+ if (codeEndFirstInstance != -1 && codeStartFirstInstance != -1) {
+ storageErrorCode = data.substring(codeStartFirstInstance,
+ codeEndFirstInstance).replace(BLOB_ERROR_CODE_START_XML, "");
+ }
+
+ int msgStartFirstInstance = data.indexOf(BLOB_ERROR_MESSAGE_START_XML);
+ int msgEndFirstInstance = data.indexOf(BLOB_ERROR_MESSAGE_END_XML);
+ if (msgEndFirstInstance != -1 && msgStartFirstInstance != -1) {
+ storageErrorMessage = data.substring(msgStartFirstInstance,
+ msgEndFirstInstance).replace(BLOB_ERROR_MESSAGE_START_XML, "");
+ }
+ }
+
+ @VisibleForTesting
+ InputStream getConnectionErrorStream() {
+ return connection.getErrorStream();
+ }
+
/**
* Returns the elapsed time in milliseconds.
*/
@@ -541,6 +775,58 @@ private boolean isNullInputStream(InputStream stream) {
return stream == null ? true : false;
}
+ /**
+ * Gets the connection request property for a key.
+ * @param key The request property key.
+ * @return request peoperty value.
+ */
+ String getConnProperty(String key) {
+ return connection.getRequestProperty(key);
+ }
+
+ /**
+ * Gets the connection url.
+ * @return url.
+ */
+ URL getConnUrl() {
+ return connection.getURL();
+ }
+
+ /**
+ * Gets the connection request method.
+ * @return request method.
+ */
+ String getConnRequestMethod() {
+ return connection.getRequestMethod();
+ }
+
+ /**
+ * Gets the connection response code.
+ * @return response code.
+ * @throws IOException
+ */
+ Integer getConnResponseCode() throws IOException {
+ return connection.getResponseCode();
+ }
+
+ /**
+ * Gets the connection output stream.
+ * @return output stream.
+ * @throws IOException
+ */
+ OutputStream getConnOutputStream() throws IOException {
+ return connection.getOutputStream();
+ }
+
+ /**
+ * Gets the connection response message.
+ * @return response message.
+ * @throws IOException
+ */
+ String getConnResponseMessage() throws IOException {
+ return connection.getResponseMessage();
+ }
+
public static class AbfsHttpOperationWithFixedResult extends AbfsHttpOperation {
/**
* Creates an instance to represent fixed results.
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
index 7033ae9a4a0399..e17db10c91cd1d 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
@@ -50,6 +50,7 @@
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_KB;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.STREAM_ID_LEN;
+import static org.apache.hadoop.fs.azurebfs.constants.InternalConstants.CAPABILITY_SAFE_READAHEAD;
import static org.apache.hadoop.util.StringUtils.toLowerCase;
/**
@@ -137,7 +138,7 @@ public AbfsInputStream(
this.tolerateOobAppends = abfsInputStreamContext.isTolerateOobAppends();
this.eTag = eTag;
this.readAheadRange = abfsInputStreamContext.getReadAheadRange();
- this.readAheadEnabled = true;
+ this.readAheadEnabled = abfsInputStreamContext.isReadAheadEnabled();
this.alwaysReadBufferSize
= abfsInputStreamContext.shouldReadBufferSizeAlways();
this.bufferedPreadDisabled = abfsInputStreamContext
@@ -147,7 +148,16 @@ public AbfsInputStream(
this.streamStatistics = abfsInputStreamContext.getStreamStatistics();
this.inputStreamId = createInputStreamId();
this.tracingContext = new TracingContext(tracingContext);
- this.tracingContext.setOperation(FSOperationType.READ);
+ /*
+ * If this inputStream is getting opened in listStatus or GetFileStatus, it would
+ * be in the flow of rename-resume. It required that all operations have the
+ * same primaryId and opType as that of the listStatus or getFileStatus which
+ * is invoking the rename-resume.
+ */
+ if (tracingContext.getOpType() != FSOperationType.LISTSTATUS
+ && this.tracingContext.getOpType() != FSOperationType.GET_FILESTATUS) {
+ this.tracingContext.setOperation(FSOperationType.READ);
+ }
this.tracingContext.setStreamID(inputStreamId);
this.context = abfsInputStreamContext;
readAheadBlockSize = abfsInputStreamContext.getReadAheadBlockSize();
@@ -745,6 +755,11 @@ byte[] getBuffer() {
return buffer;
}
+ @VisibleForTesting
+ public boolean isReadAheadEnabled() {
+ return readAheadEnabled;
+ }
+
@VisibleForTesting
public int getReadAheadRange() {
return readAheadRange;
@@ -823,11 +838,12 @@ public IOStatistics getIOStatistics() {
@Override
public String toString() {
final StringBuilder sb = new StringBuilder(super.toString());
+ sb.append("AbfsInputStream@(").append(this.hashCode()).append("){");
+ sb.append("[" + CAPABILITY_SAFE_READAHEAD + "]");
if (streamStatistics != null) {
- sb.append("AbfsInputStream@(").append(this.hashCode()).append("){");
- sb.append(streamStatistics.toString());
- sb.append("}");
+ sb.append(", ").append(streamStatistics);
}
+ sb.append("}");
return sb.toString();
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java
index 55f01bf15bcf71..05afc7b9858da8 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java
@@ -35,6 +35,8 @@ public class AbfsInputStreamContext extends AbfsStreamContext {
private boolean tolerateOobAppends;
+ private boolean isReadAheadEnabled = true;
+
private boolean alwaysReadBufferSize;
private int readAheadBlockSize;
@@ -72,6 +74,12 @@ public AbfsInputStreamContext withTolerateOobAppends(
return this;
}
+ public AbfsInputStreamContext isReadAheadEnabled(
+ final boolean isReadAheadEnabled) {
+ this.isReadAheadEnabled = isReadAheadEnabled;
+ return this;
+ }
+
public AbfsInputStreamContext withReadAheadRange(
final int readAheadRange) {
this.readAheadRange = readAheadRange;
@@ -141,6 +149,10 @@ public boolean isTolerateOobAppends() {
return tolerateOobAppends;
}
+ public boolean isReadAheadEnabled() {
+ return isReadAheadEnabled;
+ }
+
public int getReadAheadRange() {
return readAheadRange;
}
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 2e97598ef04f35..4c3c3e0d16bf81 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
@@ -19,7 +19,10 @@
package org.apache.hadoop.fs.azurebfs.services;
import java.io.IOException;
+import java.util.Timer;
+import java.util.TimerTask;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback;
@@ -52,7 +55,7 @@
* 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.
*/
-public final class AbfsLease {
+public abstract class AbfsLease {
private static final Logger LOG = LoggerFactory.getLogger(AbfsLease.class);
// Number of retries for acquiring lease
@@ -60,16 +63,19 @@ public final class AbfsLease {
// Retry interval for acquiring lease in secs
static final int DEFAULT_LEASE_ACQUIRE_RETRY_INTERVAL = 10;
- private final AbfsClient client;
+ protected final AbfsClient client;
private final String path;
private final TracingContext tracingContext;
// Lease status variables
private volatile boolean leaseFreed;
- private volatile String leaseID = null;
+ private AtomicReference leaseID = new AtomicReference<>();
private volatile Throwable exception = null;
private volatile int acquireRetryCount = 0;
private volatile ListenableScheduledFuture future = null;
+ private final Integer leaseDuration;
+
+ private Timer timer = null;
public static class LeaseException extends AzureBlobFileSystemException {
public LeaseException(Throwable t) {
@@ -81,20 +87,36 @@ public LeaseException(String s) {
}
}
- public AbfsLease(AbfsClient client, String path, TracingContext tracingContext) throws AzureBlobFileSystemException {
+ /**
+ * @param client client object for making server calls
+ * @param path path on which lease has to be acquired, renewed and freed in future
+ * @param leaseDuration duration for which lease to be taken in seconds. If given
+ * null, it will be taken as infinte-lease.
+ * @param tracingContext for tracing server calls
+ *
+ * @throws AzureBlobFileSystemException exception while calling acquireLease API
+ */
+ public AbfsLease(AbfsClient client, String path,
+ final Integer leaseDuration,
+ TracingContext tracingContext) throws AzureBlobFileSystemException {
this(client, path, DEFAULT_LEASE_ACQUIRE_MAX_RETRIES,
- DEFAULT_LEASE_ACQUIRE_RETRY_INTERVAL, tracingContext);
+ DEFAULT_LEASE_ACQUIRE_RETRY_INTERVAL, leaseDuration, tracingContext);
}
@VisibleForTesting
public AbfsLease(AbfsClient client, String path, int acquireMaxRetries,
- int acquireRetryInterval, TracingContext tracingContext) throws AzureBlobFileSystemException {
+ int acquireRetryInterval, final Integer leaseDuration, TracingContext tracingContext) throws AzureBlobFileSystemException {
this.leaseFreed = false;
this.client = client;
this.path = path;
this.tracingContext = tracingContext;
+ this.leaseDuration = leaseDuration;
- if (client.getNumLeaseThreads() < 1) {
+ /*
+ * If the number of threads to use for lease operations for infinite lease directories
+ * and the object is created for infinite-lease (leaseDuration == null).
+ */
+ if (client.getNumLeaseThreads() < 1 && leaseDuration == null) {
throw new LeaseException(ERR_NO_LEASE_THREADS);
}
@@ -104,7 +126,7 @@ public AbfsLease(AbfsClient client, String path, int acquireMaxRetries,
acquireLease(retryPolicy, 0, acquireRetryInterval, 0,
new TracingContext(tracingContext));
- while (leaseID == null && exception == null) {
+ while (leaseID.get() == null && exception == null) {
try {
future.get();
} catch (Exception e) {
@@ -122,18 +144,25 @@ public AbfsLease(AbfsClient client, String path, int acquireMaxRetries,
private void acquireLease(RetryPolicy retryPolicy, int numRetries,
int retryInterval, long delay, TracingContext tracingContext)
- throws LeaseException {
+ throws AzureBlobFileSystemException {
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),
+ if (leaseDuration != null) {
+ leaseID.set(
+ callAcquireLeaseAPI(path, leaseDuration, tracingContext).getResult()
+ .getResponseHeader(HttpHeaderConfigurations.X_MS_LEASE_ID));
+ spawnLeaseRenewTimer(path, leaseDuration * 1000);
+ return;
+ }
+ future = client.schedule(() -> callAcquireLeaseAPI(path,
+ INFINITE_LEASE_DURATION, tracingContext),
delay, TimeUnit.SECONDS);
client.addCallback(future, new FutureCallback() {
@Override
public void onSuccess(@Nullable AbfsRestOperation op) {
- leaseID = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_LEASE_ID);
+ leaseID.set(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_LEASE_ID));
LOG.debug("Acquired lease {} on {}", leaseID, path);
}
@@ -156,6 +185,28 @@ public void onFailure(Throwable throwable) {
});
}
+ private void spawnLeaseRenewTimer(String path, Integer leaseDuration) {
+ timer = new Timer();
+ timer.schedule(new TimerTask() {
+ @Override
+ public void run() {
+ try {
+ leaseID.set(callRenewLeaseAPI(path, leaseID.get(), tracingContext));
+ } catch (AzureBlobFileSystemException ignored) {
+ }
+ }
+ }, leaseDuration / 2, leaseDuration / 2);
+ }
+
+ abstract String callRenewLeaseAPI(final String path,
+ final String s,
+ final TracingContext tracingContext) throws AzureBlobFileSystemException;
+
+ abstract AbfsRestOperation callAcquireLeaseAPI(final String path,
+ final Integer leaseDuration,
+ final TracingContext tracingContext)
+ throws AzureBlobFileSystemException;
+
/**
* Cancel future and free the lease. If an exception occurs while releasing the lease, the error
* will be logged. If the lease cannot be released, AzureBlobFileSystem breakLease will need to
@@ -170,9 +221,10 @@ 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);
+ callReleaseLeaseAPI(path, leaseID.get(), tracingContext);
} catch (IOException e) {
LOG.warn("Exception when trying to release lease {} on {}. Lease will need to be broken: {}",
leaseID, path, e.getMessage());
@@ -184,12 +236,21 @@ public void free() {
}
}
+ public void cancelTimer() {
+ if (timer != null) {
+ timer.cancel();
+ }
+ }
+
+ abstract void callReleaseLeaseAPI(final String path, final String leaseID, final TracingContext tracingContext)
+ throws AzureBlobFileSystemException;
+
public boolean isFreed() {
return leaseFreed;
}
public String getLeaseID() {
- return leaseID;
+ return leaseID.get();
}
@VisibleForTesting
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsNoOpThrottlingIntercept.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsNoOpThrottlingIntercept.java
new file mode 100644
index 00000000000000..b88f4a05d337b4
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsNoOpThrottlingIntercept.java
@@ -0,0 +1,37 @@
+/**
+ * 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;
+
+final class AbfsNoOpThrottlingIntercept implements AbfsThrottlingIntercept {
+
+ public static final AbfsNoOpThrottlingIntercept INSTANCE = new AbfsNoOpThrottlingIntercept();
+
+ private AbfsNoOpThrottlingIntercept() {
+ }
+
+ @Override
+ public void updateMetrics(final AbfsRestOperationType operationType,
+ final AbfsHttpOperation abfsHttpOperation) {
+ }
+
+ @Override
+ public void sendingRequest(final AbfsRestOperationType operationType,
+ final AbfsCounters abfsCounters) {
+ }
+}
\ No newline at end of file
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOperationMetrics.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOperationMetrics.java
new file mode 100644
index 00000000000000..d19c69354a2d33
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOperationMetrics.java
@@ -0,0 +1,139 @@
+/**
+ * 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.AtomicLong;
+
+/**
+ * Stores Abfs operation metrics during each analysis period.
+ */
+class AbfsOperationMetrics {
+
+ /**
+ * No of bytes which could not be transferred due to a failed operation.
+ */
+ private final AtomicLong bytesFailed;
+
+ /**
+ * No of bytes successfully transferred during a successful operation.
+ */
+ private final AtomicLong bytesSuccessful;
+
+ /**
+ * Total no of failed operations.
+ */
+ private final AtomicLong operationsFailed;
+
+ /**
+ * Total no of successful operations.
+ */
+ private final AtomicLong operationsSuccessful;
+
+ /**
+ * Time when collection of metrics ended.
+ */
+ private long endTime;
+
+ /**
+ * Time when the collection of metrics started.
+ */
+ private final long startTime;
+
+ AbfsOperationMetrics(long startTime) {
+ this.startTime = startTime;
+ this.bytesFailed = new AtomicLong();
+ this.bytesSuccessful = new AtomicLong();
+ this.operationsFailed = new AtomicLong();
+ this.operationsSuccessful = new AtomicLong();
+ }
+
+ /**
+ *
+ * @return bytes failed to transfer.
+ */
+ AtomicLong getBytesFailed() {
+ return bytesFailed;
+ }
+
+ /**
+ *
+ * @return bytes successfully transferred.
+ */
+ AtomicLong getBytesSuccessful() {
+ return bytesSuccessful;
+ }
+
+ /**
+ *
+ * @return no of operations failed.
+ */
+ AtomicLong getOperationsFailed() {
+ return operationsFailed;
+ }
+
+ /**
+ *
+ * @return no of successful operations.
+ */
+ AtomicLong getOperationsSuccessful() {
+ return operationsSuccessful;
+ }
+
+ /**
+ *
+ * @return end time of metric collection.
+ */
+ long getEndTime() {
+ return endTime;
+ }
+
+ /**
+ *
+ * @param endTime sets the end time.
+ */
+ void setEndTime(final long endTime) {
+ this.endTime = endTime;
+ }
+
+ /**
+ *
+ * @return start time of metric collection.
+ */
+ long getStartTime() {
+ return startTime;
+ }
+
+ void addBytesFailed(long bytes) {
+ this.getBytesFailed().addAndGet(bytes);
+ }
+
+ void addBytesSuccessful(long bytes) {
+ this.getBytesSuccessful().addAndGet(bytes);
+ }
+
+ void incrementOperationsFailed() {
+ this.getOperationsFailed().incrementAndGet();
+ }
+
+ void incrementOperationsSuccessful() {
+ this.getOperationsSuccessful().incrementAndGet();
+ }
+
+}
+
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
index 61160f92bdf391..24f09c58a449df 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
@@ -22,10 +22,19 @@
import java.io.IOException;
import java.io.OutputStream;
import java.net.HttpURLConnection;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.Future;
import java.util.UUID;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService;
@@ -51,7 +60,13 @@
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.Syncable;
+import static java.net.HttpURLConnection.HTTP_CONFLICT;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK_LIST_END_TAG;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.BLOCK_LIST_START_TAG;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.LATEST_BLOCK_FORMAT;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.XML_VERSION;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.STREAM_ID_LEN;
+import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.BLOB_OPERATION_NOT_SUPPORTED;
import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_WRITE_WITHOUT_LEASE;
import static org.apache.hadoop.fs.impl.StoreImplementationUtils.isProbeForSyncable;
import static org.apache.hadoop.io.IOUtils.wrapException;
@@ -80,6 +95,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
private boolean disableOutputStreamFlush;
private boolean enableSmallWriteOptimization;
private boolean isAppendBlob;
+ private boolean isExpectHeaderEnabled;
private volatile IOException lastError;
private long lastFlushOffset;
@@ -114,7 +130,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
private final DataBlocks.BlockFactory blockFactory;
/** Current data block. Null means none currently active. */
- private DataBlocks.DataBlock activeBlock;
+ private AbfsBlock activeBlock;
/** Count of blocks uploaded. */
private long blockCount = 0;
@@ -122,9 +138,30 @@ public class AbfsOutputStream extends OutputStream implements Syncable,
/** The size of a single block. */
private final int blockSize;
+ /** The map to store blockId and Status **/
+ private final LinkedHashMap map = new LinkedHashMap<>();
+
+ /** The list of already committed blocks is stored in this list. */
+ private List committedBlockEntries = new ArrayList<>();
+
+ /** The list of all blockId's for putBlockList. */
+ private final Set blockIdList = new LinkedHashSet<>();
+
+ /** The prefix mode for decision on BLOB or DFS endpoint. */
+ private PrefixMode prefixMode;
+
+ /** The etag of the blob. */
+ private String eTag;
+
/** Executor service to carry out the parallel upload requests. */
private final ListeningExecutorService executorService;
+ /** List to validate order. */
+ private final UniqueArrayList orderedBlockList = new UniqueArrayList<>();
+
+ /** Retry fallback for append on DFS */
+ private static boolean fallbackDFSAppend = false;
+
public AbfsOutputStream(AbfsOutputStreamContext abfsOutputStreamContext)
throws IOException {
this.client = abfsOutputStreamContext.getClient();
@@ -133,6 +170,7 @@ public AbfsOutputStream(AbfsOutputStreamContext abfsOutputStreamContext)
this.position = abfsOutputStreamContext.getPosition();
this.closed = false;
this.supportFlush = abfsOutputStreamContext.isEnableFlush();
+ this.isExpectHeaderEnabled = abfsOutputStreamContext.isExpectHeaderEnabled();
this.disableOutputStreamFlush = abfsOutputStreamContext
.isDisableOutputStreamFlush();
this.enableSmallWriteOptimization
@@ -145,6 +183,7 @@ public AbfsOutputStream(AbfsOutputStreamContext abfsOutputStreamContext)
this.numOfAppendsToServerSinceLastFlush = 0;
this.writeOperations = new ConcurrentLinkedDeque<>();
this.outputStreamStatistics = abfsOutputStreamContext.getStreamStatistics();
+ this.eTag = abfsOutputStreamContext.getETag();
if (this.isAppendBlob) {
this.maxConcurrentRequestCount = 1;
@@ -163,14 +202,93 @@ public AbfsOutputStream(AbfsOutputStreamContext abfsOutputStreamContext)
abfsOutputStreamContext.getSasTokenRenewPeriodForStreamsInSeconds());
this.outputStreamId = createOutputStreamId();
this.tracingContext = new TracingContext(abfsOutputStreamContext.getTracingContext());
+ this.prefixMode = client.getAbfsConfiguration().getPrefixMode();
+ this.blockFactory = abfsOutputStreamContext.getBlockFactory();
+ this.blockSize = bufferSize;
+ if (prefixMode == PrefixMode.BLOB && abfsOutputStreamContext.getPosition() > 0) {
+ // Get the list of all the committed blocks for the given path.
+ this.committedBlockEntries = getBlockList(path, tracingContext);
+ } else {
+ // create that first block. This guarantees that an open + close sequence
+ // writes a 0-byte entry.
+ createBlockIfNeeded(position);
+ }
this.tracingContext.setStreamID(outputStreamId);
this.tracingContext.setOperation(FSOperationType.WRITE);
this.ioStatistics = outputStreamStatistics.getIOStatistics();
- this.blockFactory = abfsOutputStreamContext.getBlockFactory();
- this.blockSize = bufferSize;
- // create that first block. This guarantees that an open + close sequence
- // writes a 0-byte entry.
- createBlockIfNeeded();
+ }
+
+ private final Lock lock = new ReentrantLock();
+
+ private final ReentrantLock mapLock = new ReentrantLock();
+
+ public LinkedHashMap getMap() {
+ return map;
+ }
+
+ /**
+ * Set the eTag of the blob.
+ *
+ * @param eTag eTag.
+ */
+ void setETag(String eTag) {
+ lock.lock();
+ try {
+ this.eTag = eTag;
+ } finally {
+ lock.unlock();
+ }
+ }
+
+ /**
+ * Get eTag value of blob.
+ *
+ * @return eTag.
+ */
+ @VisibleForTesting
+ public String getETag() {
+ lock.lock();
+ try {
+ return eTag;
+ } finally {
+ lock.unlock();
+ }
+ }
+
+ public DataBlocks.BlockFactory getBlockFactory() {
+ return blockFactory;
+ }
+
+ public long getBlockCount() {
+ return blockCount;
+ }
+
+ public int getBlockSize() {
+ return blockSize;
+ }
+
+ public static class UniqueArrayList extends ArrayList {
+ @Override
+ public boolean add(T element) {
+ if (!super.contains(element)) {
+ return super.add(element);
+ }
+ return false;
+ }
+ }
+
+ /**
+ * Returns block id's which are committed for the blob.
+ * @param path The blob path.
+ * @param tracingContext Tracing context object.
+ * @return list of committed block id's.
+ * @throws AzureBlobFileSystemException
+ */
+ private List getBlockList(final String path, TracingContext tracingContext) throws AzureBlobFileSystemException {
+ List committedBlockIdList;
+ final AbfsRestOperation op = client.getBlockList(path, tracingContext);
+ committedBlockIdList = op.getResult().getBlockIdList();
+ return committedBlockIdList;
}
private String createOutputStreamId() {
@@ -207,6 +325,10 @@ public void write(final int byteVal) throws IOException {
* Writes length bytes from the specified byte array starting at off to
* this output stream.
*
+ * Before writing, this method checks for any previous write failures by
+ * calling {@link #maybeThrowLastError()}. If a failure occurred,
+ * it throws the exception that caused it.
+ *
* @param data the byte array to write.
* @param off the start off in the data.
* @param length the number of bytes to write.
@@ -216,6 +338,9 @@ public void write(final int byteVal) throws IOException {
@Override
public synchronized void write(final byte[] data, final int off, final int length)
throws IOException {
+ if (closed) {
+ throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+ }
// validate if data is not null and index out of bounds.
DataBlocks.validateWriteArgs(data, off, length);
maybeThrowLastError();
@@ -227,7 +352,20 @@ public synchronized void write(final byte[] data, final int off, final int lengt
if (hasLease() && isLeaseFreed()) {
throw new PathIOException(path, ERR_WRITE_WITHOUT_LEASE);
}
- DataBlocks.DataBlock block = createBlockIfNeeded();
+
+ if (length == 0) {
+ return;
+ }
+
+ AbfsBlock block = createBlockIfNeeded(position);
+ // Put entry in map with status as NEW which is changed to SUCCESS when successfully appended.
+ try {
+ mapLock.lock();
+ map.put(block.getBlockId(), BlockStatus.NEW);
+ orderedBlockList.add(block.getBlockId());
+ } finally {
+ mapLock.unlock();
+ }
int written = block.write(data, off, length);
int remainingCapacity = block.remainingCapacity();
@@ -254,12 +392,11 @@ public synchronized void write(final byte[] data, final int off, final int lengt
* @return the active block; null if there isn't one.
* @throws IOException on any failure to create
*/
- private synchronized DataBlocks.DataBlock createBlockIfNeeded()
+ private synchronized AbfsBlock createBlockIfNeeded(long offset)
throws IOException {
if (activeBlock == null) {
blockCount++;
- activeBlock = blockFactory
- .create(blockCount, this.blockSize, outputStreamStatistics);
+ activeBlock = new AbfsBlock(this, offset);
}
return activeBlock;
}
@@ -288,7 +425,7 @@ private synchronized void uploadCurrentBlock() throws IOException {
* @param blockToUpload block to upload.
* @throws IOException upload failure
*/
- private void uploadBlockAsync(DataBlocks.DataBlock blockToUpload,
+ private void uploadBlockAsync(AbfsBlock blockToUpload,
boolean isFlush, boolean isClose)
throws IOException {
if (this.isAppendBlob) {
@@ -327,10 +464,52 @@ private void uploadBlockAsync(DataBlocks.DataBlock blockToUpload,
* leaseId - The AbfsLeaseId for this request.
*/
AppendRequestParameters reqParams = new AppendRequestParameters(
- offset, 0, bytesLength, mode, false, leaseId);
- AbfsRestOperation op =
- client.append(path, blockUploadData.toByteArray(), reqParams,
- cachedSasToken.get(), new TracingContext(tracingContext));
+ offset, 0, bytesLength, mode, false, leaseId, isExpectHeaderEnabled);
+ AbfsRestOperation op;
+ if (!OperativeEndpoint.isIngressEnabledOnDFS(prefixMode, client.getAbfsConfiguration())) {
+ try {
+ LOG.debug("Append over Blob for ingress config value {} for path {} ",
+ client.getAbfsConfiguration().shouldIngressFallbackToDfs(), path);
+ TracingContext tracingContextBlobAppend = new TracingContext(tracingContext);
+ tracingContextBlobAppend.setFallbackDFSAppend("B " + offset);
+ op = client.append(blockToUpload.getBlockId(), path, blockUploadData.toByteArray(), reqParams,
+ cachedSasToken.get(), tracingContextBlobAppend, getETag());
+ String key = blockToUpload.getBlockId();
+ try {
+ mapLock.lock();
+ if (!getMap().containsKey(key)) {
+ throw new Exception("Block is missing with blockId " + blockToUpload.getBlockId() +
+ " for offset " + offset + " for path" + path + " with streamId " + outputStreamId);
+ } else {
+ map.put(blockToUpload.getBlockId(), BlockStatus.SUCCESS);
+ }
+ } finally {
+ mapLock.unlock();
+ }
+ } catch (AbfsRestOperationException ex) {
+ // The mechanism to fall back to DFS endpoint if blob operation is not supported.
+ if (ex.getStatusCode() == HTTP_CONFLICT && ex.getMessage().contains(BLOB_OPERATION_NOT_SUPPORTED)) {
+ prefixMode = PrefixMode.DFS;
+ LOG.debug("Retrying append due to fallback for path {} ", path);
+ TracingContext tracingContextAppend = new TracingContext(tracingContext);
+ tracingContextAppend.setFallbackDFSAppend("D");
+ fallbackDFSAppend = true;
+ op = client.append(path, blockUploadData.toByteArray(), reqParams,
+ cachedSasToken.get(), tracingContextAppend);
+ } else {
+ throw ex;
+ }
+ }
+ } else {
+ LOG.debug("Append over DFS for ingress config value {} for path {} ",
+ client.getAbfsConfiguration().shouldIngressFallbackToDfs(), path);
+ TracingContext tracingContextAppend = new TracingContext(tracingContext);
+ if (fallbackDFSAppend) {
+ tracingContextAppend.setFallbackDFSAppend("D");
+ }
+ op = client.append(path, blockUploadData.toByteArray(), reqParams,
+ cachedSasToken.get(), tracingContextAppend);
+ }
cachedSasToken.update(op.getSasToken());
perfInfo.registerResult(op.getResult());
perfInfo.registerSuccess(true);
@@ -371,7 +550,7 @@ private void failureWhileSubmit(Exception ex) throws IOException {
*
* @return the active block; null if there isn't one.
*/
- private synchronized DataBlocks.DataBlock getActiveBlock() {
+ private synchronized AbfsBlock getActiveBlock() {
return activeBlock;
}
@@ -432,6 +611,8 @@ private void maybeThrowLastError() throws IOException {
* written out. If any data remains in the payload it is committed to the
* service. Data is queued for writing and forced out to the service
* before the call returns.
+ *
+ * @throws IOException if error occurs in {@link #flushInternal(boolean)} call
*/
@Override
public void flush() throws IOException {
@@ -442,7 +623,7 @@ public void flush() throws IOException {
/** Similar to posix fsync, flush out the data in client's user buffer
* all the way to the disk device (but the disk may have it in its cache).
- * @throws IOException if error occurs
+ * @throws IOException if error occurs in {@link #flushInternal(boolean)} call
*/
@Override
public void hsync() throws IOException {
@@ -453,7 +634,8 @@ public void hsync() throws IOException {
/** Flush out the data in client's user buffer. After the return of
* this call, new readers will see the data.
- * @throws IOException if any error occurs
+ * @throws IOException if any error occurs in {@link #flushInternal(boolean)}
+ * call
*/
@Override
public void hflush() throws IOException {
@@ -510,6 +692,17 @@ public synchronized void close() throws IOException {
LOG.debug("Closing AbfsOutputStream : {}", this);
}
+ /**
+ * Flushes the internal buffer and uploads the current block to the service.
+ * Before flushing, checks if there has been failure in previous writes. If there
+ * is some data which has not been written to server, that data is written by
+ * {@link #uploadCurrentBlock()}. Post async upload has been invoked,
+ * {@link #flushWrittenBytesToService(boolean)} is called.
+ *
+ * @param isClose stream is closed after this flush call
+ * @throws IOException exception in any of the write operation or server exception
+ * on flush operation
+ */
private synchronized void flushInternal(boolean isClose) throws IOException {
maybeThrowLastError();
@@ -540,6 +733,13 @@ private synchronized void smallWriteOptimizedflushInternal(boolean isClose) thro
numOfAppendsToServerSinceLastFlush = 0;
}
+ /**
+ * Check if previous write failed. If there is a block left to upload, it will
+ * be uploaded. Before flushing, it waits for the async writes to get completed.
+ *
+ * @throws IOException exception received in previous writes, or in async-writes,
+ * or exceptions from server on Flush operation.
+ */
private synchronized void flushInternalAsync() throws IOException {
maybeThrowLastError();
if (hasActiveBlockDataToUpload()) {
@@ -556,7 +756,7 @@ private synchronized void flushInternalAsync() throws IOException {
* the dataBlock or while closing the BlockUploadData.
*/
private void writeAppendBlobCurrentBufferToService() throws IOException {
- DataBlocks.DataBlock activeBlock = getActiveBlock();
+ AbfsBlock activeBlock = getActiveBlock();
// No data, return.
if (!hasActiveBlockDataToUpload()) {
return;
@@ -573,7 +773,7 @@ private void writeAppendBlobCurrentBufferToService() throws IOException {
try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker,
"writeCurrentBufferToService", "append")) {
AppendRequestParameters reqParams = new AppendRequestParameters(offset, 0,
- bytesLength, APPEND_MODE, true, leaseId);
+ bytesLength, APPEND_MODE, true, leaseId, isExpectHeaderEnabled);
AbfsRestOperation op = client.append(path, uploadData.toByteArray(), reqParams,
cachedSasToken.get(), new TracingContext(tracingContext));
cachedSasToken.update(op.getSasToken());
@@ -590,6 +790,11 @@ private void writeAppendBlobCurrentBufferToService() throws IOException {
}
}
+ /**
+ * Waits for all the ongoing async {@link #writeOperations} to complete.
+ *
+ * @throws IOException exception from failed async write operation
+ */
private synchronized void waitForAppendsToComplete() throws IOException {
for (WriteOperation writeOperation : writeOperations) {
try {
@@ -611,6 +816,14 @@ private synchronized void waitForAppendsToComplete() throws IOException {
}
}
+ /**
+ * Wait for the previous writes to complete. Once the previous writes are
+ * successfully completed, the written data is flushed.
+ *
+ * @param isClose stream is closed after this flush
+ * @throws IOException exception from write failure or server exception on flush
+ * operation
+ */
private synchronized void flushWrittenBytesToService(boolean isClose) throws IOException {
waitForAppendsToComplete();
flushWrittenBytesToServiceInternal(position, false, isClose);
@@ -635,8 +848,69 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset,
AbfsPerfTracker tracker = client.getAbfsPerfTracker();
try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker,
"flushWrittenBytesToServiceInternal", "flush")) {
- AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, isClose,
- cachedSasToken.get(), leaseId, new TracingContext(tracingContext));
+ AbfsRestOperation op;
+ if (!OperativeEndpoint.isIngressEnabledOnDFS(prefixMode, client.getAbfsConfiguration())) {
+ LOG.debug("Flush over Blob for ingress config value {} for path {} ",
+ client.getAbfsConfiguration().shouldIngressFallbackToDfs(), path);
+ // Adds all the committed blocks if available to the list of blocks to be added in putBlockList.
+ blockIdList.addAll(committedBlockEntries);
+ boolean successValue = true;
+ String failedBlockId = "";
+ BlockStatus success = BlockStatus.SUCCESS;
+
+ // No network calls needed for empty map.
+ if (map.isEmpty()) {
+ return;
+ }
+
+ int mapEntry = 0;
+ // If any of the entry in the map doesn't have the status of SUCCESS, fail the flush.
+ for (Map.Entry entry : getMap().entrySet()) {
+ if (!success.equals(entry.getValue())) {
+ successValue = false;
+ failedBlockId = entry.getKey();
+ break;
+ } else {
+ if (!entry.getKey().equals(orderedBlockList.get(mapEntry))) {
+ LOG.debug("The order for the given offset {} with blockId {} and streamId {} " +
+ " for the path {} was not successful", offset, entry.getKey(), outputStreamId, path);
+ throw new IOException("The ordering in map is incorrect for blockId " +
+ entry.getKey() + " and offset " + offset + " for path" + path + " with streamId " + outputStreamId);
+ }
+ blockIdList.add(entry.getKey());
+ mapEntry++;
+ }
+ }
+ if (!successValue) {
+ LOG.debug("A past append for the given offset {} with blockId {} and streamId {}" +
+ " for the path {} was not successful", offset, failedBlockId, outputStreamId, path);
+ throw new IOException("A past append was not successful for blockId " +
+ failedBlockId + " and offset " + offset + " for path" + path + " with streamId " + outputStreamId);
+ }
+ // Generate the xml with the list of blockId's to generate putBlockList call.
+ String blockListXml = generateBlockListXml(blockIdList);
+ TracingContext tracingContextBlobFlush = new TracingContext(tracingContext);
+ tracingContextBlobFlush.setFallbackDFSAppend("B " + position);
+ op = client.flush(blockListXml.getBytes(), path,
+ isClose, cachedSasToken.get(), leaseId, getETag(), tracingContextBlobFlush);
+ setETag(op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG));
+ try {
+ mapLock.lock();
+ getMap().clear();
+ orderedBlockList.clear();
+ } finally {
+ mapLock.unlock();
+ }
+ } else {
+ LOG.debug("Flush over DFS for ingress config value {} for path {} ",
+ client.getAbfsConfiguration().shouldIngressFallbackToDfs(), path);
+ TracingContext tracingContextFlush = new TracingContext(tracingContext);
+ if (fallbackDFSAppend) {
+ tracingContextFlush.setFallbackDFSAppend("D");
+ }
+ op = client.flush(path, offset, retainUncommitedData, isClose,
+ cachedSasToken.get(), leaseId, tracingContextFlush);
+ }
cachedSasToken.update(op.getSasToken());
perfInfo.registerResult(op.getResult()).registerSuccess(true);
} catch (AzureBlobFileSystemException ex) {
@@ -651,6 +925,22 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset,
this.lastFlushOffset = offset;
}
+ /**
+ * Helper method to generate the xml with list of blockId's.
+ * @param blockIds The set of blockId's to be pushed to the backend.
+ * @return xml in string format.
+ */
+ private static String generateBlockListXml(Set blockIds) {
+ StringBuilder stringBuilder = new StringBuilder();
+ stringBuilder.append(XML_VERSION);
+ stringBuilder.append(BLOCK_LIST_START_TAG);
+ for (String blockId : blockIds) {
+ stringBuilder.append(String.format(LATEST_BLOCK_FORMAT, blockId));
+ }
+ stringBuilder.append(BLOCK_LIST_END_TAG);
+ return stringBuilder.toString();
+ }
+
/**
* Try to remove the completed write operations from the beginning of write
* operation FIFO queue.
@@ -750,6 +1040,16 @@ public boolean hasLease() {
return lease != null;
}
+ @VisibleForTesting
+ public Boolean areWriteOperationsTasksDone() {
+ for (WriteOperation writeOperation : writeOperations) {
+ if (!writeOperation.task.isDone()) {
+ return false;
+ }
+ }
+ return true;
+ }
+
/**
* Appending AbfsOutputStream statistics to base toString().
*
@@ -764,4 +1064,9 @@ public String toString() {
sb.append("}");
return sb.toString();
}
+
+ @VisibleForTesting
+ public String getLeaseId() {
+ return leaseId;
+ }
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java
index ad303823e0c68b..94a62abbe99812 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java
@@ -33,6 +33,8 @@ public class AbfsOutputStreamContext extends AbfsStreamContext {
private boolean enableFlush;
+ private boolean enableExpectHeader;
+
private boolean enableSmallWriteOptimization;
private boolean disableOutputStreamFlush;
@@ -63,6 +65,8 @@ public class AbfsOutputStreamContext extends AbfsStreamContext {
private TracingContext tracingContext;
+ private String eTag;
+
public AbfsOutputStreamContext(final long sasTokenRenewPeriodForStreamsInSeconds) {
super(sasTokenRenewPeriodForStreamsInSeconds);
}
@@ -78,6 +82,11 @@ public AbfsOutputStreamContext enableFlush(final boolean enableFlush) {
return this;
}
+ public AbfsOutputStreamContext enableExpectHeader(final boolean enableExpectHeader) {
+ this.enableExpectHeader = enableExpectHeader;
+ return this;
+ }
+
public AbfsOutputStreamContext enableSmallWriteOptimization(final boolean enableSmallWriteOptimization) {
this.enableSmallWriteOptimization = enableSmallWriteOptimization;
return this;
@@ -150,6 +159,12 @@ public AbfsOutputStreamContext withTracingContext(
return this;
}
+ public AbfsOutputStreamContext withETag(
+ final String eTag) {
+ this.eTag = eTag;
+ return this;
+ }
+
public AbfsOutputStreamContext build() {
// Validation of parameters to be done here.
if (streamStatistics == null) {
@@ -184,6 +199,10 @@ public boolean isEnableFlush() {
return enableFlush;
}
+ public boolean isExpectHeaderEnabled() {
+ return enableExpectHeader;
+ }
+
public boolean isDisableOutputStreamFlush() {
return disableOutputStreamFlush;
}
@@ -250,4 +269,8 @@ public ExecutorService getExecutorService() {
public TracingContext getTracingContext() {
return tracingContext;
}
+
+ public String getETag() {
+ return eTag;
+ }
}
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 c2a80f8177094b..5ed8d5c5acdaf4 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
@@ -25,6 +25,7 @@
import java.net.UnknownHostException;
import java.util.List;
+import org.apache.hadoop.classification.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -37,6 +38,9 @@
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.PUT_BLOCK_LIST;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_CONTINUE;
+
/**
* The AbfsRestOperation for Rest AbfsClient.
*/
@@ -45,6 +49,8 @@ public class AbfsRestOperation {
private final AbfsRestOperationType operationType;
// Blob FS client, which has the credentials, retry policy, and logs.
private final AbfsClient client;
+ // Return intercept instance
+ private final AbfsThrottlingIntercept intercept;
// the HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE)
private final String method;
// full URL including query parameters
@@ -71,6 +77,17 @@ public class AbfsRestOperation {
private AbfsHttpOperation result;
private AbfsCounters abfsCounters;
+ /**
+ * This variable contains the reason of last API call within the same
+ * AbfsRestOperation object.
+ */
+ private String failureReason;
+
+ /**
+ * This variable stores the tracing context used for last Rest Operation made
+ */
+ private TracingContext lastTracingContext;
+
/**
* Checks if there is non-null HTTP response.
* @return true if there is a non-null HTTP response from the ABFS call.
@@ -145,6 +162,7 @@ String getSasToken() {
|| AbfsHttpConstants.HTTP_METHOD_PATCH.equals(method));
this.sasToken = sasToken;
this.abfsCounters = client.getAbfsCounters();
+ this.intercept = client.getIntercept();
}
/**
@@ -185,10 +203,14 @@ String getSasToken() {
public void execute(TracingContext tracingContext)
throws AzureBlobFileSystemException {
+ // Since this might be a sub-sequential call triggered by a single
+ // file system call, a new tracing context should be used.
+ final TracingContext newTracingContext = createNewTracingContext(tracingContext);
+
try {
IOStatisticsBinding.trackDurationOfInvocation(abfsCounters,
AbfsStatistic.getStatNameFromHttpCall(method),
- () -> completeExecute(tracingContext));
+ () -> completeExecute(newTracingContext));
} catch (AzureBlobFileSystemException aze) {
throw aze;
} catch (IOException e) {
@@ -202,10 +224,10 @@ public void execute(TracingContext tracingContext)
* HTTP operations.
* @param tracingContext TracingContext instance to track correlation IDs
*/
- private void completeExecute(TracingContext tracingContext)
+ void completeExecute(TracingContext tracingContext)
throws AzureBlobFileSystemException {
// see if we have latency reports from the previous requests
- String latencyHeader = this.client.getAbfsPerfTracker().getClientLatency();
+ String latencyHeader = getClientLatency();
if (latencyHeader != null && !latencyHeader.isEmpty()) {
AbfsHttpHeader httpHeader =
new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_ABFS_CLIENT_LATENCY, latencyHeader);
@@ -226,14 +248,62 @@ private void completeExecute(TracingContext tracingContext)
}
}
- if (result.getStatusCode() >= HttpURLConnection.HTTP_BAD_REQUEST) {
+ int status = result.getStatusCode();
+ /*
+ If even after exhausting all retries, the http status code has an
+ invalid value it qualifies for InvalidAbfsRestOperationException.
+ All http status code less than 1xx range are considered as invalid
+ status codes.
+ */
+ if (status < HTTP_CONTINUE) {
+ throw new InvalidAbfsRestOperationException(null, retryCount);
+ }
+
+ if (status >= HttpURLConnection.HTTP_BAD_REQUEST) {
throw new AbfsRestOperationException(result.getStatusCode(), result.getStorageErrorCode(),
result.getStorageErrorMessage(), null, result);
}
-
LOG.trace("{} REST operation complete", operationType);
}
+ @VisibleForTesting
+ String getClientLatency() {
+ return client.getAbfsPerfTracker().getClientLatency();
+ }
+
+ /**
+ * Sign an operation.
+ * @param httpOperation operation to sign
+ * @param bytesToSign how many bytes to sign for shared key auth.
+ * @throws IOException failure
+ */
+ @VisibleForTesting
+ public void signRequest(final AbfsHttpOperation httpOperation,
+ int bytesToSign) throws IOException {
+ switch (client.getAuthType()) {
+ case Custom:
+ case OAuth:
+ LOG.debug("Authenticating request with OAuth2 access token");
+ httpOperation.getConnection()
+ .setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION,
+ client.getAccessToken());
+ break;
+ case SAS:
+ // do nothing; the SAS token should already be appended to the query string
+ httpOperation.setMaskForSAS(); //mask sig/oid from url for logs
+ break;
+ case SharedKey:
+ default:
+ // sign the HTTP request
+ LOG.debug("Signing request with shared key");
+ // sign the HTTP request
+ client.getSharedKeyCredentials().signRequest(
+ httpOperation.getConnection(),
+ bytesToSign);
+ break;
+ }
+ }
+
/**
* Executes a single HTTP operation to complete the REST operation. If it
* fails, there may be a retry. The retryCount is incremented with each
@@ -241,33 +311,14 @@ private void completeExecute(TracingContext tracingContext)
*/
private boolean executeHttpOperation(final int retryCount,
TracingContext tracingContext) throws AzureBlobFileSystemException {
- AbfsHttpOperation httpOperation = null;
+ AbfsHttpOperation httpOperation;
try {
// initialize the HTTP request and open the connection
- httpOperation = new AbfsHttpOperation(url, method, requestHeaders);
+ httpOperation = createHttpOperation();
incrementCounter(AbfsStatistic.CONNECTIONS_MADE, 1);
- tracingContext.constructHeader(httpOperation);
+ tracingContext.constructHeader(httpOperation, failureReason);
- switch(client.getAuthType()) {
- case Custom:
- case OAuth:
- LOG.debug("Authenticating request with OAuth2 access token");
- httpOperation.getConnection().setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION,
- client.getAccessToken());
- break;
- case SAS:
- // do nothing; the SAS token should already be appended to the query string
- httpOperation.setMaskForSAS(); //mask sig/oid from url for logs
- break;
- case SharedKey:
- // sign the HTTP request
- LOG.debug("Signing request with shared key");
- // sign the HTTP request
- client.getSharedKeyCredentials().signRequest(
- httpOperation.getConnection(),
- hasRequestBody ? bufferLength : 0);
- break;
- }
+ signRequest(httpOperation, hasRequestBody ? bufferLength : 0);
} catch (IOException e) {
LOG.debug("Auth failure: {}, {}", method, url);
throw new AbfsRestOperationException(-1, null,
@@ -278,13 +329,14 @@ private boolean executeHttpOperation(final int retryCount,
// dump the headers
AbfsIoUtils.dumpHeadersToDebugLog("Request Headers",
httpOperation.getConnection().getRequestProperties());
- AbfsClientThrottlingIntercept.sendingRequest(operationType, abfsCounters);
-
+ intercept.sendingRequest(operationType, abfsCounters);
if (hasRequestBody) {
// HttpUrlConnection requires
httpOperation.sendRequest(buffer, bufferOffset, bufferLength);
incrementCounter(AbfsStatistic.SEND_REQUESTS, 1);
- incrementCounter(AbfsStatistic.BYTES_SENT, bufferLength);
+ if (!(operationType.name().equals(PUT_BLOCK_LIST))) {
+ incrementCounter(AbfsStatistic.BYTES_SENT, bufferLength);
+ }
}
httpOperation.processResponse(buffer, bufferOffset, bufferLength);
@@ -300,10 +352,11 @@ private boolean executeHttpOperation(final int retryCount,
} catch (UnknownHostException ex) {
String hostname = null;
hostname = httpOperation.getHost();
- LOG.warn("Unknown host name: %s. Retrying to resolve the host name...",
+ failureReason = RetryReason.getAbbreviation(ex, null, null);
+ LOG.warn("Unknown host name: {}. Retrying to resolve the host name...",
hostname);
if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) {
- throw new InvalidAbfsRestOperationException(ex);
+ throw new InvalidAbfsRestOperationException(ex, retryCount);
}
return false;
} catch (IOException ex) {
@@ -311,18 +364,35 @@ private boolean executeHttpOperation(final int retryCount,
LOG.debug("HttpRequestFailure: {}, {}", httpOperation.toString(), ex);
}
+ failureReason = RetryReason.getAbbreviation(ex, -1, "");
+
if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) {
- throw new InvalidAbfsRestOperationException(ex);
+ throw new InvalidAbfsRestOperationException(ex, retryCount);
}
return false;
} finally {
- AbfsClientThrottlingIntercept.updateMetrics(operationType, httpOperation);
+ int status = httpOperation.getStatusCode();
+ /*
+ A status less than 300 (2xx range) or greater than or equal
+ to 500 (5xx range) should contribute to throttling metrics being updated.
+ Less than 200 or greater than or equal to 500 show failed operations. 2xx
+ range contributes to successful operations. 3xx range is for redirects
+ and 4xx range is for user errors. These should not be a part of
+ throttling backoff computation.
+ */
+ boolean updateMetricsResponseCode = (status < HttpURLConnection.HTTP_MULT_CHOICE
+ || status >= HttpURLConnection.HTTP_INTERNAL_ERROR);
+ if (updateMetricsResponseCode) {
+ intercept.updateMetrics(operationType, httpOperation);
+ }
}
LOG.debug("HttpRequest: {}: {}", operationType, httpOperation.toString());
if (client.getRetryPolicy().shouldRetry(retryCount, httpOperation.getStatusCode())) {
+ int status = httpOperation.getStatusCode();
+ failureReason = RetryReason.getAbbreviation(null, status, httpOperation.getStorageErrorMessage());
return false;
}
@@ -331,6 +401,47 @@ private boolean executeHttpOperation(final int retryCount,
return true;
}
+ @VisibleForTesting
+ String getMethod() {
+ return method;
+ }
+
+ @VisibleForTesting
+ void setResult(AbfsHttpOperation result) {
+ this.result = result;
+ }
+
+ /**
+ * Creates new object of {@link AbfsHttpOperation} with the url, method, and
+ * requestHeaders fields of the AbfsRestOperation object.
+ */
+ @VisibleForTesting
+ AbfsHttpOperation createHttpOperation() throws IOException {
+ return new AbfsHttpOperation(url, method, requestHeaders);
+ }
+
+ /**
+ * Creates a new Tracing context before entering the retry loop of a rest operation
+ * This will ensure all rest operations have unique
+ * tracing context that will be used for all the retries
+ * @param tracingContext
+ * @return tracingContext
+ */
+ @VisibleForTesting
+ final TracingContext createNewTracingContext(final TracingContext tracingContext) {
+ lastTracingContext = new TracingContext(tracingContext);
+ return lastTracingContext;
+ }
+
+ /**
+ * Returns the tracing contest used for last rest operation made
+ * @return
+ */
+ @VisibleForTesting
+ public final TracingContext getLastTracingContext() {
+ return lastTracingContext;
+ }
+
/**
* Incrementing Abfs counters with a long value.
*
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationType.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationType.java
index 830297f381b91d..1bff1e5665081a 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationType.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationType.java
@@ -36,10 +36,27 @@ public enum AbfsRestOperationType {
SetOwner,
SetPathProperties,
SetPermissions,
+ SetBlobMetadata,
+ SetContainerMetadata,
Append,
Flush,
ReadFile,
DeletePath,
CheckAccess,
LeasePath,
+ LeaseBlob,
+ PutBlob,
+ GetBlobProperties,
+ GetContainerProperties,
+ GetBlobMetadata,
+ GetContainerMetadata,
+ PutBlock,
+ PutBlockList,
+ GetBlockList,
+ DeleteBlob,
+ CreateContainer,
+ DeleteContainer,
+ GetListBlobProperties,
+ CopyBlob,
+ GetBlob
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingIntercept.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingIntercept.java
new file mode 100644
index 00000000000000..0ceb4335fcef44
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingIntercept.java
@@ -0,0 +1,49 @@
+/**
+ * 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.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * An interface for Abfs Throttling Interface.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface AbfsThrottlingIntercept {
+
+ /**
+ * Updates the metrics for successful and failed read and write operations.
+ * @param operationType Only applicable for read and write operations.
+ * @param abfsHttpOperation Used for status code and data transferred.
+ */
+ void updateMetrics(AbfsRestOperationType operationType,
+ AbfsHttpOperation abfsHttpOperation);
+
+ /**
+ * Called before the request is sent. Client-side throttling
+ * uses this to suspend the request, if necessary, to minimize errors and
+ * maximize throughput.
+ * @param operationType Only applicable for read and write operations.
+ * @param abfsCounters Used for counters.
+ */
+ void sendingRequest(AbfsRestOperationType operationType,
+ AbfsCounters abfsCounters);
+
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingInterceptFactory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingInterceptFactory.java
new file mode 100644
index 00000000000000..0eabe18872d4f9
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingInterceptFactory.java
@@ -0,0 +1,102 @@
+/**
+ * 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.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+
+import org.apache.hadoop.fs.azurebfs.utils.WeakReferenceMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class to get an instance of throttling intercept class per account.
+ */
+final class AbfsThrottlingInterceptFactory {
+
+ private AbfsThrottlingInterceptFactory() {
+ }
+
+ private static AbfsConfiguration abfsConfig;
+
+ /**
+ * List of references notified of loss.
+ */
+ private static List lostReferences = new ArrayList<>();
+
+ private static final Logger LOG = LoggerFactory.getLogger(
+ AbfsThrottlingInterceptFactory.class);
+
+ /**
+ * Map which stores instance of ThrottlingIntercept class per account.
+ */
+ private static WeakReferenceMap
+ interceptMap = new WeakReferenceMap<>(
+ AbfsThrottlingInterceptFactory::factory,
+ AbfsThrottlingInterceptFactory::referenceLost);
+
+ /**
+ * Returns instance of throttling intercept.
+ * @param accountName Account name.
+ * @return instance of throttling intercept.
+ */
+ private static AbfsClientThrottlingIntercept factory(final String accountName) {
+ return new AbfsClientThrottlingIntercept(accountName, abfsConfig);
+ }
+
+ /**
+ * Reference lost callback.
+ * @param accountName key lost.
+ */
+ private static void referenceLost(String accountName) {
+ lostReferences.add(accountName);
+ }
+
+ /**
+ * Returns an instance of AbfsThrottlingIntercept.
+ *
+ * @param accountName The account for which we need instance of throttling intercept.
+ @param abfsConfiguration The object of abfsconfiguration class.
+ * @return Instance of AbfsThrottlingIntercept.
+ */
+ static synchronized AbfsThrottlingIntercept getInstance(String accountName,
+ AbfsConfiguration abfsConfiguration) {
+ abfsConfig = abfsConfiguration;
+ AbfsThrottlingIntercept intercept;
+ if (!abfsConfiguration.isAutoThrottlingEnabled()) {
+ return AbfsNoOpThrottlingIntercept.INSTANCE;
+ }
+ // If singleton is enabled use a static instance of the intercept class for all accounts
+ if (!abfsConfiguration.accountThrottlingEnabled()) {
+ intercept = AbfsClientThrottlingIntercept.initializeSingleton(
+ abfsConfiguration);
+ } else {
+ // Return the instance from the map
+ intercept = interceptMap.get(accountName);
+ if (intercept == null) {
+ intercept = new AbfsClientThrottlingIntercept(accountName,
+ abfsConfiguration);
+ interceptMap.put(accountName, intercept);
+ }
+ }
+ return intercept;
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobList.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobList.java
new file mode 100644
index 00000000000000..fdac6559507fd6
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobList.java
@@ -0,0 +1,62 @@
+/**
+ * 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.ArrayList;
+import java.util.List;
+
+/**
+ * Contains result of GetBlobList API in processable state.
+ */
+public class BlobList {
+
+ /**
+ * List of {@link BlobProperty} returned by server
+ */
+ private final List blobProperties = new ArrayList<>();
+
+ /**
+ * Since there could be many blobs which can be returned by server on the
+ * GetBlobList API on a path and server wants to return only limited number of
+ * blob-information in one go. The expectation from the server is to use a token
+ * called as nextMarker and call the GetBlobList API again for the
+ * same path.
+ */
+ private String nextMarker;
+
+ void addBlobProperty(final BlobProperty blobProperty) {
+ blobProperties.add(blobProperty);
+ }
+
+ void setNextMarker(String nextMarker) {
+ this.nextMarker = nextMarker;
+ }
+
+ public List getBlobPropertyList() {
+ return blobProperties;
+ }
+
+ public String getNextMarker() {
+ return nextMarker;
+ }
+
+ public BlobList() {
+
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobListXmlParser.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobListXmlParser.java
new file mode 100644
index 00000000000000..d00675201a4cfa
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobListXmlParser.java
@@ -0,0 +1,300 @@
+/**
+ * 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.Stack;
+
+import org.xml.sax.Attributes;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.DefaultHandler;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.utils.DateTimeUtils;
+
+/**
+ * Parses the response inputSteam and populates an object of {@link BlobList}. Parsing
+ * creates a list of {@link BlobProperty}.
+ *
+ * BlobList API XML response example
+ *
+ */
+
+
+public class BlobListXmlParser extends DefaultHandler {
+ /**
+ * Object that contains the parsed response.
+ */
+ private final BlobList blobList;
+ private final String url;
+ /**
+ * {@link BlobProperty} for which at a given moment, the parsing is going on.
+ *
+ * Following XML elements will be parsed and added in currentBlobProperty.
+ * 1. Blob: for explicit files and directories
+ *
+ * blob-name
+ *
+ *
+ * value
+ *
+ *
+ *
+ * 2. BlobPrefix: for directories both explicit and implicit
+ *
+ * blob-prefix
+ *
+ */
+ private BlobProperty currentBlobProperty;
+ /**
+ * Maintains the value in a given XML-element.
+ */
+ private StringBuilder bld = new StringBuilder();
+ /**
+ * Maintains the stack of XML-elements in memory at a given moment.
+ */
+ private final Stack elements = new Stack<>();
+
+ /**
+ * Set an object of {@link BlobList} to populate from the parsing.
+ * Set the url for which GetBlobList API is called.
+ */
+ public BlobListXmlParser(final BlobList blobList, final String url) {
+ this.blobList = blobList;
+ this.url = url;
+ }
+
+ /**
+ *
Receive notification of the start of an element.
+ * If the xml start tag is "Blob", it defines that a new BlobProperty information
+ * is going to be parsed.
+ */
+ @Override
+ public void startElement(final String uri,
+ final String localName,
+ final String qName,
+ final Attributes attributes) throws SAXException {
+ elements.push(localName);
+ if (AbfsHttpConstants.BLOB.equals(localName) || AbfsHttpConstants.BLOB_PREFIX.equals(localName)) {
+ currentBlobProperty = new BlobProperty();
+ }
+ }
+
+ /**
+ *
Receive notification of the end of an element.
+ * Whenever an XML-tag is closed, the parent-tag and current-tag shall be
+ * checked and correct property shall be set in the active {@link #currentBlobProperty}.
+ * If the current-tag is "Blob", it means that there are no more properties to
+ * be set in the the active {@link #currentBlobProperty}, and it shall be the
+ * {@link #blobList}.
+ */
+ @Override
+ public void endElement(final String uri,
+ final String localName,
+ final String qName)
+ throws SAXException {
+ String currentNode = elements.pop();
+
+ // Check if the ending tag is correct to the starting tag in the stack.
+ if (!currentNode.equals(localName)) {
+ throw new SAXException(AbfsHttpConstants.INVALID_XML);
+ }
+
+ String parentNode = "";
+ if (elements.size() > 0) {
+ parentNode = elements.peek();
+ }
+
+ String value = bld.toString();
+ if (value.isEmpty()) {
+ value = null;
+ }
+
+ /*
+ * If the closing tag is Blob, there are no more properties to be set in
+ * currentBlobProperty.
+ */
+ if (AbfsHttpConstants.BLOB.equals(currentNode)) {
+ blobList.addBlobProperty(currentBlobProperty);
+ currentBlobProperty = null;
+ }
+
+ /*
+ * If the closing tag is BlobPrefix, there are no more properties to be set in
+ * currentBlobProperty and this is a directory (implicit or explicit)
+ * If implicit, it will be added only once/
+ * If explicit it will be added with Blob Tag as well.
+ */
+ if (AbfsHttpConstants.BLOB_PREFIX.equals(currentNode)) {
+ currentBlobProperty.setIsDirectory(true);
+ blobList.addBlobProperty(currentBlobProperty);
+ currentBlobProperty = null;
+ }
+
+ /*
+ * If the closing tag is Next Marker, it needs to be saved with the
+ * list of blobs currently fetched
+ */
+ if (AbfsHttpConstants.NEXT_MARKER.equals(currentNode)) {
+ blobList.setNextMarker(value);
+ }
+
+ /*
+ * If the closing tag is Name, then it is either for a blob
+ * or for a blob prefix denoting a directory. We will save this
+ * in current BlobProperty for both
+ */
+ if (currentNode.equals(AbfsHttpConstants.NAME)
+ && (parentNode.equals(AbfsHttpConstants.BLOB)
+ || parentNode.equals(AbfsHttpConstants.BLOB_PREFIX))) {
+ // In case of BlobPrefix Name will have a slash at the end
+ // Remove the "/" at the end of name
+ if (value.endsWith(AbfsHttpConstants.FORWARD_SLASH)) {
+ value = value.substring(0, value.length() - 1);
+ }
+
+ currentBlobProperty.setName(value);
+ currentBlobProperty.setPath(new Path(AbfsHttpConstants.ROOT_PATH + value));
+ currentBlobProperty.setUrl(url + AbfsHttpConstants.ROOT_PATH + value);
+ }
+
+ /*
+ * For case:
+ *
+ * ...
+ *
+ * value
+ * value
+ *
+ * ...
+ *
+ * ParentNode will be Metadata for all key1, key2, ... , keyN.
+ */
+ if (parentNode.equals(AbfsHttpConstants.METADATA)) {
+ currentBlobProperty.addMetadata(currentNode, value);
+ // For Marker blobs hdi_isFolder will be present as metadata
+ if (AbfsHttpConstants.HDI_ISFOLDER.equals(currentNode)) {
+ currentBlobProperty.setIsDirectory(Boolean.valueOf(value));
+ }
+ }
+
+ /*
+ * For case:
+ *
+ * ...
+ *
+ * date-time-value
+ * date-time-value
+ * Etag
+ * owner user id
+ * owning group id
+ * permission string
+ * access control list
+ * file | directory
+ * size-in-bytes
+ * id
+ * pending | success | aborted | failed
+ * source url
+ * bytes copied/bytes total
+ * datetime
+ * error string
+ *
+ * ...
+ *
+ * ParentNode will be Properties for Content-Length, ResourceType.
+ */
+ if (parentNode.equals(AbfsHttpConstants.PROPERTIES)) {
+ if (currentNode.equals(AbfsHttpConstants.CREATION_TIME)) {
+ currentBlobProperty.setCreationTime(DateTimeUtils.parseLastModifiedTime(value));
+ }
+ if (currentNode.equals(AbfsHttpConstants.LAST_MODIFIED_TIME)) {
+ currentBlobProperty.setLastModifiedTime(DateTimeUtils.parseLastModifiedTime(value));
+ }
+ if (currentNode.equals(AbfsHttpConstants.ETAG)) {
+ currentBlobProperty.setETag(value);
+ }
+ if (currentNode.equals(AbfsHttpConstants.OWNER)) {
+ currentBlobProperty.setOwner(value);
+ }
+ if (currentNode.equals(AbfsHttpConstants.GROUP)) {
+ currentBlobProperty.setGroup(value);
+ }
+ if (currentNode.equals(AbfsHttpConstants.PERMISSIONS)) {
+ currentBlobProperty.setPermission(value);
+ }
+ if (currentNode.equals(AbfsHttpConstants.ACL)) {
+ currentBlobProperty.setAcl(value);
+ }
+ if (currentNode.equals(AbfsHttpConstants.RESOURCE_TYPE)) {
+ if (AbfsHttpConstants.DIRECTORY.equals(value)) {
+ currentBlobProperty.setIsDirectory(true);
+ }
+ }
+ if (currentNode.equals(AbfsHttpConstants.CONTENT_LEN)) {
+ currentBlobProperty.setContentLength(Long.parseLong(value));
+ }
+ if (currentNode.equals(AbfsHttpConstants.COPY_ID)) {
+ currentBlobProperty.setCopyId(value);
+ }
+ if (currentNode.equals(AbfsHttpConstants.COPY_STATUS)) {
+ currentBlobProperty.setCopyStatus(value);
+ }
+ if (currentNode.equals(AbfsHttpConstants.COPY_SOURCE)) {
+ currentBlobProperty.setCopySourceUrl(value);
+ }
+ if (currentNode.equals(AbfsHttpConstants.COPY_PROGRESS)) {
+ currentBlobProperty.setCopyProgress(value);
+ }
+ if (currentNode.equals(AbfsHttpConstants.COPY_COMPLETION_TIME)) {
+ currentBlobProperty.setCopyCompletionTime(DateTimeUtils.parseLastModifiedTime(value));
+ }
+ if (currentNode.equals(AbfsHttpConstants.COPY_STATUS_DESCRIPTION)) {
+ currentBlobProperty.setCopyStatusDescription(value);
+ }
+ }
+ /*
+ * refresh bld for the next XML-tag value
+ */
+ bld = new StringBuilder();
+ }
+
+ /**
+ * Receive notification of character data inside an element. No heuristics to
+ * apply. Just append the {@link #bld}.
+ */
+ @Override
+ public void characters(final char[] ch, final int start, final int length)
+ throws SAXException {
+ bld.append(ch, start, length);
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobProperty.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobProperty.java
new file mode 100644
index 00000000000000..ea7351d2120a8e
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlobProperty.java
@@ -0,0 +1,217 @@
+/**
+ * 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.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+
+/**
+ * Encapsulates all the information related to a Blob as fetched
+ * on blob endpoint APIs
+ */
+public class BlobProperty {
+ private String name;
+ private Path path;
+ private String url;
+ private Boolean isDirectory = false;
+ private String eTag;
+ private long lastModifiedTime;
+ private long creationTime;
+ private String owner;
+ private String group;
+ private String permission;
+ private String acl;
+ private Long contentLength = 0L;
+ private String copyId;
+ private String copyStatus;
+ private String copySourceUrl;
+ private String copyProgress;
+ private String copyStatusDescription;
+ private long copyCompletionTime;
+ private Map metadata = new HashMap<>();
+
+ private AzureBlobFileSystemException ex;
+
+ public BlobProperty() {
+
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public Path getPath() {
+ return path;
+ }
+
+ public String getUrl() {
+ return url;
+ }
+
+ public Boolean getIsDirectory() {
+ return isDirectory;
+ }
+
+ public String getETag() {
+ return eTag;
+ }
+
+ public long getLastModifiedTime() {
+ return lastModifiedTime;
+ }
+
+ public long getCreationTime() {
+ return creationTime;
+ }
+
+ public String getOwner() {
+ return owner;
+ }
+
+ public String getGroup() {
+ return group;
+ }
+
+ public String getPermission() {
+ return permission;
+ }
+
+ public String getAcl() {
+ return acl;
+ }
+
+ public Long getContentLength() {
+ return contentLength;
+ }
+
+ public String getCopyId() {
+ return copyId;
+ }
+
+ public String getCopyStatus() {
+ return copyStatus;
+ }
+
+ public String getCopySourceUrl() {
+ return copySourceUrl;
+ }
+
+ public String getCopyProgress() {
+ return copyProgress;
+ }
+
+ public String getCopyStatusDescription() {
+ return copyStatusDescription;
+ }
+
+ public long getCopyCompletionTime() {
+ return copyCompletionTime;
+ }
+
+ public Map getMetadata() {
+ return metadata;
+ }
+
+ public AzureBlobFileSystemException getFailureException() {
+ return ex;
+ }
+
+ public Path getBlobDstPath(Path dstBlobPath) {
+ return null;
+ }
+
+ public void setName(final String name) {
+ this.name = name;
+ }
+
+ public void setPath(final Path path) {
+ this.path = path;
+ }
+
+ public void setUrl(final String url) {
+ this.url = url;
+ }
+
+ public void setIsDirectory(final Boolean isDirectory) {
+ this.isDirectory = isDirectory;
+ }
+
+ public void setETag(final String eTag) {
+ this.eTag = eTag;
+ }
+
+ public void setLastModifiedTime(final long lastModifiedTime) {
+ this.lastModifiedTime = lastModifiedTime;
+ }
+
+ public void setCreationTime(final long creationTime) {
+ this.creationTime = creationTime;
+ }
+
+ public void setOwner(final String owner) {
+ this.owner = owner;
+ }
+
+ public void setGroup(final String group) {
+ this.group = group;
+ }
+
+ public void setPermission(final String permission) {
+ this.permission = permission;
+ }
+
+ public void setAcl(final String acl) {
+ this.acl = acl;
+ }
+
+ public void setContentLength(final Long contentLength) {
+ this.contentLength = contentLength;
+ }
+
+ public void setCopyId(final String copyId) {
+ this.copyId = copyId;
+ }
+
+ public void setCopyStatus(final String copyStatus) {
+ this.copyStatus = copyStatus;
+ }
+
+ public void setCopySourceUrl(final String copySourceUrl) {
+ this.copySourceUrl = copySourceUrl;
+ }
+
+ public void setCopyProgress(final String copyProgress) {
+ this.copyProgress = copyProgress;
+ }
+
+ public void setCopyStatusDescription(final String copyStatusDescription) {
+ this.copyStatusDescription = copyStatusDescription;
+ }
+
+ public void setCopyCompletionTime(final long copyCompletionTime) {
+ this.copyCompletionTime = copyCompletionTime;
+ }
+
+ public void addMetadata(String key, String value) {
+ metadata.put(key, value);
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlockStatus.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlockStatus.java
new file mode 100644
index 00000000000000..387f09f4ecc1ac
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/BlockStatus.java
@@ -0,0 +1,25 @@
+/**
+ * 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
+ *
+ * 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;
+
+public enum BlockStatus {
+ NEW,
+ SUCCESS,
+ FAILED
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java
index 89d99471a82145..dee1d374d4a046 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java
@@ -24,6 +24,8 @@
import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_CONTINUE;
+
/**
* Retry policy used by AbfsClient.
* */
@@ -56,6 +58,13 @@ public class ExponentialRetryPolicy {
*/
private static final double MAX_RANDOM_RATIO = 1.2;
+ /**
+ * Qualifies for retry based on
+ * https://learn.microsoft.com/en-us/azure/active-directory/
+ * managed-identities-azure-resources/how-to-use-vm-token#error-handling
+ */
+ private static final int HTTP_TOO_MANY_REQUESTS = 429;
+
/**
* Holds the random number generator used to calculate randomized backoff intervals
*/
@@ -118,7 +127,12 @@ public ExponentialRetryPolicy(final int retryCount, final int minBackoff, final
/**
* Returns if a request should be retried based on the retry count, current response,
- * and the current strategy.
+ * and the current strategy. The valid http status code lies in the range of 1xx-5xx.
+ * But an invalid status code might be set due to network or timeout kind of issues.
+ * Such invalid status code also qualify for retry.
+ * HTTP status code 410 qualifies for retry based on
+ * https://docs.microsoft.com/en-in/azure/virtual-machines/linux/
+ * instance-metadata-service?tabs=windows#errors-and-debugging
*
* @param retryCount The current retry attempt count.
* @param statusCode The status code of the response, or -1 for socket error.
@@ -126,8 +140,10 @@ public ExponentialRetryPolicy(final int retryCount, final int minBackoff, final
*/
public boolean shouldRetry(final int retryCount, final int statusCode) {
return retryCount < this.retryCount
- && (statusCode == -1
+ && (statusCode < HTTP_CONTINUE
|| statusCode == HttpURLConnection.HTTP_CLIENT_TIMEOUT
+ || statusCode == HttpURLConnection.HTTP_GONE
+ || statusCode == HTTP_TOO_MANY_REQUESTS
|| (statusCode >= HttpURLConnection.HTTP_INTERNAL_ERROR
&& statusCode != HttpURLConnection.HTTP_NOT_IMPLEMENTED
&& statusCode != HttpURLConnection.HTTP_VERSION));
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobConsumer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobConsumer.java
new file mode 100644
index 00000000000000..bf6981505e4ead
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobConsumer.java
@@ -0,0 +1,51 @@
+/**
+ * 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.List;
+
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+
+public class ListBlobConsumer {
+
+ private final ListBlobQueue listBlobQueue;
+
+ public ListBlobConsumer(final ListBlobQueue listBlobQueue) {
+ this.listBlobQueue = listBlobQueue;
+ }
+
+ public List consume() throws AzureBlobFileSystemException {
+ if (listBlobQueue.getException() != null) {
+ throw listBlobQueue.getException();
+ }
+ return listBlobQueue.dequeue();
+ }
+
+ public Boolean isCompleted() {
+ return listBlobQueue.getIsCompleted()
+ && listBlobQueue.size() == 0;
+ }
+
+ /**
+ * Register consumer failure.
+ */
+ public void fail() {
+ listBlobQueue.consumptionFailed();
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobProducer.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobProducer.java
new file mode 100644
index 00000000000000..a198dc706c84a0
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobProducer.java
@@ -0,0 +1,111 @@
+/**
+ * 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.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+
+/**
+ * ListBlob API can give maximum of 5000 blobs. If there are (~n*5000) blobs, the
+ * client would need to call the listBlob API n times. This would have two consequences:
+ *
+ *
+ * The consumer of the result of lists of blob would have to wait until all
+ * the blobs are received. The consumer could have used the time to start
+ * processing the blobs already in memory. The wait for receiving all the blobs
+ * would lead the processing more time. Lets say consumer need m time-units to process
+ * one blob. Lets assume that each set of blobs have x blobs. In total there
+ * have to be n sets Lets say that client needs t time to get all the blobs. If consumer
+ * wait for all the blobs to be received, the total time taken would be:
+ *
t + (n * x * m)
+ * Now, lets assume that consumer in parallel work on the available the blobs,
+ * time taken would be:
+ *
t + (((n * x) - t/m) * m)
+ *
+ *
+ * Since the information of the blobs have to be maintained in memory until the
+ * computation on the list is done. On low configuration machine, it may lead
+ * to OOM.
+ *
+ *
+ * In this design, the producer on a parallel thread to the main thread, will call
+ * ListBlob API and will populate {@link ListBlobQueue}, which would be dequeued by
+ * the main thread which will run the computation on the available blobs.
+ *
+ * How its different from {@link AbfsListStatusRemoteIterator}?
+ * It provides an iterator which on {@link AbfsListStatusRemoteIterator#hasNext()} checks
+ * if there are blobs available in memory. If not it will call Listing API on server for
+ * the next set of blobs. But here, it make the process sequential. As in, when the processing
+ * on whole set of blobs available in memory are done, after that only next set of blobs are
+ * fetched.
+ */
+public class ListBlobProducer {
+
+ private final AbfsClient client;
+
+ private final ListBlobQueue listBlobQueue;
+
+ private final String src;
+
+ private final TracingContext tracingContext;
+
+ private String nextMarker;
+ private final Thread thread;
+
+ public ListBlobProducer(final String src,
+ final AbfsClient abfsClient,
+ final ListBlobQueue listBlobQueue,
+ final String initNextMarker,
+ TracingContext tracingContext) {
+ this.src = src;
+ this.client = abfsClient;
+ this.tracingContext = tracingContext;
+ this.listBlobQueue = listBlobQueue;
+ listBlobQueue.setProducer(this);
+ this.nextMarker = initNextMarker;
+ thread = new Thread(() -> {
+ do {
+ int maxResult = listBlobQueue.availableSize();
+ if (maxResult == 0) {
+ continue;
+ }
+ AbfsRestOperation op = null;
+ try {
+ op = client.getListBlobs(nextMarker, src, null, maxResult, tracingContext);
+ } catch (AzureBlobFileSystemException ex) {
+ listBlobQueue.setFailed(ex);
+ return;
+ }
+ BlobList blobList = op.getResult().getBlobList();
+ nextMarker = blobList.getNextMarker();
+ listBlobQueue.enqueue(blobList.getBlobPropertyList());
+ if (nextMarker == null) {
+ listBlobQueue.complete();
+ }
+ } while(nextMarker != null && !listBlobQueue.getConsumptionFailed());
+ });
+ thread.start();
+ }
+
+ @VisibleForTesting
+ public void waitForProcessCompletion() throws InterruptedException {
+ thread.join();
+ }
+}
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 00000000000000..3de0c85ec50006
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ListBlobQueue.java
@@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Queue;
+
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+
+public class ListBlobQueue {
+
+ private final Queue blobLists;
+
+ private int totalProduced = 0;
+
+ private int totalConsumed = 0;
+
+ private Boolean isCompleted = false;
+ private Boolean isConsumptionFailed = false;
+
+ private AzureBlobFileSystemException failureFromProducer;
+
+ /**
+ * Since, Producer just spawns a thread and there are no public method for the
+ * class. Keeping its address in this object will prevent accidental GC close
+ * on the producer object.
+ */
+ private ListBlobProducer producer;
+
+ private final int maxSize;
+ private final int maxConsumedBlobCount;
+
+ /**
+ * @param maxSize maxSize of the queue.
+ * @param maxConsumedBlobCount maximum number of blobs that would be returned
+ * by {@link #dequeue()} method.
+ */
+ public ListBlobQueue(int maxSize, int maxConsumedBlobCount) {
+ blobLists = new ArrayDeque<>(maxSize);
+ this.maxSize = maxSize;
+ this.maxConsumedBlobCount = maxConsumedBlobCount;
+ }
+
+ /**
+ * @param initBlobList list of blobProperties to be enqueued in th queue
+ * @param maxSize maxSize of the queue.
+ * @param maxConsumedBlobCount maximum number of blobs that would be returned
+ * by {@link #dequeue()} method.
+ */
+ public ListBlobQueue(List initBlobList, int maxSize, int maxConsumedBlobCount) {
+ this(maxSize, maxConsumedBlobCount);
+ if (initBlobList != null) {
+ enqueue(initBlobList);
+ }
+ }
+
+ void setProducer(ListBlobProducer producer) {
+ if (this.producer == null) {
+ this.producer = producer;
+ }
+ }
+
+ void setFailed(AzureBlobFileSystemException failure) {
+ failureFromProducer = failure;
+ }
+
+ public void complete() {
+ isCompleted = true;
+ }
+
+ void consumptionFailed() {
+ isConsumptionFailed = true;
+ }
+
+ Boolean getConsumptionFailed() {
+ return isConsumptionFailed;
+ }
+
+ public Boolean getIsCompleted() {
+ return isCompleted;
+ }
+
+ AzureBlobFileSystemException getException() {
+ return failureFromProducer;
+ }
+
+ public void enqueue(List blobProperties) {
+ blobLists.addAll(blobProperties);
+ }
+
+ public List dequeue() {
+ List blobProperties = new ArrayList<>();
+ int counter = 0;
+ while (counter < maxConsumedBlobCount && blobLists.size() > 0) {
+ blobProperties.add(blobLists.poll());
+ counter++;
+ }
+ return blobProperties;
+ }
+
+ public int size() {
+ return blobLists.size();
+ }
+
+ public int availableSize() {
+ return maxSize - blobLists.size();
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/OperativeEndpoint.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/OperativeEndpoint.java
new file mode 100644
index 00000000000000..93667eb77df84b
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/OperativeEndpoint.java
@@ -0,0 +1,48 @@
+/**
+ * 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
+ *
+ * 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.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+
+/**
+ * This class is mainly to unify the fallback for all API's to DFS endpoint at a single spot.
+ */
+public class OperativeEndpoint {
+ public static boolean isMkdirEnabledOnDFS(AbfsConfiguration abfsConfiguration) {
+ if (abfsConfiguration.getPrefixMode() == PrefixMode.BLOB) {
+ return abfsConfiguration.shouldMkdirFallbackToDfs();
+ } else {
+ return true;
+ }
+ }
+
+ public static boolean isIngressEnabledOnDFS(PrefixMode prefixMode, AbfsConfiguration abfsConfiguration) {
+ if (prefixMode == PrefixMode.BLOB) {
+ return abfsConfiguration.shouldIngressFallbackToDfs();
+ } else {
+ return true;
+ }
+ }
+
+ public static boolean isReadEnabledOnDFS(AbfsConfiguration abfsConfiguration) {
+ if (abfsConfiguration.getPrefixMode() == PrefixMode.BLOB) {
+ return abfsConfiguration.shouldReadFallbackToDfs();
+ }
+ return true;
+ }
+}
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 00000000000000..e4f2790565fbff
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/PathInformation.java
@@ -0,0 +1,37 @@
+/**
+ * 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;
+
+public class PathInformation {
+ private Boolean pathExists;
+ private Boolean isDirectory;
+
+ public PathInformation(Boolean pathExists, Boolean isDirectory) {
+ this.pathExists = pathExists;
+ this.isDirectory = isDirectory;
+ }
+
+ public Boolean getPathExists() {
+ return pathExists;
+ }
+
+ public Boolean getIsDirectory() {
+ return isDirectory;
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/PrefixMode.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/PrefixMode.java
new file mode 100644
index 00000000000000..853a22f4f21c9e
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/PrefixMode.java
@@ -0,0 +1,25 @@
+/**
+ * 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;
+
+public enum PrefixMode {
+ DFS,
+ BLOB
+}
+
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
index 4eefb9fdf2c7e3..0f91afe0982dbf 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
@@ -101,6 +101,7 @@ private void init() {
// hide instance constructor
private ReadBufferManager() {
+ LOGGER.trace("Creating readbuffer manager with HADOOP-18546 patch");
}
@@ -544,7 +545,6 @@ public synchronized void purgeBuffersForStream(AbfsInputStream stream) {
LOGGER.debug("Purging stale buffers for AbfsInputStream {} ", stream);
readAheadQueue.removeIf(readBuffer -> readBuffer.getStream() == stream);
purgeList(stream, completedReadList);
- purgeList(stream, inProgressList);
}
/**
@@ -642,4 +642,9 @@ void testMimicFullUseAndAddFailedBuffer(ReadBuffer buf) {
freeList.clear();
completedReadList.add(buf);
}
+
+ @VisibleForTesting
+ int getNumBuffers() {
+ return NUM_BUFFERS;
+ }
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicityUtils.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicityUtils.java
new file mode 100644
index 00000000000000..7490d3c090a4b6
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameAtomicityUtils.java
@@ -0,0 +1,399 @@
+/**
+ * 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.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.TimeZone;
+
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+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;
+
+/**
+ * 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 are given. This file is created by {@link #preRename(Boolean, String)} ()} 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 rename
+ * would be resumed as per the file.
+ */
+public class RenameAtomicityUtils {
+
+ private static final Logger LOG = LoggerFactory.getLogger(
+ RenameAtomicityUtils.class);
+
+ private final AzureBlobFileSystem azureBlobFileSystem;
+ private Path srcPath;
+ private Path dstPath;
+ private TracingContext tracingContext;
+ private Boolean isReDone;
+
+ private static final int MAX_RENAME_PENDING_FILE_SIZE = 10000000;
+ private static final int FORMATTING_BUFFER = 10000;
+
+ public static final String SUFFIX = "-RenamePending.json";
+
+ private static final ObjectReader READER = new ObjectMapper()
+ .configure(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES, true)
+ .readerFor(JsonNode.class);
+
+ public RenameAtomicityUtils(final AzureBlobFileSystem azureBlobFileSystem,
+ final Path srcPath,
+ final Path dstPath,
+ final TracingContext tracingContext) throws IOException {
+ this.azureBlobFileSystem = azureBlobFileSystem;
+ this.srcPath = srcPath;
+ this.dstPath = dstPath;
+ this.tracingContext = tracingContext;
+ }
+
+ public RenameAtomicityUtils(final AzureBlobFileSystem azureBlobFileSystem,
+ final Path renamePendingJsonPath,
+ final RedoRenameInvocation redoRenameInvocation,
+ final String srcEtag,
+ final AbfsInputStream renamePendingJsonInputStream)
+ throws IOException {
+ this.azureBlobFileSystem = azureBlobFileSystem;
+ final RenamePendingFileInfo renamePendingFileInfo = readFile(
+ renamePendingJsonPath, renamePendingJsonInputStream);
+ if (renamePendingFileInfo != null
+ && renamePendingFileInfo.eTag.equalsIgnoreCase(srcEtag)) {
+ redoRenameInvocation.redo(renamePendingFileInfo.destination,
+ renamePendingFileInfo.src);
+ isReDone = true;
+ } else {
+ isReDone = false;
+ }
+ }
+
+ private RenamePendingFileInfo readFile(final Path redoFile,
+ final AbfsInputStream redoFileInputStream)
+ throws IOException {
+ Path f = redoFile;
+ byte[] bytes = new byte[MAX_RENAME_PENDING_FILE_SIZE];
+ int l = redoFileInputStream.read(bytes);
+ if (l <= 0) {
+ // Jira HADOOP-12678 -Handle empty rename pending metadata file during
+ // atomic rename in redo path. If during renamepending file is created
+ // but not written yet, then this means that rename operation
+ // has not started yet. So we should delete rename pending metadata file.
+ LOG.error("Deleting empty rename pending file "
+ + redoFile + " -- no data available");
+ deleteRenamePendingFile(azureBlobFileSystem, redoFile);
+ return null;
+ }
+ if (l == MAX_RENAME_PENDING_FILE_SIZE) {
+ throw new IOException(
+ "Error reading pending rename file contents -- "
+ + "maximum file size exceeded");
+ }
+ String contents = new String(bytes, 0, l, StandardCharsets.UTF_8);
+
+ // parse the JSON
+ JsonNode json = null;
+ boolean committed;
+ try {
+ json = READER.readValue(contents);
+ committed = true;
+ } catch (JsonMappingException e) {
+
+ // The -RedoPending.json file is corrupted, so we assume it was
+ // not completely written
+ // and the redo operation did not commit.
+ committed = false;
+ } catch (JsonParseException e) {
+ committed = false;
+ }
+
+ if (!committed) {
+ LOG.error("Deleting corruped rename pending file {} \n {}",
+ redoFile, contents);
+
+ // delete the -RenamePending.json file
+ deleteRenamePendingFile(azureBlobFileSystem, redoFile);
+ return null;
+ }
+
+ // initialize this object's fields
+ JsonNode oldFolderName = json.get("OldFolderName");
+ JsonNode newFolderName = json.get("NewFolderName");
+ JsonNode eTag = json.get("ETag");
+
+ if (oldFolderName != null && StringUtils.isNotEmpty(
+ oldFolderName.textValue())
+ && newFolderName != null && StringUtils.isNotEmpty(
+ newFolderName.textValue()) && eTag != null && StringUtils.isNotEmpty(
+ eTag.textValue())) {
+ RenamePendingFileInfo renamePendingFileInfo = new RenamePendingFileInfo();
+ renamePendingFileInfo.destination = new Path(newFolderName.textValue());
+ renamePendingFileInfo.src = new Path(oldFolderName.textValue());
+ renamePendingFileInfo.eTag = eTag.textValue();
+ return renamePendingFileInfo;
+ }
+ return null;
+ }
+
+ private void deleteRenamePendingFile(FileSystem fs, Path redoFile)
+ throws IOException {
+ try {
+ fs.delete(redoFile, false);
+ } catch (IOException e) {
+ // If the rename metadata was not found then somebody probably
+ // raced with us and finished the delete first
+ if (e instanceof FileNotFoundException) {
+ LOG.warn("rename pending file " + redoFile + " is already deleted");
+ } else {
+ throw e;
+ }
+ }
+ }
+
+ /**
+ * Write to disk the information needed to redo folder rename,
+ * in JSON format. The file name will be
+ * {@code abfs:///folderName-RenamePending.json}
+ * The file format will be:
+ *
+ * @throws IOException Thrown when fail to write file.
+ */
+ public void preRename(final Boolean isCreateOperationOnBlobEndpoint,
+ final String eTag) throws IOException {
+ Path path = getRenamePendingFilePath();
+ LOG.debug("Preparing to write atomic rename state to {}", path.toString());
+ OutputStream output = null;
+
+ String contents = makeRenamePendingFileContents(eTag);
+
+ // Write file.
+ try {
+ output = azureBlobFileSystem.create(path, false);
+ output.write(contents.getBytes(Charset.forName("UTF-8")));
+ output.flush();
+ output.close();
+ } catch (IOException e) {
+ /*
+ * Scenario: file has been deleted by parallel thread before the RenameJSON
+ * could be written and flushed.
+ * 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 ((!isCreateOperationOnBlobEndpoint
+ && e instanceof FileNotFoundException) || (
+ isCreateOperationOnBlobEndpoint && getWrappedException(
+ e) instanceof AbfsRestOperationException &&
+ ((AbfsRestOperationException) getWrappedException(
+ e)).getStatusCode()
+ == HttpURLConnection.HTTP_PRECON_FAILED)) {
+ /*
+ * In case listStatus done on directory before any content could be written,
+ * that particular thread running on some worker-node of the cluster would
+ * delete the RenamePending JSON file.
+ * ref: https://issues.apache.org/jira/browse/HADOOP-12678.
+ * To recover from parallel delete, we will give it a second try.
+ */
+ output = azureBlobFileSystem.create(path, false);
+ output.write(contents.getBytes(Charset.forName("UTF-8")));
+ output.flush();
+ output.close();
+ return;
+ }
+ throw new IOException(
+ "Unable to write RenamePending file for folder rename from "
+ + srcPath.toUri().getPath() + " to " + dstPath.toUri().getPath(),
+ e);
+ }
+ }
+
+ private Throwable getWrappedException(final IOException e) {
+ if (e.getCause() != null) {
+ return e.getCause().getCause();
+ }
+ return 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) {
+ SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
+ sdf.setTimeZone(TimeZone.getTimeZone("UTC"));
+ String time = sdf.format(new Date());
+ if(!eTag.startsWith("\"") && !eTag.endsWith("\"")) {
+ eTag = quote(eTag);
+ }
+
+ // Make file contents as a string. Again, quote file names, escaping
+ // characters as appropriate.
+ String contents = "{\n"
+ + " FormatVersion: \"1.0\",\n"
+ + " OperationUTCTime: \"" + time + "\",\n"
+ + " OldFolderName: " + quote(srcPath.toUri().getPath()) + ",\n"
+ + " NewFolderName: " + quote(dstPath.toUri().getPath()) + ",\n"
+ + " ETag: " + eTag + "\n"
+ + "}\n";
+
+ return contents;
+ }
+
+ /**
+ * This is an exact copy of org.codehaus.jettison.json.JSONObject.quote
+ * method.
+ *
+ * Produce a string in double quotes with backslash sequences in all the
+ * right places. A backslash will be inserted within , allowing JSON
+ * text to be delivered in HTML. In JSON text, a string cannot contain a
+ * control character or an unescaped quote or backslash.
+ * @param string A String
+ * @return A String correctly formatted for insertion in a JSON text.
+ */
+ private String quote(String string) {
+ if (string == null || string.length() == 0) {
+ return "\"\"";
+ }
+
+ char c = 0;
+ int i;
+ int len = string.length();
+ StringBuilder sb = new StringBuilder(len + 4);
+ String t;
+
+ sb.append('"');
+ for (i = 0; i < len; i += 1) {
+ c = string.charAt(i);
+ switch (c) {
+ case '\\':
+ case '"':
+ sb.append('\\');
+ sb.append(c);
+ break;
+ case '/':
+ sb.append('\\');
+ sb.append(c);
+ break;
+ case '\b':
+ sb.append("\\b");
+ break;
+ case '\t':
+ sb.append("\\t");
+ break;
+ case '\n':
+ sb.append("\\n");
+ break;
+ case '\f':
+ sb.append("\\f");
+ break;
+ case '\r':
+ sb.append("\\r");
+ break;
+ default:
+ if (c < ' ') {
+ t = "000" + Integer.toHexString(c);
+ sb.append("\\u" + t.substring(t.length() - 4));
+ } else {
+ sb.append(c);
+ }
+ }
+ }
+ sb.append('"');
+ return sb.toString();
+ }
+
+ /** Clean up after execution of rename.
+ * @throws IOException Thrown when fail to clean up.
+ * */
+ public void cleanup() throws IOException {
+
+ // Remove RenamePending file
+ azureBlobFileSystem.delete(getRenamePendingFilePath(), false);
+
+ // Freeing source folder lease is not necessary since the source
+ // folder file was deleted.
+ }
+
+ public void cleanup(Path redoFile) throws IOException {
+ azureBlobFileSystem.delete(redoFile, false);
+ }
+
+ private Path getRenamePendingFilePath() {
+ String fileName = srcPath.toUri().getPath() + SUFFIX;
+ Path fileNamePath = new Path(fileName);
+ return fileNamePath;
+ }
+
+ private static class RenamePendingFileInfo {
+ public Path destination;
+ public Path src;
+ public String eTag;
+ }
+
+ public static interface RedoRenameInvocation {
+ void redo(Path destination, Path src) throws
+ AzureBlobFileSystemException;
+ }
+
+ public Boolean isRedone() {
+ return isReDone;
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameNonAtomicUtils.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameNonAtomicUtils.java
new file mode 100644
index 00000000000000..3a106d46fb67c1
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RenameNonAtomicUtils.java
@@ -0,0 +1,52 @@
+/**
+ * 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 org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+
+public class RenameNonAtomicUtils extends RenameAtomicityUtils {
+
+ public RenameNonAtomicUtils(final AzureBlobFileSystem azureBlobFileSystem,
+ final Path srcPath,
+ final Path dstPath,
+ final TracingContext tracingContext) throws IOException {
+ super(azureBlobFileSystem, srcPath, dstPath, tracingContext);
+ }
+
+ @Override
+ public void preRename(final Boolean isCreateOperationOnBlobEndpoint,
+ final String eTag)
+ throws IOException {
+
+ }
+
+ @Override
+ public void cleanup() throws IOException {
+
+ }
+
+ @Override
+ public Boolean isRedone() {
+ return true;
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RetryReason.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RetryReason.java
new file mode 100644
index 00000000000000..40e8cdc1e07bab
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RetryReason.java
@@ -0,0 +1,102 @@
+/**
+ * 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
+ *
+ * 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.LinkedList;
+import java.util.List;
+
+import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.ClientErrorRetryReason;
+import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.ConnectionResetRetryReason;
+import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.ConnectionTimeoutRetryReason;
+import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.ReadTimeoutRetryReason;
+import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.RetryReasonCategory;
+import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.ServerErrorRetryReason;
+import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.UnknownHostRetryReason;
+import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.UnknownIOExceptionRetryReason;
+import org.apache.hadoop.fs.azurebfs.services.retryReasonCategories.UnknownSocketExceptionRetryReason;
+
+
+/**
+ * This utility class exposes methods to convert a server response-error to a
+ * category of error.
+ */
+final class RetryReason {
+
+ /**
+ * Linked-list of the implementations of RetryReasonCategory. The objects in the
+ * list are arranged by the rank of their significance.
+ *
+ *
ServerError (statusCode==5XX), ClientError (statusCode==4XX) are
+ * independent of other retryReason categories.
+ *
Since {@link java.net.SocketException} is subclass of
+ * {@link java.io.IOException},
+ * hence, {@link UnknownIOExceptionRetryReason} is placed before
+ * {@link UnknownSocketExceptionRetryReason}
+ *
Since, connectionTimeout, readTimeout, and connectionReset are
+ * {@link java.net.SocketTimeoutException} exceptions with different messages,
+ * hence, {@link ConnectionTimeoutRetryReason}, {@link ReadTimeoutRetryReason},
+ * {@link ConnectionResetRetryReason} are above {@link UnknownIOExceptionRetryReason}.
+ * There is no order between the three reasons as they are differentiated
+ * by exception-message.
+ *
Since, {@link java.net.UnknownHostException} is subclass of
+ * {@link java.io.IOException}, {@link UnknownHostRetryReason} is placed
+ * over {@link UnknownIOExceptionRetryReason}
+ *
+ */
+ private static List rankedReasonCategories
+ = new LinkedList() {{
+ add(new ServerErrorRetryReason());
+ add(new ClientErrorRetryReason());
+ add(new UnknownIOExceptionRetryReason());
+ add(new UnknownSocketExceptionRetryReason());
+ add(new ConnectionTimeoutRetryReason());
+ add(new ReadTimeoutRetryReason());
+ add(new UnknownHostRetryReason());
+ add(new ConnectionResetRetryReason());
+ }};
+
+ private RetryReason() {
+
+ }
+
+ /**
+ * Method to get correct abbreviation for a given set of exception, statusCode,
+ * storageStatusCode.
+ *
+ * @param ex exception caught during server communication.
+ * @param statusCode statusCode in the server response.
+ * @param storageErrorMessage storageErrorMessage in the server response.
+ *
+ * @return abbreviation for the the given set of exception, statusCode, storageStatusCode.
+ */
+ static String getAbbreviation(Exception ex,
+ Integer statusCode,
+ String storageErrorMessage) {
+ String result = null;
+ for (RetryReasonCategory retryReasonCategory : rankedReasonCategories) {
+ final String abbreviation
+ = retryReasonCategory.captureAndGetAbbreviation(ex,
+ statusCode, storageErrorMessage);
+ if (abbreviation != null) {
+ result = abbreviation;
+ }
+ }
+ return result;
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RetryReasonConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RetryReasonConstants.java
new file mode 100644
index 00000000000000..8a0af183e30aea
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/RetryReasonConstants.java
@@ -0,0 +1,39 @@
+/**
+ * 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
+ *
+ * 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;
+
+public final class RetryReasonConstants {
+
+ private RetryReasonConstants() {
+
+ }
+ public static final String CONNECTION_TIMEOUT_JDK_MESSAGE = "connect timed out";
+ public static final String READ_TIMEOUT_JDK_MESSAGE = "Read timed out";
+ public static final String CONNECTION_RESET_MESSAGE = "Connection reset";
+ public static final String OPERATION_BREACH_MESSAGE = "Operations per second is over the account limit.";
+ public static final String CONNECTION_RESET_ABBREVIATION = "CR";
+ public static final String CONNECTION_TIMEOUT_ABBREVIATION = "CT";
+ public static final String READ_TIMEOUT_ABBREVIATION = "RT";
+ public static final String INGRESS_LIMIT_BREACH_ABBREVIATION = "ING";
+ public static final String EGRESS_LIMIT_BREACH_ABBREVIATION = "EGR";
+ public static final String OPERATION_LIMIT_BREACH_ABBREVIATION = "OPR";
+ public static final String UNKNOWN_HOST_EXCEPTION_ABBREVIATION = "UH";
+ public static final String IO_EXCEPTION_ABBREVIATION = "IOE";
+ public static final String SOCKET_EXCEPTION_ABBREVIATION = "SE";
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TimerFunctionality.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TimerFunctionality.java
new file mode 100644
index 00000000000000..52428fdd54a19f
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/TimerFunctionality.java
@@ -0,0 +1,25 @@
+/**
+ * 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;
+
+public enum TimerFunctionality {
+ RESUME,
+
+ SUSPEND
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ClientErrorRetryReason.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ClientErrorRetryReason.java
new file mode 100644
index 00000000000000..cf1c47e3eb0dc5
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ClientErrorRetryReason.java
@@ -0,0 +1,43 @@
+/**
+ * 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
+ *
+ * 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.retryReasonCategories;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_STATUS_CATEGORY_QUOTIENT;
+
+/**
+ * Category that can capture server-response errors for 4XX status-code.
+ */
+public class ClientErrorRetryReason extends RetryReasonCategory {
+
+ @Override
+ Boolean canCapture(final Exception ex,
+ final Integer statusCode,
+ final String serverErrorMessage) {
+ if (statusCode == null || statusCode / HTTP_STATUS_CATEGORY_QUOTIENT != 4) {
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ String getAbbreviation(final Integer statusCode,
+ final String serverErrorMessage) {
+ return statusCode + "";
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ConnectionResetRetryReason.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ConnectionResetRetryReason.java
new file mode 100644
index 00000000000000..702f8875646324
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ConnectionResetRetryReason.java
@@ -0,0 +1,42 @@
+/**
+ * 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
+ *
+ * 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.retryReasonCategories;
+
+import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_RESET_ABBREVIATION;
+import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_RESET_MESSAGE;
+
+/**
+ * Category that can capture server-response errors for connection-reset exception.
+ */
+public class ConnectionResetRetryReason extends
+ RetryReasonCategory {
+
+ @Override
+ Boolean canCapture(final Exception ex,
+ final Integer statusCode,
+ final String serverErrorMessage) {
+ return checkExceptionMessage(ex, CONNECTION_RESET_MESSAGE);
+ }
+
+ @Override
+ String getAbbreviation(final Integer statusCode,
+ final String serverErrorMessage) {
+ return CONNECTION_RESET_ABBREVIATION;
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ConnectionTimeoutRetryReason.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ConnectionTimeoutRetryReason.java
new file mode 100644
index 00000000000000..28f35dcc805468
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ConnectionTimeoutRetryReason.java
@@ -0,0 +1,43 @@
+/**
+ * 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
+ *
+ * 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.retryReasonCategories;
+
+
+import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_ABBREVIATION;
+import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_JDK_MESSAGE;
+
+/**
+ * Category that can capture server-response errors for connection-timeout.
+ */
+public class ConnectionTimeoutRetryReason extends
+ RetryReasonCategory {
+
+ @Override
+ String getAbbreviation(final Integer statusCode,
+ final String serverErrorMessage) {
+ return CONNECTION_TIMEOUT_ABBREVIATION;
+ }
+
+ @Override
+ Boolean canCapture(final Exception ex,
+ final Integer statusCode,
+ final String serverErrorMessage) {
+ return checkExceptionMessage(ex, CONNECTION_TIMEOUT_JDK_MESSAGE);
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ReadTimeoutRetryReason.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ReadTimeoutRetryReason.java
new file mode 100644
index 00000000000000..4663d9a52bbd6c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ReadTimeoutRetryReason.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
+ *
+ * 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.retryReasonCategories;
+
+import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.READ_TIMEOUT_ABBREVIATION;
+import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.READ_TIMEOUT_JDK_MESSAGE;
+
+/**
+ * Category that can capture server-response errors for read-timeout.
+ */
+public class ReadTimeoutRetryReason extends RetryReasonCategory {
+
+ @Override
+ Boolean canCapture(final Exception ex,
+ final Integer statusCode,
+ final String serverErrorMessage) {
+ return checkExceptionMessage(ex, READ_TIMEOUT_JDK_MESSAGE);
+ }
+
+ @Override
+ String getAbbreviation(final Integer statusCode,
+ final String serverErrorMessage) {
+ return READ_TIMEOUT_ABBREVIATION;
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/RetryReasonCategory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/RetryReasonCategory.java
new file mode 100644
index 00000000000000..893451b496f457
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/RetryReasonCategory.java
@@ -0,0 +1,90 @@
+/**
+ * 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
+ *
+ * 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.retryReasonCategories;
+
+import java.util.Locale;
+
+/**
+ * Provides methods to define if given exception can be categorised to certain category.
+ * Each category has a different implementation of the abstract class.
+ */
+public abstract class RetryReasonCategory {
+
+ /**
+ * Returns if given server response error can be categorised by the implementation.
+ *
+ * @param ex exception captured in the server response.
+ * @param statusCode statusCode on the server response
+ * @param serverErrorMessage serverErrorMessage on the server response.
+ *
+ * @return
true if server response error can be categorised by the implementation
+ *
false if response error can not be categorised by the implementation
+ */
+ abstract Boolean canCapture(Exception ex,
+ Integer statusCode,
+ String serverErrorMessage);
+
+ /**
+ * Returns the abbreviation corresponding to the server response error.
+ *
+ * @param statusCode statusCode on the server response
+ * @param serverErrorMessage serverErrorMessage on the server response.
+ *
+ * @return abbreviation on the basis of the statusCode and the serverErrorMessage
+ */
+ abstract String getAbbreviation(Integer statusCode, String serverErrorMessage);
+
+ /**
+ * Converts the server-error response to an abbreviation if the response can be
+ * categorised by the implementation.
+ *
+ * @param ex exception received while making API request
+ * @param statusCode statusCode received in the server-response
+ * @param serverErrorMessage error-message received in the server-response
+ *
+ * @return abbreviation if the server-response can be categorised by the implementation.
+ * null if the server-response can not be categorised by the implementation.
+ */
+ public String captureAndGetAbbreviation(Exception ex,
+ Integer statusCode,
+ String serverErrorMessage) {
+ if (canCapture(ex, statusCode, serverErrorMessage)) {
+ return getAbbreviation(statusCode, serverErrorMessage);
+ }
+ return null;
+ }
+
+ /**
+ * Checks if a required search-string is in the exception's message.
+ */
+ Boolean checkExceptionMessage(final Exception exceptionCaptured,
+ final String search) {
+ if (search == null) {
+ return false;
+ }
+ if (exceptionCaptured != null
+ && exceptionCaptured.getMessage() != null
+ && exceptionCaptured.getMessage()
+ .toLowerCase(Locale.US)
+ .contains(search.toLowerCase(Locale.US))) {
+ return true;
+ }
+ return false;
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ServerErrorRetryReason.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ServerErrorRetryReason.java
new file mode 100644
index 00000000000000..dd67a0cb8cbba2
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/ServerErrorRetryReason.java
@@ -0,0 +1,67 @@
+/**
+ * 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
+ *
+ * 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.retryReasonCategories;
+
+import static java.net.HttpURLConnection.HTTP_UNAVAILABLE;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_STATUS_CATEGORY_QUOTIENT;
+import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.EGRESS_OVER_ACCOUNT_LIMIT;
+import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.INGRESS_OVER_ACCOUNT_LIMIT;
+import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.EGRESS_LIMIT_BREACH_ABBREVIATION;
+import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.INGRESS_LIMIT_BREACH_ABBREVIATION;
+import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.OPERATION_BREACH_MESSAGE;
+import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.OPERATION_LIMIT_BREACH_ABBREVIATION;
+
+/**
+ * Category that can capture server-response errors for 5XX status-code.
+ */
+public class ServerErrorRetryReason extends RetryReasonCategory {
+
+ @Override
+ Boolean canCapture(final Exception ex,
+ final Integer statusCode,
+ final String serverErrorMessage) {
+ if (statusCode == null || statusCode / HTTP_STATUS_CATEGORY_QUOTIENT != 5) {
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ String getAbbreviation(final Integer statusCode,
+ final String serverErrorMessage) {
+ if (statusCode == HTTP_UNAVAILABLE && serverErrorMessage != null) {
+ String splitedServerErrorMessage = serverErrorMessage.split(System.lineSeparator(),
+ 2)[0];
+ if (INGRESS_OVER_ACCOUNT_LIMIT.getErrorMessage().equalsIgnoreCase(
+ splitedServerErrorMessage)) {
+ return INGRESS_LIMIT_BREACH_ABBREVIATION;
+ }
+ if (EGRESS_OVER_ACCOUNT_LIMIT.getErrorMessage().equalsIgnoreCase(
+ splitedServerErrorMessage)) {
+ return EGRESS_LIMIT_BREACH_ABBREVIATION;
+ }
+ if (OPERATION_BREACH_MESSAGE.equalsIgnoreCase(
+ splitedServerErrorMessage)) {
+ return OPERATION_LIMIT_BREACH_ABBREVIATION;
+ }
+ return HTTP_UNAVAILABLE + "";
+ }
+ return statusCode + "";
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownHostRetryReason.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownHostRetryReason.java
new file mode 100644
index 00000000000000..c329348d81f8d6
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownHostRetryReason.java
@@ -0,0 +1,45 @@
+/**
+ * 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
+ *
+ * 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.retryReasonCategories;
+
+import java.net.UnknownHostException;
+
+import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.UNKNOWN_HOST_EXCEPTION_ABBREVIATION;
+
+/**
+ * Category that can capture server-response errors for {@link UnknownHostException}.
+ */
+public class UnknownHostRetryReason extends RetryReasonCategory {
+
+ @Override
+ Boolean canCapture(final Exception ex,
+ final Integer statusCode,
+ final String serverErrorMessage) {
+ if (ex instanceof UnknownHostException) {
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ String getAbbreviation(final Integer statusCode,
+ final String serverErrorMessage) {
+ return UNKNOWN_HOST_EXCEPTION_ABBREVIATION;
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownIOExceptionRetryReason.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownIOExceptionRetryReason.java
new file mode 100644
index 00000000000000..8a69ebb928d684
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownIOExceptionRetryReason.java
@@ -0,0 +1,47 @@
+/**
+ * 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
+ *
+ * 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.retryReasonCategories;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.IO_EXCEPTION_ABBREVIATION;
+
+
+/**
+ * Category that can capture server-response errors for {@link IOException}.
+ */
+public class UnknownIOExceptionRetryReason extends
+ RetryReasonCategory {
+
+ @Override
+ Boolean canCapture(final Exception ex,
+ final Integer statusCode,
+ final String serverErrorMessage) {
+ if (ex instanceof IOException) {
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ String getAbbreviation(final Integer statusCode,
+ final String serverErrorMessage) {
+ return IO_EXCEPTION_ABBREVIATION;
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownSocketExceptionRetryReason.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownSocketExceptionRetryReason.java
new file mode 100644
index 00000000000000..18e9f115feaf69
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/UnknownSocketExceptionRetryReason.java
@@ -0,0 +1,46 @@
+/**
+ * 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
+ *
+ * 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.retryReasonCategories;
+
+import java.net.SocketException;
+
+import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.SOCKET_EXCEPTION_ABBREVIATION;
+
+/**
+ * Category that can capture server-response errors for {@link SocketException}.
+ */
+public class UnknownSocketExceptionRetryReason extends
+ RetryReasonCategory {
+
+ @Override
+ Boolean canCapture(final Exception ex,
+ final Integer statusCode,
+ final String serverErrorMessage) {
+ if (ex instanceof SocketException) {
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ String getAbbreviation(final Integer statusCode,
+ final String serverErrorMessage) {
+ return SOCKET_EXCEPTION_ABBREVIATION;
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/package-info.java
new file mode 100644
index 00000000000000..7d8078620af777
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/retryReasonCategories/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+/**
+ * A retryReasonCategory defines methods applicable on server-response errors.
+ */
+@Private
+@Evolving
+package org.apache.hadoop.fs.azurebfs.services.retryReasonCategories;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
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 5a115451df159b..e2ab95aec91028 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
@@ -61,12 +61,25 @@ public class TracingContext {
private final TracingHeaderFormat format; // header ID display options
private Listener listener = null; // null except when testing
//final concatenated ID list set into x-ms-client-request-id header
+ private String fallbackDFSAppend = "B";
private String header = EMPTY_STRING;
+ /**
+ * If {@link #primaryRequestId} is null, this field shall be set equal
+ * to the last part of the {@link #clientRequestId}'s UUID
+ * in {@link #constructHeader(AbfsHttpOperation, String)} only on the
+ * first API call for an operation. Subsequent retries for that operation
+ * will not change this field. In case {@link #primaryRequestId} is non-null,
+ * this field shall not be set.
+ */
+ private String primaryRequestIdForRetry;
+
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-]*";
+ private Integer operatedBlobCount = null;
+
/**
* Initialize TracingContext
* @param clientCorrelationID Provided over config by client
@@ -139,6 +152,10 @@ public void setOperation(FSOperationType operation) {
this.opType = operation;
}
+ public int getRetryCount() {
+ return retryCount;
+ }
+
public void setRetryCount(int retryCount) {
this.retryCount = retryCount;
}
@@ -147,20 +164,34 @@ public void setListener(Listener listener) {
this.listener = listener;
}
+ public void setFallbackDFSAppend(String fallbackDFSAppend) {
+ this.fallbackDFSAppend = fallbackDFSAppend;
+ }
+
+ public String getFallbackDFSAppend() {
+ return fallbackDFSAppend;
+ }
+
/**
* Concatenate all identifiers separated by (:) into a string and set into
* X_MS_CLIENT_REQUEST_ID header of the http operation
* @param httpOperation AbfsHttpOperation instance to set header into
* connection
+ * @param previousFailure Failure seen before this API trigger on same operation
+ * from AbfsClient.
*/
- public void constructHeader(AbfsHttpOperation httpOperation) {
+ public void constructHeader(AbfsHttpOperation httpOperation, String previousFailure) {
clientRequestId = UUID.randomUUID().toString();
switch (format) {
case ALL_ID_FORMAT: // Optional IDs (e.g. streamId) may be empty
header =
clientCorrelationID + ":" + clientRequestId + ":" + fileSystemID + ":"
- + primaryRequestId + ":" + streamID + ":" + opType + ":"
- + retryCount;
+ + getPrimaryRequestIdForHeader(retryCount > 0) + ":" + streamID
+ + ":" + opType + ":" + retryCount;
+ header = addFailureReasons(header, previousFailure) + ":" + fallbackDFSAppend;
+ if (operatedBlobCount != null) {
+ header += (":" + operatedBlobCount);
+ }
break;
case TWO_ID_FORMAT:
header = clientCorrelationID + ":" + clientRequestId;
@@ -172,6 +203,39 @@ public void constructHeader(AbfsHttpOperation httpOperation) {
listener.callTracingHeaderValidator(header, format);
}
httpOperation.setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, header);
+ /*
+ * In case the primaryRequestId is an empty-string and if it is the first try to
+ * API call (previousFailure shall be null), maintain the last part of clientRequestId's
+ * UUID in primaryRequestIdForRetry. This field shall be used as primaryRequestId part
+ * of the x-ms-client-request-id header in case of retry of the same API-request.
+ */
+ if (primaryRequestId.isEmpty() && previousFailure == null) {
+ String[] clientRequestIdParts = clientRequestId.split("-");
+ primaryRequestIdForRetry = clientRequestIdParts[
+ clientRequestIdParts.length - 1];
+ }
+ }
+
+ /**
+ * Provide value to be used as primaryRequestId part of x-ms-client-request-id header.
+ * @param isRetry define if it's for a retry case.
+ * @return {@link #primaryRequestIdForRetry}:If the {@link #primaryRequestId}
+ * is an empty-string, and it's a retry iteration.
+ * {@link #primaryRequestId} for other cases.
+ */
+ private String getPrimaryRequestIdForHeader(final Boolean isRetry) {
+ if (!primaryRequestId.isEmpty() || !isRetry) {
+ return primaryRequestId;
+ }
+ return primaryRequestIdForRetry;
+ }
+
+ private String addFailureReasons(final String header,
+ final String previousFailure) {
+ if (previousFailure == null) {
+ return header;
+ }
+ return String.format("%s_%s", header, previousFailure);
}
/**
@@ -182,4 +246,19 @@ public String getHeader() {
return header;
}
+ public String getPrimaryRequestId() {
+ return primaryRequestId;
+ }
+
+ public void setOperatedBlobCount(Integer count) {
+ operatedBlobCount = count;
+ }
+
+ public Integer getOperatedBlobCount() {
+ return operatedBlobCount;
+ }
+
+ public FSOperationType getOpType() {
+ return opType;
+ }
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/UriUtils.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/UriUtils.java
index e27d54b443ca20..857b20b2fd80bd 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/UriUtils.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/UriUtils.java
@@ -169,6 +169,20 @@ public static String getMaskedUrl(URL url) {
return url.toString().replace(queryString, maskedQueryString);
}
+ public static String encodeMetadataAttribute(String value) throws UnsupportedEncodingException {
+ // We have to URL encode the attribute as it could
+ // have URI special characters which unless encoded will result
+ // in 403 errors from the server. This is due to metadata properties
+ // being sent in the HTTP header of the request which is in turn used
+ // on the server side to authorize the request.
+ return value == null ? null : URLEncoder.encode(value, StandardCharsets.UTF_8.name());
+ }
+
+ public static String decodeMetadataAttribute(String encoded) throws UnsupportedEncodingException {
+ return encoded == null ? null :
+ java.net.URLDecoder.decode(encoded, StandardCharsets.UTF_8.name());
+ }
+
private UriUtils() {
}
}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/WeakReferenceMap.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/WeakReferenceMap.java
new file mode 100644
index 00000000000000..cd47809689c416
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/WeakReferenceMap.java
@@ -0,0 +1,333 @@
+/*
+ * 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.utils;
+
+import java.lang.ref.WeakReference;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+import javax.annotation.Nullable;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.store.LogExactlyOnce;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * A map of keys type K to objects of type V which uses weak references,
+ * so does lot leak memory through long-lived references
+ * at the expense of losing references when GC takes place..
+ *
+ * This class is intended be used instead of ThreadLocal storage when
+ * references are to be cleaned up when the instance holding.
+ * In this use case, the key is the Long key.
+ *
+ * Concurrency.
+ * The class assumes that map entries are rarely contended for when writing,
+ * and that not blocking other threads is more important than atomicity.
+ * - a ConcurrentHashMap is used to map keys to weak references, with
+ * all its guarantees.
+ * - there is no automatic pruning.
+ * - see {@link #create(Object)} for the concurrency semantics on entry creation.
+ */
+@InterfaceAudience.Private
+public class WeakReferenceMap {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(WeakReferenceMap.class);
+
+ /**
+ * The reference map.
+ */
+ private final Map> map = new ConcurrentHashMap<>();
+
+ /**
+ * Supplier of new instances.
+ */
+ private final Function super K, ? extends V> factory;
+
+ /**
+ * Nullable callback when a get on a key got a weak reference back.
+ * The assumption is that this is for logging/stats, which is why
+ * no attempt is made to use the call as a supplier of a new value.
+ */
+ private final Consumer super K> referenceLost;
+
+ /**
+ * Counter of references lost.
+ */
+ private final AtomicLong referenceLostCount = new AtomicLong();
+
+ /**
+ * Counter of entries created.
+ */
+ private final AtomicLong entriesCreatedCount = new AtomicLong();
+
+ /**
+ * Log to report loss of a reference during the create phase, which
+ * is believed to be a cause of HADOOP-18456.
+ */
+ private final LogExactlyOnce referenceLostDuringCreation = new LogExactlyOnce(LOG);
+
+ /**
+ * instantiate.
+ * @param factory supplier of new instances
+ * @param referenceLost optional callback on lost references.
+ */
+ public WeakReferenceMap(
+ Function super K, ? extends V> factory,
+ @Nullable final Consumer super K> referenceLost) {
+
+ this.factory = requireNonNull(factory);
+ this.referenceLost = referenceLost;
+ }
+
+ @Override
+ public String toString() {
+ return "WeakReferenceMap{" +
+ "size=" + size() +
+ ", referenceLostCount=" + referenceLostCount +
+ ", entriesCreatedCount=" + entriesCreatedCount +
+ '}';
+ }
+
+ /**
+ * Map size.
+ * @return the current map size.
+ */
+ public int size() {
+ return map.size();
+ }
+
+ /**
+ * Clear all entries.
+ */
+ public void clear() {
+ map.clear();
+ }
+
+ /**
+ * look up the value, returning the possibly empty weak reference
+ * to a value, or null if no value was found.
+ * @param key key to look up
+ * @return null if there is no entry, a weak reference if found
+ */
+ public WeakReference lookup(K key) {
+ return map.get(key);
+ }
+
+ /**
+ * Get the value, creating if needed.
+ * @param key key.
+ * @return an instance.
+ */
+ public V get(K key) {
+ final WeakReference currentWeakRef = lookup(key);
+ // resolve it, after which if not null, we have a strong reference
+ V strongVal = resolve(currentWeakRef);
+ if (strongVal != null) {
+ // all good.
+ return strongVal;
+ }
+
+ // here, either currentWeakRef was null, or its reference was GC'd.
+ if (currentWeakRef != null) {
+ // garbage collection removed the reference.
+
+ // explicitly remove the weak ref from the map if it has not
+ // been updated by this point
+ // this is here just for completeness.
+ map.remove(key, currentWeakRef);
+
+ // log/report the loss.
+ noteLost(key);
+ }
+
+ // create a new value and add it to the map
+ return create(key);
+ }
+
+ /**
+ * Create a new instance under a key.
+ *
+ * The instance is created, added to the map and then the
+ * map value retrieved.
+ * This ensures that the reference returned is that in the map,
+ * even if there is more than one entry being created at the same time.
+ * If that race does occur, it will be logged the first time it happens
+ * for this specific map instance.
+ *
+ * HADOOP-18456 highlighted the risk of a concurrent GC resulting a null
+ * value being retrieved and so returned.
+ * To prevent this:
+ *
+ *
A strong reference is retained to the newly created instance
+ * in a local variable.
+ *
That variable is used after the resolution process, to ensure
+ * the JVM doesn't consider it "unreachable" and so eligible for GC.
+ *
A check is made for the resolved reference being null, and if so,
+ * the put() is repeated
+ *
+ * @param key key
+ * @return the created value
+ */
+ public V create(K key) {
+ entriesCreatedCount.incrementAndGet();
+ /*
+ Get a strong ref so even if a GC happens in this method the reference is not lost.
+ It is NOT enough to have a reference in a field, it MUST be used
+ so as to ensure the reference isn't optimized away prematurely.
+ "A reachable object is any object that can be accessed in any potential continuing
+ computation from any live thread."
+ */
+
+ final V strongRef = requireNonNull(factory.apply(key),
+ "factory returned a null instance");
+ V resolvedStrongRef;
+ do {
+ WeakReference newWeakRef = new WeakReference<>(strongRef);
+
+ // put it in the map
+ map.put(key, newWeakRef);
+
+ // get it back from the map
+ WeakReference retrievedWeakRef = map.get(key);
+ // resolve that reference, handling the situation where somehow it was removed from the map
+ // between the put() and the get()
+ resolvedStrongRef = resolve(retrievedWeakRef);
+ if (resolvedStrongRef == null) {
+ referenceLostDuringCreation.warn("reference to %s lost during creation", key);
+ noteLost(key);
+ }
+ } while (resolvedStrongRef == null);
+
+ // note if there was any change in the reference.
+ // as this forces strongRef to be kept in scope
+ if (strongRef != resolvedStrongRef) {
+ LOG.debug("Created instance for key {}: {} overwritten by {}",
+ key, strongRef, resolvedStrongRef);
+ }
+
+ return resolvedStrongRef;
+ }
+
+ /**
+ * Put a value under the key.
+ * A null value can be put, though on a get() call
+ * a new entry is generated
+ *
+ * @param key key
+ * @param value value
+ * @return any old non-null reference.
+ */
+ public V put(K key, V value) {
+ return resolve(map.put(key, new WeakReference<>(value)));
+ }
+
+ /**
+ * Remove any value under the key.
+ * @param key key
+ * @return any old non-null reference.
+ */
+ public V remove(K key) {
+ return resolve(map.remove(key));
+ }
+
+ /**
+ * Does the map have a valid reference for this object?
+ * no-side effects: there's no attempt to notify or cleanup
+ * if the reference is null.
+ * @param key key to look up
+ * @return true if there is a valid reference.
+ */
+ public boolean containsKey(K key) {
+ final WeakReference current = lookup(key);
+ return resolve(current) != null;
+ }
+
+ /**
+ * Given a possibly null weak reference, resolve
+ * its value.
+ * @param r reference to resolve
+ * @return the value or null
+ */
+ protected V resolve(WeakReference r) {
+ return r == null ? null : r.get();
+ }
+
+ /**
+ * Prune all null weak references, calling the referenceLost
+ * callback for each one.
+ *
+ * non-atomic and non-blocking.
+ * @return the number of entries pruned.
+ */
+ public int prune() {
+ int count = 0;
+ final Iterator>> it = map.entrySet().iterator();
+ while (it.hasNext()) {
+ final Map.Entry> next = it.next();
+ if (next.getValue().get() == null) {
+ it.remove();
+ count++;
+ noteLost(next.getKey());
+ }
+ }
+ return count;
+ }
+
+ /**
+ * Notify the reference lost callback.
+ * @param key key of lost reference
+ */
+ private void noteLost(final K key) {
+ // increment local counter
+ referenceLostCount.incrementAndGet();
+
+ // and call any notification function supplied in the constructor
+ if (referenceLost != null) {
+ referenceLost.accept(key);
+ }
+ }
+
+ /**
+ * Get count of references lost as detected
+ * during prune() or get() calls.
+ * @return count of references lost
+ */
+ public final long getReferenceLostCount() {
+ return referenceLostCount.get();
+ }
+
+ /**
+ * Get count of entries created on demand.
+ * @return count of entries created
+ */
+ public final long getEntriesCreatedCount() {
+ return entriesCreatedCount.get();
+ }
+}
+
diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md
index dfb7f3f42a5cf9..ac77765f9e0b52 100644
--- a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md
+++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md
@@ -769,6 +769,26 @@ Hflush() being the only documented API that can provide persistent data
transfer, Flush() also attempting to persist buffered data will lead to
performance issues.
+<<<<<<< HEAD
+=======
+### Hundred Continue Options
+
+`fs.azure.account.expect.header.enabled`: This configuration parameter is used
+to specify whether you wish to send a expect 100 continue header with each
+append request or not. It is configured to true by default. This flag configures
+the client to check with the Azure store before uploading a block of data from
+an output stream. This allows the client to throttle back gracefully -before
+actually attempting to upload the block. In experiments this provides
+significant throughput improvements under heavy load. For more information :
+- https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Expect
+
+
+### Account level throttling Options
+
+`fs.azure.account.operation.idle.timeout`: This value specifies the time after which the timer for the analyzer (read or
+write) should be paused until no new request is made again. The default value for the same is 60 seconds.
+
+>>>>>>> c88011c6046... HADOOP-18146: ABFS: Added changes for expect hundred continue header (#4039)
### HNS Check Options
Config `fs.azure.account.hns.enabled` provides an option to specify whether
the storage account is HNS enabled or not. In case the config is not provided,
@@ -874,6 +894,9 @@ when there are too many writes from the same process.
time. Effectively this will be the threadpool size within the
AbfsOutputStream instance. Set the value in between 1 to 8 both inclusive.
+`fs.azure.analysis.period`: The time after which sleep duration is recomputed after analyzing metrics. The default value
+for the same is 10 seconds.
+
`fs.azure.write.max.requests.to.queue`: To set the maximum write requests
that can be queued. Memory consumption of AbfsOutputStream instance can be
tuned with this config considering each queued request holds a buffer. Set
diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md b/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md
index 933f86be3e8961..e256fbef546cb2 100644
--- a/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md
+++ b/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md
@@ -602,26 +602,76 @@ various test combinations, it will:
2. Run tests for all combinations
3. Summarize results across all the test combination runs.
-As a pre-requisite step, fill config values for test accounts and credentials
-needed for authentication in `src/test/resources/azure-auth-keys.xml.template`
-and rename as `src/test/resources/azure-auth-keys.xml`.
+Below are the pre-requiste steps to follow:
+1. Copy
-**To add a new test combination:** Templates for mandatory test combinations
-for PR validation are present in `dev-support/testrun-scripts/runtests.sh`.
-If a new one needs to be added, add a combination set within
-`dev-support/testrun-scripts/runtests.sh` similar to the ones already defined
-and
-1. Provide a new combination name
-2. Update properties and values array which need to be effective for the test
-combination
-3. Call generateconfigs
+ ./src/test/resources/azure-auth-keys.xml.template
+ TO
+ ./src/test/resources/azure-auth-keys.xml
+ Update account names that should be used in the test run for HNS and non-HNS
+ combinations in the 2 properties present in the xml (account name should be
+ without domain part), namely
+
+ fs.azure.hnsTestAccountName
+ fs.azure.nonHnsTestAccountName
+ azure-auth-keys.xml is listed in .gitignore, so any accidental account name leak is prevented.
+
+```
+XInclude is supported, so for extra security secrets may be
+kept out of the source tree then referenced through an an XInclude element:
+
+
+```
+
+2. Create account config files (one config file per account) in folder:
+
+ ./src/test/resources/accountSettings/
+ Follow the instruction in the start of the template file
+
+ accountName_settings.xml.template
+ within accountSettings folder while creating account config file.
+ New files created in folder accountSettings is listed in .gitignore to
+ prevent accidental cred leaks.
**To run PR validation:** Running command
-* `dev-support/testrun-scripts/runtests.sh` will generate configurations for
-each of the combinations defined and run tests for all the combinations.
-* `dev-support/testrun-scripts/runtests.sh -c {combinationname}` Specific
-combinations can be provided with -c option. If combinations are provided
-with -c option, tests for only those combinations will be run.
+* `dev-support/testrun-scripts/runtests.sh` will prompt as below:
+```bash
+Choose action:
+[Note - SET_ACTIVE_TEST_CONFIG will help activate the config for IDE/single test class runs]
+1) SET_ACTIVE_TEST_CONFIG 4) SET_OR_CHANGE_TEST_ACCOUNT
+2) RUN_TEST 5) PRINT_LOG4J_LOG_PATHS_FROM_LAST_RUN
+3) CLEAN_UP_OLD_TEST_CONTAINERS
+#? 2
+```
+Enter 1: for setting active combination for IDE test run/single mvn test class runs.
+
+Enter 2: for choosing the combination to choose for mvn full test suite.
+
+Enter 3: For clean-up of any abruptly ending test leaving auto generated test
+container on the account.
+
+Enter 4: To create/modify the config file that decides the account to use for specific test combination.
+
+Enter 5: To print the log4j paths the last test runs.
+
+On next prompt, current list of combinations to choose are provided.
+Sample for Run_TEST action:
+```bash
+Enter parallel test run process count [default - 8]: 4
+Set the active test combination to run the action:
+1) HNS-OAuth 3) nonHNS-SharedKey 5) AllCombinationsTestRun
+2) HNS-SharedKey 4) AppendBlob-HNS-OAuth 6) Quit
+#? 1
+
+Combination specific property setting: [ key=fs.azure.account.auth.type , value=OAuth ]
+
+Activated [src/test/resources/abfs-combination-test-configs.xml] - for account: snvijayacontracttest for combination HNS-OAuth
+Running test for combination HNS-OAuth on account snvijayacontracttest [ProcessCount=4]
+Test run report can be seen in dev-support/testlogs/2022-10-07_05-23-22/Test-Logs-HNS-OAuth.txt
+````
+
+Provide the option for the action chosen first.
**Test logs:** Test runs will create a folder within dev-support/testlogs to
save the test logs. Folder name will be the test start timestamp. The mvn verify
@@ -632,25 +682,18 @@ consolidated results of all the combination runs will be saved into a file as
Test-Results.log in the same folder. When run for PR validation, the
consolidated test results needs to be pasted into the PR comment section.
-**To generate config for use in IDE:** Running command with -a (activate) option
-`dev-support/testrun-scripts/runtests.sh -a {combination name}` will update
-the effective config relevant for the specific test combination. Hence the same
-config files used by the mvn test runs can be used for IDE without any manual
-updates needed within config file.
-
-**Other command line options:**
-* -a Specify the combination name which needs to be
-activated. This is to be used to generate config for use in IDE.
-* -c Specify the combination name for test runs. If this
-config is specified, tests for only the specified combinations will run. All
-combinations of tests will be running if this config is not specified.
-* -t ABFS mvn tests are run in parallel mode. Tests by default
-are run with 8 thread count. It can be changed by providing -t
-
-In order to test ABFS, please add the following configuration to your
-`src/test/resources/azure-auth-keys.xml` file. Note that the ABFS tests include
-compatibility tests which require WASB credentials, in addition to the ABFS
-credentials.
+**To add a new test combination:** Templates for mandatory test combinations
+for PR validation are present in `dev-support/testrun-scripts/runtests.sh`.
+If a new one needs to be added, add a combination to
+`dev-support/testrun-scripts/runtests.sh`.
+(Refer to current active combinations within
+`SECTION: COMBINATION DEFINITIONS AND TRIGGER` and
+`SECTION: TEST COMBINATION METHODS` in the script).
+
+**Test Configuration Details:**
+
+ Note that the ABFS tests include compatibility tests which require WASB
+ credentials, in addition to the ABFS credentials.
```xml
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AzureBlobStorageTestAccount.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AzureBlobStorageTestAccount.java
index 5d2d5d4afdc3f4..72793eaecac04d 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AzureBlobStorageTestAccount.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AzureBlobStorageTestAccount.java
@@ -21,6 +21,7 @@
import com.microsoft.azure.storage.*;
import com.microsoft.azure.storage.blob.*;
import com.microsoft.azure.storage.core.Base64;
+import jdk.nashorn.internal.parser.Token;
import org.junit.Assert;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -33,11 +34,14 @@
import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
import org.apache.hadoop.fs.azure.metrics.AzureFileSystemMetricsSystem;
import org.apache.hadoop.fs.azure.integration.AzureTestUtils;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException;
+import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
import org.apache.hadoop.metrics2.AbstractMetric;
import org.apache.hadoop.metrics2.MetricsRecord;
import org.apache.hadoop.metrics2.MetricsSink;
import org.apache.hadoop.metrics2.MetricsTag;
import org.apache.hadoop.metrics2.impl.TestMetricsConfig;
+import org.apache.kerby.kerberos.kerb.provider.TokenProvider;
import java.io.File;
import java.io.IOException;
@@ -71,6 +75,7 @@ public final class AzureBlobStorageTestAccount implements AutoCloseable,
public static final String MOCK_CONTAINER_NAME = "mockContainer";
public static final String WASB_AUTHORITY_DELIMITER = "@";
public static final String WASB_SCHEME = "wasb";
+ public static final String WASBS_SCHEME = "wasbs";
public static final String PATH_DELIMITER = "/";
public static final String AZURE_ROOT_CONTAINER = "$root";
public static final String MOCK_WASB_URI = "wasb://" + MOCK_CONTAINER_NAME
@@ -456,13 +461,13 @@ public static void setMockAccountKey(Configuration conf, String accountName) {
private static URI createAccountUri(String accountName)
throws URISyntaxException {
- return new URI(WASB_SCHEME + ":" + PATH_DELIMITER + PATH_DELIMITER
+ return new URI(WASBS_SCHEME + ":" + PATH_DELIMITER + PATH_DELIMITER
+ accountName);
}
private static URI createAccountUri(String accountName, String containerName)
throws URISyntaxException {
- return new URI(WASB_SCHEME + ":" + PATH_DELIMITER + PATH_DELIMITER
+ return new URI(WASBS_SCHEME + ":" + PATH_DELIMITER + PATH_DELIMITER
+ containerName + WASB_AUTHORITY_DELIMITER + accountName);
}
@@ -505,6 +510,37 @@ static CloudStorageAccount createStorageAccount(String accountName,
credentials = new StorageCredentialsAccountAndKey(
accountName.split("\\.")[0], accountKey);
}
+ return new CloudStorageAccount(credentials);
+ }
+
+ static CloudStorageAccount createStorageAccount(String accountName,
+ Configuration conf, boolean allowAnonymous, NativeAzureFileSystem fs)
+ throws URISyntaxException,
+ KeyProviderException, IOException {
+ String accountKey = AzureNativeFileSystemStore
+ .getAccountKeyFromConfiguration(accountName, conf);
+ final StorageCredentials credentials;
+ if (fs.getStore().getTokenProviderStore() != null){
+ AccessTokenProvider tokenProvider = fs.getStore().getTokenProviderStore();
+ int iend = accountName.indexOf("."); //this finds the first occurrence of "."
+ if (iend != -1) {
+ accountName = accountName.substring(0 , iend); //this will give abc
+ }
+ credentials = new StorageCredentialsToken(accountName, tokenProvider.getToken().getAccessToken());
+ return new CloudStorageAccount(credentials, true);
+ }
+ if (accountKey == null) {
+ if (allowAnonymous) {
+ credentials = StorageCredentialsAnonymous.ANONYMOUS;
+ } else {
+ LOG.warn("Skipping live Azure test because of missing key for"
+ + " account '" + accountName + "'.");
+ return null;
+ }
+ } else {
+ credentials = new StorageCredentialsAccountAndKey(
+ accountName.split("\\.")[0], accountKey);
+ }
return new CloudStorageAccount(credentials);
}
@@ -540,6 +576,18 @@ static CloudStorageAccount createTestAccount(Configuration conf)
return createStorageAccount(testAccountName, conf, false);
}
+ static CloudStorageAccount createTestAccount(Configuration conf, NativeAzureFileSystem fs)
+ throws URISyntaxException, KeyProviderException, IOException {
+ AzureTestUtils.assumeNamespaceDisabled(conf);
+
+ String testAccountName = verifyWasbAccountNameInConfig(conf);
+ if (testAccountName == null) {
+ LOG.warn("Skipping live Azure test because of missing test account");
+ return null;
+ }
+ return createStorageAccount(testAccountName, conf, false, fs);
+ }
+
public static enum CreateOptions {
UseSas, Readonly, CreateContainer, useThrottling
}
@@ -565,15 +613,12 @@ public static AzureBlobStorageTestAccount create(
throws Exception {
saveMetricsConfigFile();
NativeAzureFileSystem fs = null;
+ fs = new NativeAzureFileSystem();
CloudBlobContainer container = null;
Configuration conf = createTestConfiguration(initialConfiguration);
configurePageBlobDir(conf);
configureAtomicRenameDir(conf);
- CloudStorageAccount account = createTestAccount(conf);
- if (account == null) {
- return null;
- }
- fs = new NativeAzureFileSystem();
+ CloudStorageAccount account = null;
String containerName = useContainerSuffixAsContainerName
? containerNameSuffix
: String.format(
@@ -581,44 +626,45 @@ public static AzureBlobStorageTestAccount create(
System.getProperty("user.name"),
UUID.randomUUID().toString(),
containerNameSuffix);
+ String accountName = verifyWasbAccountNameInConfig(conf);
+ // Check if throttling is turned on and set throttling parameters
+ // appropriately.
+ conf.setBoolean(KEY_DISABLE_THROTTLING,
+ !createOptions.contains(CreateOptions.useThrottling));
+
+// // Set account URI and initialize Azure file system.
+ URI accountUri = createAccountUri(accountName, containerName);
+ fs.initialize(accountUri, conf);
+ if (fs.getStore().getTokenProviderStore() != null){
+ account = createTestAccount(conf, fs);
+ } else {
+ account = createTestAccount(conf);
+ }
+ if (account == null) {
+ return null;
+ }
container = account.createCloudBlobClient().getContainerReference(
containerName);
if (createOptions.contains(CreateOptions.CreateContainer)) {
container.createIfNotExists();
}
- String accountName = verifyWasbAccountNameInConfig(conf);
if (createOptions.contains(CreateOptions.UseSas)) {
String sas = generateSAS(container,
createOptions.contains(CreateOptions.Readonly));
- if (!createOptions.contains(CreateOptions.CreateContainer)) {
- // The caller doesn't want the container to be pre-created,
- // so delete it now that we have generated the SAS.
- container.delete();
- }
- // Remove the account key from the configuration to make sure we don't
- // cheat and use that.
- // but only if not in secure mode, which requires that login
if (!conf.getBoolean(AzureNativeFileSystemStore.KEY_USE_SECURE_MODE, false)) {
conf.set(ACCOUNT_KEY_PROPERTY_NAME + accountName, "");
}
// Set the SAS key.
conf.set(SAS_PROPERTY_NAME + containerName + "." + accountName, sas);
+ fs.setConf(conf);
+ if (!createOptions.contains(CreateOptions.CreateContainer)) {
+ // The caller doesn't want the container to be pre-created,
+ // so delete it now that we have generated the SAS.
+ container.delete();
+ }
}
-
- // Check if throttling is turned on and set throttling parameters
- // appropriately.
- if (createOptions.contains(CreateOptions.useThrottling)) {
- conf.setBoolean(KEY_DISABLE_THROTTLING, false);
- } else {
- conf.setBoolean(KEY_DISABLE_THROTTLING, true);
- }
-
configureSecureModeTestSettings(conf);
- // Set account URI and initialize Azure file system.
- URI accountUri = createAccountUri(accountName, containerName);
- fs.initialize(accountUri, conf);
-
// Create test account initializing the appropriate member variables.
//
AzureBlobStorageTestAccount testAcct =
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java
index f86af954933626..ca3deefff695ac 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java
@@ -189,7 +189,7 @@ public void run() {
LOG.info("Freeing lease");
lease.free();
}
- } catch (StorageException se) {
+ } catch (StorageException | IOException se) {
LOG.warn("Unable to free lease.", se);
}
}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
index 6d11207c479b51..4f26777f512489 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
@@ -38,6 +38,7 @@
import org.apache.commons.codec.net.URLCodec;
import org.apache.commons.lang3.NotImplementedException;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
import org.apache.http.client.utils.URIBuilder;
import com.microsoft.azure.storage.AccessCondition;
@@ -178,6 +179,13 @@ public CloudBlobContainerWrapper getContainerReference(String name)
return container;
}
+ @Override
+ public CloudBlobContainerWrapper getContainerReference(final String name,
+ final AccessTokenProvider tokenProvider)
+ throws URISyntaxException, StorageException {
+ return getContainerReference(name);
+ }
+
class MockCloudBlobContainerWrapper extends CloudBlobContainerWrapper {
private boolean created = false;
private HashMap metadata;
@@ -234,6 +242,13 @@ public CloudBlobDirectoryWrapper getDirectoryReference(String relativePath)
relativePath, true)));
}
+ @Override
+ public CloudBlobDirectoryWrapper getDirectoryReference(String relativePath, AccessTokenProvider tokenProvider)
+ throws URISyntaxException, StorageException {
+ return new MockCloudBlobDirectoryWrapper(new URI(fullUriString(
+ relativePath, true)));
+ }
+
@Override
public CloudBlockBlobWrapper getBlockBlobReference(String relativePath)
throws URISyntaxException, StorageException {
@@ -241,12 +256,27 @@ public CloudBlockBlobWrapper getBlockBlobReference(String relativePath)
false)), null, 0);
}
+ @Override
+ public CloudBlobWrapper getBlockBlobReference(final String relativePath,
+ final AccessTokenProvider tokenProvider)
+ throws URISyntaxException, StorageException, IOException {
+ return new MockCloudBlockBlobWrapper(new URI(fullUriString(relativePath,
+ false)), null, 0);
+ }
+
@Override
public CloudPageBlobWrapper getPageBlobReference(String blobAddressUri)
throws URISyntaxException, StorageException {
return new MockCloudPageBlobWrapper(new URI(blobAddressUri), null, 0);
}
+ @Override
+ public CloudBlobWrapper getPageBlobReference(String blobAddressUri,
+ final AccessTokenProvider tokenProvider)
+ throws URISyntaxException, StorageException, IOException {
+ return new MockCloudPageBlobWrapper(new URI(blobAddressUri), null, 0);
+ }
+
// helper to create full URIs for directory and blob.
// use withTrailingSlash=true to get a good path for a directory.
private String fullUriString(String relativePath, boolean withTrailingSlash) {
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java
index 8ac36c299b65b5..a4dce44bde4055 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java
@@ -1689,7 +1689,7 @@ public void run() {
firstEndTime = System.currentTimeMillis();
lease.free();
LOG.info(name + " freed lease " + lease.getLeaseID());
- } catch (StorageException e) {
+ } catch (StorageException | IOException e) {
fail("Unanticipated exception");
}
} else if (name.equals("second-thread")) {
@@ -1712,7 +1712,7 @@ public void run() {
try {
lease.free();
LOG.info(name + " freed lease " + lease.getLeaseID());
- } catch (StorageException e) {
+ } catch (StorageException | IOException e) {
assertTrue("Unanticipated exception", false);
}
} else {
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
index fd2f2690daea21..b79e49390ac46f 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
@@ -25,6 +25,9 @@
import java.util.UUID;
import java.util.concurrent.Callable;
+import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.PrefixMode;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -38,8 +41,10 @@
import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient;
import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore;
import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
@@ -55,6 +60,8 @@
import static org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.WASB_ACCOUNT_NAME_DOMAIN_SUFFIX;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DNS_PREFIX;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX;
import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND;
import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -69,7 +76,7 @@ public abstract class AbstractAbfsIntegrationTest extends
AbstractAbfsTestWithTimeout {
private static final Logger LOG =
- LoggerFactory.getLogger(AbstractAbfsIntegrationTest.class);
+ LoggerFactory.getLogger(AbstractAbfsIntegrationTest.class);
private boolean isIPAddress;
private NativeAzureFileSystem wasb;
@@ -106,7 +113,7 @@ protected AbstractAbfsIntegrationTest() throws Exception {
if (authType == AuthType.SharedKey) {
assumeTrue("Not set: " + FS_AZURE_ACCOUNT_KEY,
- abfsConfig.get(FS_AZURE_ACCOUNT_KEY) != null);
+ abfsConfig.get(FS_AZURE_ACCOUNT_KEY) != null);
// Update credentials
} else {
assumeTrue("Not set: " + FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME,
@@ -239,6 +246,9 @@ public Hashtable call() throws Exception {
}
}
+ public AccessTokenProvider getAccessTokenProvider(final AzureBlobFileSystem fs) {
+ return ITestAbfsClient.getAccessTokenProvider(fs.getAbfsStore().getClient());
+ }
public void loadConfiguredFileSystem() throws Exception {
// disable auto-creation of filesystem
@@ -412,8 +422,7 @@ private static String convertTestUrls(
}
if (data != null) {
- data = data.replace("." + fromDnsPrefix + ".",
- "." + toDnsPrefix + ".");
+ data = data.replace(fromDnsPrefix, toDnsPrefix);
}
return data;
}
@@ -427,6 +436,24 @@ public AzureBlobFileSystemStore getAbfsStore(final AzureBlobFileSystem fs) {
return fs.getAbfsStore();
}
+ public PrefixMode getPrefixMode(final AzureBlobFileSystem fs) {
+ return fs.getAbfsStore().getAbfsConfiguration().getPrefixMode();
+ }
+
+ public AbfsClient getClient(final AzureBlobFileSystem fs) {
+ return fs.getAbfsStore().getClient();
+ }
+
+ public boolean isNamespaceEnabled(final AzureBlobFileSystem fs) throws AzureBlobFileSystemException {
+ return fs.getAbfsStore()
+ .getIsNamespaceEnabled(getTestTracingContext(fs, true));
+ }
+
+ public void setAbfsClient(AzureBlobFileSystemStore abfsStore,
+ AbfsClient client) {
+ abfsStore.setClient(client);
+ }
+
public Path makeQualified(Path path) throws java.io.IOException {
return getFileSystem().makeQualified(path);
}
@@ -469,7 +496,7 @@ protected AbfsOutputStream createAbfsOutputStreamWithFlushEnabled(
return (AbfsOutputStream) abfss.createFile(path, fs.getFsStatistics(),
true, FsPermission.getDefault(), FsPermission.getUMask(fs.getConf()),
- getTestTracingContext(fs, false));
+ getTestTracingContext(fs, false), null);
}
/**
@@ -486,4 +513,30 @@ protected long assertAbfsStatistics(AbfsStatistic statistic,
(long) metricMap.get(statistic.getStatName()));
return expectedValue;
}
+
+ /**
+ * For creating directory with implicit parents. Doesn't change already explicit
+ * parents.
+ */
+ void createAzCopyDirectory(Path path) throws Exception {
+ AzcopyHelper azcopyHelper = new AzcopyHelper(
+ getAccountName(), getFileSystemName(), getFileSystem().getAbfsStore()
+ .getAbfsConfiguration()
+ .getRawConfiguration(), getFileSystem().getAbfsStore().getPrefixMode());
+ azcopyHelper.createFolderUsingAzcopy(
+ getFileSystem().makeQualified(path).toUri().getPath().substring(1));
+ }
+
+ /**
+ * For creating files with implicit parents. Doesn't change already explicit
+ * parents.
+ */
+ void createAzCopyFile(Path path) throws Exception {
+ AzcopyHelper azcopyHelper = new AzcopyHelper(getAccountName(),
+ getFileSystemName(), getFileSystem().getAbfsStore()
+ .getAbfsConfiguration()
+ .getRawConfiguration(), getFileSystem().getAbfsStore().getPrefixMode());
+ azcopyHelper.createFileUsingAzcopy(
+ getFileSystem().makeQualified(path).toUri().getPath().substring(1));
+ }
}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AzcopyHelper.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AzcopyHelper.java
new file mode 100644
index 00000000000000..c15d094c83c907
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AzcopyHelper.java
@@ -0,0 +1,217 @@
+/**
+ * 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;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileWriter;
+import java.io.IOException;
+
+import org.junit.Assume;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.services.PrefixMode;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.FORWARD_SLASH;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SAS_FIXED_TOKEN;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DNS_PREFIX;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX;
+
+public class AzcopyHelper {
+
+ File hadoopAzureDir;
+ String azcopyDirPath;
+ private String accountName;
+ private String fileSystemName;
+ private Configuration configuration;
+ private PrefixMode mode;
+
+ public AzcopyHelper(String accountName, String fileSystemName, Configuration configuration ,PrefixMode mode) throws Exception {
+ this.accountName = accountName.replace(ABFS_DNS_PREFIX, WASB_DNS_PREFIX);
+ this.fileSystemName = fileSystemName;
+ this.configuration = configuration;
+ this.mode = mode;
+ }
+
+ public void downloadAzcopyExecutableIfNotPresent() throws IOException, InterruptedException {
+ // Skip execution if Prefix Mode is DFS.
+ Assume.assumeTrue(mode == PrefixMode.BLOB);
+
+ // Find the hadoop-azure directory from the current working directory
+ File currentDir = new File(System.getProperty("user.dir"));
+ if (!currentDir.isDirectory() && !currentDir.getName().equals("hadoop-azure")) {
+ hadoopAzureDir = findHadoopAzureDir(currentDir);
+ if (hadoopAzureDir == null) {
+ throw new FileNotFoundException("hadoop-azure directory not found");
+ }
+ } else {
+ hadoopAzureDir = currentDir;
+ }
+
+ // Check if azcopy directory is present in the hadoop-azure directory, create it if it doesn't exist
+ azcopyDirPath = hadoopAzureDir.getAbsolutePath() + "/azcopy";
+ File azcopyDir = new File(azcopyDirPath);
+ if (!azcopyDir.exists()) {
+ boolean created = azcopyDir.mkdir();
+ // Check if azcopy is present in the azcopy directory
+ String azcopyPath = azcopyDirPath + "/azcopy";
+ File azcopyFile = new File(azcopyPath);
+ if (!azcopyFile.exists()) {
+ // If azcopy is not present, download and extract it
+ String downloadUrl = "https://aka.ms/downloadazcopy-v10-linux";
+ String downloadCmd = "wget " + downloadUrl + " -O azcopy.tar.gz" + " --no-check-certificate";
+ String[] downloadCmdArr = {"bash", "-c", downloadCmd};
+ Process downloadProcess = Runtime.getRuntime().exec(downloadCmdArr);
+ downloadProcess.waitFor();
+
+ // Extract the azcopy executable from the tarball
+ String extractCmd = "tar -xf azcopy.tar.gz -C " + hadoopAzureDir.getAbsolutePath();
+ String[] extractCmdArr = {"bash", "-c", extractCmd};
+ Process extractProcess = Runtime.getRuntime().exec(extractCmdArr);
+ extractProcess.waitFor();
+
+ // Rename the azcopy_linux_amd64_* directory to 'azcopy' and move it to the hadoop-azure directory
+ String renameCmd = "mv " + hadoopAzureDir.getAbsolutePath() + "/azcopy_linux_amd64_*/* " + azcopyDirPath;
+ String[] renameCmdArr = {"bash", "-c", renameCmd};
+ Process renameProcess = Runtime.getRuntime().exec(renameCmdArr);
+ renameProcess.waitFor();
+
+ // Remove the downloaded tarball and azcopy folder
+ String cleanupCmd = "rm -rf " + hadoopAzureDir.getAbsolutePath() + "/azcopy_linux_amd64_* azcopy.tar.gz";
+ String[] cleanupCmdArr = {"bash", "-c", cleanupCmd};
+ Process cleanupProcess = Runtime.getRuntime().exec(cleanupCmdArr);
+ cleanupProcess.waitFor();
+
+ // Set the execute permission on the azcopy executable
+ String chmodCmd = "chmod +x " + azcopyDirPath;
+ String[] chmodCmdArr = {"bash", "-c", chmodCmd};
+ Process chmodProcess = Runtime.getRuntime().exec(chmodCmdArr);
+ chmodProcess.waitFor();
+ }
+ }
+ // Change working directory to the hadoop-azure directory
+ System.setProperty("user.dir", hadoopAzureDir.getAbsolutePath());
+ }
+
+ private File findHadoopAzureDir(File dir) {
+ if (dir == null) {
+ return null;
+ }
+ File[] files = dir.listFiles();
+ if (files == null) {
+ return null;
+ }
+ for (File file : files) {
+ if (file.isDirectory() && file.getName().equals("hadoop-azure")) {
+ return file;
+ } else {
+ File hadoopAzureDir = findHadoopAzureDir(file);
+ if (hadoopAzureDir != null) {
+ return hadoopAzureDir;
+ }
+ }
+ }
+ return null;
+ }
+
+ public void createFileOrFolder(String pathFromContainerRoot, boolean isFile) throws Exception {
+ downloadAzcopyExecutableIfNotPresent();
+ String url = "https://" + accountName + FORWARD_SLASH + fileSystemName + FORWARD_SLASH +
+ pathFromContainerRoot;
+ // Add the SAS token in config file (should be Account SAS or Container SAS").
+ String configuredFixedToken = configuration.get(FS_AZURE_SAS_FIXED_TOKEN, null);
+ if (configuredFixedToken != null) {
+ if (isFile) {
+ createFileCreationScript(azcopyDirPath, "createFile" + Thread.currentThread().getName() + ".sh", azcopyDirPath, configuredFixedToken, url);
+ } else {
+ createFolderCreationScript(azcopyDirPath, "createFolder" + Thread.currentThread().getName() + ".sh", azcopyDirPath, configuredFixedToken, url);
+ }
+ } else {
+ throw new Exception("The SAS token provided is null");
+ }
+ String path;
+ if (isFile) {
+ path = azcopyDirPath + "/createFile" + Thread.currentThread().getName() + ".sh";
+ } else {
+ path = azcopyDirPath + "/createFolder" + Thread.currentThread().getName() + ".sh";
+ }
+ try {
+ ProcessBuilder pb = new ProcessBuilder(path);
+ Process p = pb.start();
+ // wait for the process to finish
+ int exitCode = p.waitFor();
+ } catch (IOException e) {
+ throw new IOException(e.getMessage());
+ } catch (InterruptedException e) {
+ throw new InterruptedException(e.getMessage());
+ }
+ String cleanupCmd = "rm -rf " + path;
+ String[] cleanupCmdArr = {"bash", "-c", cleanupCmd};
+ Process cleanupProcess = Runtime.getRuntime().exec(cleanupCmdArr);
+ cleanupProcess.waitFor();
+ }
+
+ public void createFileUsingAzcopy(String pathFromContainerRoot) throws Exception {
+ // Add the path you want to copy to as config.
+ if (pathFromContainerRoot != null) {
+ createFileOrFolder(pathFromContainerRoot, true);
+ }
+ }
+
+ public void createFolderUsingAzcopy(String pathFromContainerRoot) throws Exception {
+ // Add the path you want to copy to as config.
+ if (pathFromContainerRoot != null) {
+ createFileOrFolder(pathFromContainerRoot, false);
+ }
+ }
+
+ public static void createFileCreationScript(String folderPath, String scriptName, String azcopyPath, String sasToken, String containerName) {
+ String blobPath = containerName + "?" + sasToken; // construct the blob path
+ String scriptContent = "blobPath=\"" + blobPath + "\"\n"
+ + "echo $blobPath\n"
+ + azcopyPath + "/azcopy copy \"" + azcopyPath + "/NOTICE.txt\" $blobPath\n"; // construct the script content
+ File scriptFile = new File(folderPath, scriptName);
+ try {
+ FileWriter writer = new FileWriter(scriptFile);
+ writer.write(scriptContent);
+ writer.close();
+ boolean written = scriptFile.setExecutable(true); // make the script executable
+ System.out.println("Script created at " + scriptFile.getAbsolutePath());
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+
+ public static void createFolderCreationScript(String folderPath, String scriptName, String azcopyPath, String sasToken, String containerName) {
+ String blobPath = containerName + "?" + sasToken; // construct the blob path
+ String scriptContent = "blobPath=\"" + blobPath + "\"\n"
+ + "echo $blobPath\n"
+ + azcopyPath + "/azcopy copy \"" + azcopyPath + "\" $blobPath --recursive\n"; // construct the script content
+ File scriptFile = new File(folderPath, scriptName);
+ try {
+ FileWriter writer = new FileWriter(scriptFile);
+ writer.write(scriptContent);
+ writer.close();
+ boolean written = scriptFile.setExecutable(true); // make the script executable
+ System.out.println("Script created at " + scriptFile.getAbsolutePath());
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/BlobDirectoryStateHelper.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/BlobDirectoryStateHelper.java
new file mode 100644
index 00000000000000..6e174b36046cde
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/BlobDirectoryStateHelper.java
@@ -0,0 +1,130 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.services.BlobProperty;
+import org.apache.hadoop.fs.azurebfs.services.PrefixMode;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+
+public class BlobDirectoryStateHelper {
+
+ /**
+ * To assert that a path exists as implicit directory we need two things to assert.
+ * 1. List blobs on the path should return some entries.
+ * 2. GetBlobProperties on path should fail.
+ * @param path to be checked
+ * @param fs AzureBlobFileSystem for API calls
+ * @return boolean whether the path exists as Implicit directory or not
+ */
+ public static boolean isImplicitDirectory(Path path, AzureBlobFileSystem fs, TracingContext testTracingContext) throws Exception {
+ path = new Path(fs.makeQualified(path).toUri().getPath());
+ if (fs.getAbfsStore().getPrefixMode() == PrefixMode.BLOB) {
+ List blobProperties = fs.getAbfsStore()
+ .getListBlobs(path,null, null, testTracingContext, 2, true);
+ if (blobProperties.size() == 0) {
+ return false;
+ }
+ try {
+ fs.getAbfsStore().getBlobProperty(
+ path, testTracingContext);
+ }
+ catch (AbfsRestOperationException ex) {
+ if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) {
+ return true;
+ }
+ }
+ return false;
+ }
+ else {
+ FileStatus[] statuses = fs.getAbfsStore()
+ .listStatus(path, testTracingContext);
+ if (statuses.length == 0) {
+ return false;
+ }
+ try {
+ FileStatus status = fs.getAbfsStore().getFileStatus(
+ path, testTracingContext, true);
+ return !status.isDirectory();
+ }
+ catch (AbfsRestOperationException ex) {
+ if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ /**
+ * To assert that a path exists as explicit directory
+ * For PrefixMode Blob: GetBlobProperties on path should succeed and marker should be present
+ * For PrefixMode DFS: GetFileStatus on path should succeed and marker should be present
+ * For Root on Blob: GetContainerProperty
+ * @param path to be checked
+ * @param fs AzureBlobFileSystem for API calls
+ * @return boolean whether the path exists as Implicit directory or not
+ */
+ public static boolean isExplicitDirectory(Path path, AzureBlobFileSystem fs, TracingContext testTracingContext) {
+ path = new Path(fs.makeQualified(path).toUri().getPath());
+
+ if (fs.getAbfsStore().getPrefixMode() == PrefixMode.DFS) {
+ FileStatus status;
+ try {
+ status = fs.getAbfsStore()
+ .getFileStatus(path, testTracingContext, false);
+ }
+ catch (IOException ex) {
+ return false;
+ }
+ return status.isDirectory();
+ }
+
+ else if (path.isRoot()) {
+ BlobProperty prop;
+ try {
+ prop = fs.getAbfsStore().getContainerProperty(testTracingContext);
+ }
+ catch(AzureBlobFileSystemException ex) {
+ return false;
+ }
+ return prop.getIsDirectory();
+ }
+
+ else {
+ BlobProperty prop;
+ try {
+ prop = fs.getAbfsStore().getBlobProperty(
+ path, testTracingContext);
+ }
+ catch(AzureBlobFileSystemException ex) {
+ return false;
+ }
+ return prop.getIsDirectory();
+ }
+ }
+}
\ No newline at end of file
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsDurationTrackers.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsDurationTrackers.java
index 0997b3dbd44d4b..afd91824e38e9e 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsDurationTrackers.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsDurationTrackers.java
@@ -101,11 +101,16 @@ public void testAbfsHttpCallsDurations() throws IOException {
*/
private void assertDurationTracker(IOStatistics ioStatistics) {
for (AbfsStatistic abfsStatistic : HTTP_DURATION_TRACKER_LIST) {
- Assertions.assertThat(lookupMeanStatistic(ioStatistics,
- abfsStatistic.getStatName() + StoreStatisticNames.SUFFIX_MEAN).mean())
- .describedAs("The DurationTracker Named " + abfsStatistic.getStatName()
- + " Doesn't match the expected value.")
- .isGreaterThan(0.0);
+ // Since delete calls go to wasb endpoint and it doesn't have implementation for abfsStatistics
+ // we are avoiding incrementing stats for it. Can be reverted when we have blob implementation for
+ // delete.
+ if (!HTTP_DELETE_REQUEST.getStatName().equals(abfsStatistic.getStatName())) {
+ Assertions.assertThat(lookupMeanStatistic(ioStatistics,
+ abfsStatistic.getStatName() + StoreStatisticNames.SUFFIX_MEAN).mean())
+ .describedAs("The DurationTracker Named " + abfsStatistic.getStatName()
+ + " Doesn't match the expected value.")
+ .isGreaterThan(0.0);
+ }
}
}
}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java
index e2973912e23234..76e12b37f27455 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java
@@ -33,13 +33,18 @@
import org.junit.Test;
import org.mockito.Mockito;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator;
import org.apache.hadoop.fs.azurebfs.services.ListingSupport;
import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.ABFS_DNS_PREFIX;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.WASB_DNS_PREFIX;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.ArgumentMatchers.anyList;
@@ -344,4 +349,66 @@ private List createFilesUnderDirectory(int numFiles, Path rootPath,
return fileNames;
}
+
+ @Test
+ public void testListStatusIteratorReturnStatusWithPathWithSameUriGivenInConfig()
+ throws Exception {
+ AzureBlobFileSystem fs = getFileSystem();
+ String accountName = getAccountName();
+ Boolean isAccountNameInDfs = accountName.contains(ABFS_DNS_PREFIX);
+ String dnsAssertion;
+ if (isAccountNameInDfs) {
+ dnsAssertion = ABFS_DNS_PREFIX;
+ } else {
+ dnsAssertion = WASB_DNS_PREFIX;
+ }
+
+ final Path path = new Path("/testDir/file");
+ fs.create(path);
+ assertListStatusIteratorPath(fs, accountName, dnsAssertion, path);
+
+ final Configuration configuration;
+ if (isAccountNameInDfs) {
+ configuration = new Configuration(getRawConfiguration());
+ configuration.set(FS_DEFAULT_NAME_KEY,
+ configuration.get(FS_DEFAULT_NAME_KEY)
+ .replace(ABFS_DNS_PREFIX, WASB_DNS_PREFIX));
+ dnsAssertion = WASB_DNS_PREFIX;
+
+ } else {
+ configuration = new Configuration(getRawConfiguration());
+ configuration.set(FS_DEFAULT_NAME_KEY,
+ configuration.get(FS_DEFAULT_NAME_KEY)
+ .replace(WASB_DNS_PREFIX, ABFS_DNS_PREFIX));
+ dnsAssertion = ABFS_DNS_PREFIX;
+ }
+ fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration);
+ assertListStatusIteratorPath(fs, accountName, dnsAssertion, path);
+ }
+
+ private void assertListStatusIteratorPath(final AzureBlobFileSystem fs,
+ final String accountName,
+ final String dnsAssertion,
+ final Path path) throws IOException {
+ RemoteIterator fileStatusRemoteIterator = fs.listStatusIterator(
+ new Path(
+ "abfs://" + fs.getAbfsClient().getFileSystem() + "@" + accountName
+ + path.getParent().toUri().getPath()));
+
+ while (fileStatusRemoteIterator.hasNext()) {
+ FileStatus fileStatus = fileStatusRemoteIterator.next();
+ Assertions.assertThat(fileStatus.getPath().toString())
+ .contains(dnsAssertion);
+ }
+
+ fileStatusRemoteIterator = fs.listStatusIterator(new Path(
+ "abfs://" + fs.getAbfsClient().getFileSystem() + "@" + accountName
+ + path.toUri().getPath()));
+
+ while (fileStatusRemoteIterator.hasNext()) {
+ FileStatus fileStatus = fileStatusRemoteIterator.next();
+ Assertions.assertThat(fileStatus.getPath().toString())
+ .contains(dnsAssertion);
+ }
+ }
}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsMsiTokenProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsMsiTokenProvider.java
index d871befa430058..c93561005db7e5 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsMsiTokenProvider.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsMsiTokenProvider.java
@@ -19,15 +19,22 @@
package org.apache.hadoop.fs.azurebfs;
import java.io.IOException;
+import java.lang.reflect.Field;
+import java.net.HttpURLConnection;
import java.util.Date;
-
import org.junit.Test;
+import org.mockito.MockedStatic;
+import org.mockito.Mockito;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
+import org.apache.hadoop.fs.azurebfs.oauth2.AzureADAuthenticator;
import org.apache.hadoop.fs.azurebfs.oauth2.AzureADToken;
import org.apache.hadoop.fs.azurebfs.oauth2.MsiTokenProvider;
+import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_OAUTH_TOKEN_FETCH_RETRY_MAX_ATTEMPTS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assume.assumeThat;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.CoreMatchers.not;
@@ -40,6 +47,7 @@
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT;
import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT;
+import static org.mockito.Mockito.times;
/**
* Test MsiTokenProvider.
@@ -47,6 +55,8 @@
public final class ITestAbfsMsiTokenProvider
extends AbstractAbfsIntegrationTest {
+ private static final int HTTP_TOO_MANY_REQUESTS = 429;
+
public ITestAbfsMsiTokenProvider() throws Exception {
super();
}
@@ -90,4 +100,109 @@ private String getTrimmedPasswordString(AbfsConfiguration conf, String key,
return value.trim();
}
+ /**
+ * Test to verify that token fetch is retried for throttling errors (too many requests 429).
+ * @throws Exception
+ */
+ @Test
+ public void testRetryForThrottling() throws Exception {
+ AbfsConfiguration conf = getConfiguration();
+
+ // Exception to be thrown with throttling error code 429.
+ AzureADAuthenticator.HttpException httpException
+ = new AzureADAuthenticator.HttpException(HTTP_TOO_MANY_REQUESTS,
+ "abc", "abc", "abc", "abc", "abc");
+
+ String tenantGuid = "abcd";
+ String clientId = "abcd";
+ String authEndpoint = getTrimmedPasswordString(conf,
+ FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT,
+ DEFAULT_FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT);
+ String authority = getTrimmedPasswordString(conf,
+ FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY,
+ DEFAULT_FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY);
+
+ // Mock the getTokenSingleCall to throw exception so the retry logic comes into place.
+ try (MockedStatic adAuthenticator = Mockito.mockStatic(
+ AzureADAuthenticator.class, Mockito.CALLS_REAL_METHODS)) {
+ adAuthenticator.when(
+ () -> AzureADAuthenticator.getTokenSingleCall(Mockito.anyString(),
+ Mockito.anyString(), Mockito.any(), Mockito.anyString(),
+ Mockito.anyBoolean())).thenThrow(httpException);
+
+ // Mock the tokenFetchRetryPolicy to verify retries.
+ ExponentialRetryPolicy exponentialRetryPolicy = Mockito.spy(
+ conf.getOauthTokenFetchRetryPolicy());
+ Field tokenFetchRetryPolicy = AzureADAuthenticator.class.getDeclaredField(
+ "tokenFetchRetryPolicy");
+ tokenFetchRetryPolicy.setAccessible(true);
+ tokenFetchRetryPolicy.set(ExponentialRetryPolicy.class,
+ exponentialRetryPolicy);
+
+ AccessTokenProvider tokenProvider = new MsiTokenProvider(authEndpoint,
+ tenantGuid, clientId, authority);
+ AzureADToken token = null;
+ intercept(AzureADAuthenticator.HttpException.class,
+ tokenProvider::getToken);
+
+ // If the status code doesn't qualify for retry shouldRetry returns false and the loop ends.
+ // It being called multiple times verifies that the retry was done for the throttling status code 429.
+ Mockito.verify(exponentialRetryPolicy,
+ times(DEFAULT_AZURE_OAUTH_TOKEN_FETCH_RETRY_MAX_ATTEMPTS + 1))
+ .shouldRetry(Mockito.anyInt(), Mockito.anyInt());
+ }
+ }
+
+ /**
+ * Test to verify that token fetch is not retried for resource not found errors.
+ * @throws Exception
+ */
+ @Test
+ public void testNoRetryForResourceNotFound() throws Exception {
+ AbfsConfiguration conf = getConfiguration();
+
+ // Exception to be thrown 404 error code.
+ AzureADAuthenticator.HttpException httpException
+ = new AzureADAuthenticator.HttpException(HttpURLConnection.HTTP_NOT_FOUND,
+ "abc", "abc", "abc", "abc", "abc");
+
+ String tenantGuid = "abcd";
+ String clientId = "abcd";
+ String authEndpoint = getTrimmedPasswordString(conf,
+ FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT,
+ DEFAULT_FS_AZURE_ACCOUNT_OAUTH_MSI_ENDPOINT);
+ String authority = getTrimmedPasswordString(conf,
+ FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY,
+ DEFAULT_FS_AZURE_ACCOUNT_OAUTH_MSI_AUTHORITY);
+
+ // Mock the getTokenSingleCall to throw exception.
+ try (MockedStatic adAuthenticator = Mockito.mockStatic(
+ AzureADAuthenticator.class, Mockito.CALLS_REAL_METHODS)) {
+ adAuthenticator.when(
+ () -> AzureADAuthenticator.getTokenSingleCall(Mockito.anyString(),
+ Mockito.anyString(), Mockito.any(), Mockito.anyString(),
+ Mockito.anyBoolean())).thenThrow(httpException);
+
+ // Mock the tokenFetchRetryPolicy to verify no retries.
+ ExponentialRetryPolicy exponentialRetryPolicy = Mockito.spy(
+ conf.getOauthTokenFetchRetryPolicy());
+ Field tokenFetchRetryPolicy = AzureADAuthenticator.class.getDeclaredField(
+ "tokenFetchRetryPolicy");
+ tokenFetchRetryPolicy.setAccessible(true);
+ tokenFetchRetryPolicy.set(ExponentialRetryPolicy.class,
+ exponentialRetryPolicy);
+
+ AccessTokenProvider tokenProvider = new MsiTokenProvider(authEndpoint,
+ tenantGuid, clientId, authority);
+ AzureADToken token = null;
+ intercept(AzureADAuthenticator.HttpException.class,
+ tokenProvider::getToken);
+
+ // If the status code doesn't qualify for retry shouldRetry returns false and the loop ends.
+ // It being called only once verifies that retry doesn't come into place..
+ Mockito.verify(exponentialRetryPolicy,
+ times(1))
+ .shouldRetry(Mockito.anyInt(), Mockito.anyInt());
+ }
+ }
}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java
index 66b8da89572a1e..e7bed75f93143f 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java
@@ -21,6 +21,8 @@
import java.io.IOException;
import java.util.Map;
+import org.apache.hadoop.fs.azurebfs.services.OperativeEndpoint;
+import org.apache.hadoop.fs.azurebfs.services.PrefixMode;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.junit.Test;
@@ -127,8 +129,11 @@ public void testAbfsHttpSendStatistics() throws IOException {
// Operation: AbfsOutputStream close.
// Network Stats calculation: 1 flush (with close) is send.
// 1 flush request = 1 connection and 1 send request
- expectedConnectionsMade++;
- expectedRequestsSent++;
+ if (OperativeEndpoint.isIngressEnabledOnDFS(getPrefixMode(fs), fs.getAbfsStore().getAbfsConfiguration())) {
+ // No network call made for flush without append in case of blob endpoint.
+ expectedConnectionsMade++;
+ expectedRequestsSent++;
+ }
// --------------------------------------------------------------------
// Operation: Re-create the file / create overwrite scenario
@@ -181,7 +186,6 @@ public void testAbfsHttpSendStatistics() throws IOException {
assertAbfsStatistics(SEND_REQUESTS, expectedRequestsSent, metricMap);
assertAbfsStatistics(AbfsStatistic.BYTES_SENT, expectedBytesSent, metricMap);
}
-
}
/**
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java
index f4f0f231037e18..beada775ae87b3 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java
@@ -32,11 +32,14 @@
import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_INFO;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.APPENDBLOB_MAX_WRITE_BUFFER_SIZE;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_BUFFER_SIZE;
import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtLevel;
/**
* Test read, write and seek.
@@ -45,20 +48,29 @@
*/
@RunWith(Parameterized.class)
public class ITestAbfsReadWriteAndSeek extends AbstractAbfsScaleTest {
- private static final Path TEST_PATH = new Path("/testfile");
-
- @Parameterized.Parameters(name = "Size={0}")
+ private static final String TEST_PATH = "/testfile";
+
+ /**
+ * Parameterize on read buffer size and readahead.
+ * For test performance, a full x*y test matrix is not used.
+ * @return the test parameters
+ */
+ @Parameterized.Parameters(name = "Size={0}-readahead={1}")
public static Iterable