From e2ffb8c04f895baec1db5bf000e77a8315342aa4 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Mon, 28 Feb 2022 14:24:37 +0530 Subject: [PATCH 01/53] ABFS: Added changes for expect hundred continue header with append requests --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 13 ++++ .../fs/azurebfs/AzureBlobFileSystem.java | 2 - .../azurebfs/constants/AbfsHttpConstants.java | 1 + .../azurebfs/constants/ConfigurationKeys.java | 2 + .../constants/FileSystemConfigurations.java | 3 +- .../constants/HttpHeaderConfigurations.java | 1 + .../InvalidAbfsRestOperationException.java | 30 ++++++--- .../fs/azurebfs/services/AbfsClient.java | 13 ++++ .../AbfsClientThrottlingAnalyzer.java | 2 +- .../AbfsClientThrottlingIntercept.java | 27 +++++--- .../AbfsClientThrottlingInterceptFactory.java | 33 ++++++++++ .../azurebfs/services/AbfsHttpOperation.java | 7 +- .../azurebfs/services/AbfsRestOperation.java | 14 ++-- .../ITestAzureBlobFileSystemCreate.java | 65 +++++++++++++++++++ .../constants/TestConfigurationKeys.java | 1 + 15 files changed, 187 insertions(+), 27 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 9719da7dc168a..d7c5c64e31e0e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -117,6 +117,15 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_OPTIMIZE_FOOTER_READ) private boolean optimizeFooterRead; + @BooleanConfigurationValidatorAnnotation( + ConfigurationKey = FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED, + DefaultValue = DEFAULT_FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED) + private boolean isExpectHeaderEnabled; + + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ACCOUNT_IS_SINGLETON_ENABLED, + DefaultValue = DEFAULT_FS_AZURE_ACCOUNT_IS_SINGLETON_ENABLED) + private boolean isSingletonEnabled; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_READ_BUFFER_SIZE, MinValue = MIN_BUFFER_SIZE, MaxValue = MAX_BUFFER_SIZE, @@ -684,6 +693,10 @@ public String getAppendBlobDirs() { return this.azureAppendBlobDirs; } + public boolean isExpectHeaderEnabled() { return this.isExpectHeaderEnabled; } + + public boolean isSingletonEnabled() {return this.isSingletonEnabled; } + public String getAzureInfiniteLeaseDirs() { return this.azureInfiniteLeaseDirs; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 47866140145e9..474ee846fcb4e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -214,8 +214,6 @@ public void initialize(URI uri, Configuration configuration) } } - AbfsClientThrottlingIntercept.initializeSingleton(abfsConfiguration.isAutoThrottlingEnabled()); - LOG.debug("Initializing AzureBlobFileSystem for {} complete", uri); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java index 5cf7ec565b59e..e82f38bd5f068 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java @@ -103,6 +103,7 @@ public final class AbfsHttpConstants { public static final String DEFAULT_SCOPE = "default:"; public static final String PERMISSION_FORMAT = "%04d"; public static final String SUPER_USER = "$superuser"; + public static final String HUNDRED_CONTINUE = "100-continue"; public static final char CHAR_FORWARD_SLASH = '/'; public static final char CHAR_EXCLAMATION_POINT = '!'; 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 12beb5a9bbabe..b69617643d18c 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,6 +35,8 @@ public final class ConfigurationKeys { * path to determine HNS status. */ public static final String FS_AZURE_ACCOUNT_IS_HNS_ENABLED = "fs.azure.account.hns.enabled"; + public static final String FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED = "fs.azure.account.expect.header.enabled"; + public static final String FS_AZURE_ACCOUNT_IS_SINGLETON_ENABLED = "fs.azure.account.singleton.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"; 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 f58c61e8908a6..433acbaa4f244 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,8 @@ 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 boolean DEFAULT_FS_AZURE_ACCOUNT_IS_SINGLETON_ENABLED = false; public static final String USER_HOME_DIRECTORY_PREFIX = "/user"; private static final int SIXTY_SECONDS = 60 * 1000; 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 d4065ac2836d0..ec68f1f923680 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 @@ -60,6 +60,7 @@ public final class HttpHeaderConfigurations { public static final String X_MS_UMASK = "x-ms-umask"; public static final String X_MS_NAMESPACE_ENABLED = "x-ms-namespace-enabled"; public static final String X_MS_ABFS_CLIENT_LATENCY = "x-ms-abfs-client-latency"; + public static final String EXPECT = "Expect"; public static final String X_MS_ENCRYPTION_KEY = "x-ms-encryption-key"; public static final String X_MS_ENCRYPTION_KEY_SHA256 = "x-ms-encryption-key-sha256"; public static final String X_MS_ENCRYPTION_ALGORITHM = "x-ms-encryption-algorithm"; 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 196202121343a..cce2ad1d2ca90 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 @@ -30,14 +30,24 @@ @InterfaceAudience.Public @InterfaceStability.Evolving public class InvalidAbfsRestOperationException extends AbfsRestOperationException { - public InvalidAbfsRestOperationException( - final Exception innerException) { - super( - AzureServiceErrorCode.UNKNOWN.getStatusCode(), - AzureServiceErrorCode.UNKNOWN.getErrorCode(), - innerException != null - ? innerException.toString() - : "InvalidAbfsRestOperationException", - innerException); - } + public InvalidAbfsRestOperationException( + final Exception innerException) { + super( + AzureServiceErrorCode.UNKNOWN.getStatusCode(), + AzureServiceErrorCode.UNKNOWN.getErrorCode(), + innerException != null + ? innerException.toString() + : "InvalidAbfsRestOperationException", + innerException); + } + + public InvalidAbfsRestOperationException(final Exception innerException, int retryCount) { + super( + AzureServiceErrorCode.UNKNOWN.getStatusCode(), + AzureServiceErrorCode.UNKNOWN.getErrorCode(), + innerException != null + ? innerException.toString() + : "InvalidAbfsRestOperationException" + "RetryCount: " + String.valueOf(retryCount), + innerException); + } } 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 08142a17a8aa1..7c6734acaad65 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 @@ -96,6 +96,7 @@ public class AbfsClient implements Closeable { private AccessTokenProvider tokenProvider; private SASTokenProvider sasTokenProvider; private final AbfsCounters abfsCounters; + private boolean isExpectHeaderEnabled; private final ListeningScheduledExecutorService executorService; @@ -111,6 +112,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.isExpectHeaderEnabled = abfsConfiguration.isExpectHeaderEnabled(); String encryptionKey = this.abfsConfiguration .getClientProvidedEncryptionKey(); @@ -517,6 +519,9 @@ public AbfsRestOperation append(final String path, final byte[] buffer, addCustomerProvidedKeyHeaders(requestHeaders); // 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. + if (isExpectHeaderEnabled == true) { + requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); + } requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); if (reqParams.getLeaseId() != null) { @@ -1138,6 +1143,14 @@ protected AbfsCounters getAbfsCounters() { return abfsCounters; } + /** + * Getter for abfsConfiguration from AbfsClient. + * @return AbfsConfiguration instance + */ + protected AbfsConfiguration getAbfsConfiguration() { + return abfsConfiguration; + } + public int getNumLeaseThreads() { return abfsConfiguration.getNumLeaseThreads(); } 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 6dfd352954d34..e44ae052158d7 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 @@ -256,7 +256,7 @@ public void run() { /** * Stores Abfs operation metrics during each analysis period. */ - static class AbfsOperationMetrics { + class AbfsOperationMetrics { private AtomicLong bytesFailed; private AtomicLong bytesSuccessful; private AtomicLong operationsFailed; 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 7303e833418db..bf83bdc6df32e 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 @@ -46,6 +46,16 @@ public final class AbfsClientThrottlingIntercept { private AbfsClientThrottlingAnalyzer readThrottler = null; private AbfsClientThrottlingAnalyzer writeThrottler = null; private static boolean isAutoThrottlingEnabled = false; + private String accountName = ""; + + // Hide default constructor + public AbfsClientThrottlingIntercept(String accountName) { + isAutoThrottlingEnabled = true; + LOG.debug("Client-side throttling is enabled for the ABFS file system."); + this.readThrottler = new AbfsClientThrottlingAnalyzer("read"); + this.writeThrottler = new AbfsClientThrottlingAnalyzer("write"); + this.accountName = accountName; + } // Hide default constructor private AbfsClientThrottlingIntercept() { @@ -53,18 +63,19 @@ private AbfsClientThrottlingIntercept() { writeThrottler = new AbfsClientThrottlingAnalyzer("write"); } - public static synchronized void initializeSingleton(boolean enableAutoThrottling) { + public static synchronized AbfsClientThrottlingIntercept initializeSingleton(boolean enableAutoThrottling) { if (!enableAutoThrottling) { - return; + return null; } if (singleton == null) { singleton = new AbfsClientThrottlingIntercept(); isAutoThrottlingEnabled = true; LOG.debug("Client-side throttling is enabled for the ABFS file system."); } + return singleton; } - static void updateMetrics(AbfsRestOperationType operationType, + void updateMetrics(AbfsRestOperationType operationType, AbfsHttpOperation abfsHttpOperation) { if (!isAutoThrottlingEnabled || abfsHttpOperation == null) { return; @@ -82,7 +93,7 @@ static void updateMetrics(AbfsRestOperationType operationType, case Append: contentLength = abfsHttpOperation.getBytesSent(); if (contentLength > 0) { - singleton.writeThrottler.addBytesTransferred(contentLength, + this.writeThrottler.addBytesTransferred(contentLength, isFailedOperation); } break; @@ -90,7 +101,7 @@ static void updateMetrics(AbfsRestOperationType operationType, String range = abfsHttpOperation.getConnection().getRequestProperty(HttpHeaderConfigurations.RANGE); contentLength = getContentLengthIfKnown(range); if (contentLength > 0) { - singleton.readThrottler.addBytesTransferred(contentLength, + this.readThrottler.addBytesTransferred(contentLength, isFailedOperation); } break; @@ -104,7 +115,7 @@ static void updateMetrics(AbfsRestOperationType operationType, * uses this to suspend the request, if necessary, to minimize errors and * maximize throughput. */ - static void sendingRequest(AbfsRestOperationType operationType, + void sendingRequest(AbfsRestOperationType operationType, AbfsCounters abfsCounters) { if (!isAutoThrottlingEnabled) { return; @@ -112,13 +123,13 @@ static void sendingRequest(AbfsRestOperationType operationType, switch (operationType) { case ReadFile: - if (singleton.readThrottler.suspendIfNecessary() + if (this.readThrottler.suspendIfNecessary() && abfsCounters != null) { abfsCounters.incrementCounter(AbfsStatistic.READ_THROTTLES, 1); } break; case Append: - if (singleton.writeThrottler.suspendIfNecessary() + if (this.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/AbfsClientThrottlingInterceptFactory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java new file mode 100644 index 0000000000000..31edada643a04 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java @@ -0,0 +1,33 @@ +package org.apache.hadoop.fs.azurebfs.services; + +import org.apache.hadoop.fs.azurebfs.Abfs; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class AbfsClientThrottlingInterceptFactory { + private static final Logger LOG = LoggerFactory.getLogger(AbfsClientThrottlingInterceptFactory.class); + private static Map instanceMapping = new ConcurrentHashMap<>(); + + public static synchronized AbfsClientThrottlingIntercept getInstance(String accountName, boolean isAutoThrottlingEnabled, + boolean isSingletonEnabled) { + AbfsClientThrottlingIntercept instance; + if (isSingletonEnabled) { + instance = AbfsClientThrottlingIntercept.initializeSingleton(isAutoThrottlingEnabled); + } else { + if (!isAutoThrottlingEnabled) { + return null; + } + if (instanceMapping.get(accountName) == null) { + LOG.debug("The accountNameis: {} ", accountName); + instance = new AbfsClientThrottlingIntercept(accountName); + instanceMapping.put(accountName, instance); + } else { + instance = instanceMapping.get(accountName); + } + } + return instance; + } +} 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 413bf3686898b..d5da47847c2cd 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 @@ -320,7 +320,12 @@ public void sendRequest(byte[] buffer, int offset, int length) throws IOExceptio // accompanying statusCode this.bytesSent = length; outputStream.write(buffer, offset, length); - } finally { + } catch (Exception e) { + this.statusCode = this.connection.getResponseCode(); + this.bytesSent = length; + throw new IOException(e); + } + finally { if (this.isTraceEnabled) { this.sendRequestTimeMs = elapsedTimeMs(startTime); } 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 74b267d563eb2..67fa7653bd213 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 @@ -242,6 +242,10 @@ private void completeExecute(TracingContext tracingContext) private boolean executeHttpOperation(final int retryCount, TracingContext tracingContext) throws AzureBlobFileSystemException { AbfsHttpOperation httpOperation = null; + String accountName = this.client.getAbfsConfiguration().getAccountName(); + boolean isAutoThrottlingEnabled = this.client.getAbfsConfiguration().isAutoThrottlingEnabled(); + boolean isSingletonEnabled = this.client.getAbfsConfiguration().isSingletonEnabled(); + AbfsClientThrottlingIntercept intercept; try { // initialize the HTTP request and open the connection httpOperation = new AbfsHttpOperation(url, method, requestHeaders); @@ -278,7 +282,8 @@ private boolean executeHttpOperation(final int retryCount, // dump the headers AbfsIoUtils.dumpHeadersToDebugLog("Request Headers", httpOperation.getConnection().getRequestProperties()); - AbfsClientThrottlingIntercept.sendingRequest(operationType, abfsCounters); + intercept = AbfsClientThrottlingInterceptFactory.getInstance(accountName, isAutoThrottlingEnabled, isSingletonEnabled); + intercept.sendingRequest(operationType, abfsCounters); if (hasRequestBody) { // HttpUrlConnection requires @@ -303,7 +308,7 @@ private boolean executeHttpOperation(final int retryCount, 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) { @@ -312,12 +317,13 @@ private boolean executeHttpOperation(final int retryCount, } if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) { - throw new InvalidAbfsRestOperationException(ex); + throw new InvalidAbfsRestOperationException(ex,retryCount); } return false; } finally { - AbfsClientThrottlingIntercept.updateMetrics(operationType, httpOperation); + AbfsClientThrottlingIntercept instance = AbfsClientThrottlingInterceptFactory.getInstance(accountName, isAutoThrottlingEnabled, isSingletonEnabled); + instance.updateMetrics(operationType, httpOperation); } LOG.debug("HttpRequest: {}: {}", operationType, httpOperation); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 2f23ac5c5c708..df7ccab0bd4d8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -22,6 +22,7 @@ import java.io.FilterOutputStream; import java.io.IOException; import java.lang.reflect.Field; +import java.net.URI; import java.util.EnumSet; import java.util.UUID; @@ -36,6 +37,8 @@ import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.test.GenericTestUtils; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*; +import static org.junit.Assume.assumeTrue; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; @@ -43,6 +46,8 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientThrottlingIntercept; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientThrottlingInterceptFactory; import org.apache.hadoop.fs.azurebfs.services.TestAbfsClient; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; @@ -475,6 +480,66 @@ private AzureBlobFileSystemStore setAzureBlobSystemStoreField( return abfsStore; } + @Test + public void testCreatMultipleAccountThrottling() + throws Throwable { + final AzureBlobFileSystem currentFs = getFileSystem(); + Configuration config = new Configuration(this.getRawConfiguration()); + String fileSystemName = TEST_CONTAINER_PREFIX + UUID.randomUUID().toString(); + String accountName = config.get(FS_AZURE_ACCOUNT_NAME); + if (accountName == null) { + // check if accountName is set using different config key + accountName = config.get(FS_AZURE_ABFS_ACCOUNT1_NAME); + } + assumeTrue("Not set: " + FS_AZURE_ABFS_ACCOUNT1_NAME, + accountName != null && !accountName.isEmpty()); + + final String abfsUrl = fileSystemName + "@" + accountName; + URI defaultUri = null; + String abfsScheme = "abfss"; + try { + defaultUri = new URI(abfsScheme, abfsUrl, null, null, null); + } catch (Exception ex) { + throw new AssertionError(ex); + } + + Configuration rawConfig1 = new Configuration(); + rawConfig1.addResource(TEST_CONFIGURATION_FILE_NAME); + + AbfsConfiguration abfsConfig = new AbfsConfiguration(rawConfig1, accountName); + + final AzureBlobFileSystem fs = + (AzureBlobFileSystem) FileSystem.newInstance(defaultUri, + rawConfig1); + System.out.println(fs.toString()); + + AbfsClient mockClient = TestAbfsClient.getMockAbfsClient(fs.getAbfsStore().getClient(), fs.getAbfsStore().getAbfsConfiguration()); + + AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); + abfsStore = setAzureBlobSystemStoreField(abfsStore, "client", mockClient); + + AbfsRestOperation successOp = mock(AbfsRestOperation.class); + AbfsHttpOperation http500Op = mock(AbfsHttpOperation.class); + when(http500Op.getStatusCode()).thenReturn(HTTP_INTERNAL_ERROR); + when(successOp.getResult()).thenReturn(http500Op); + + AbfsClientThrottlingIntercept instance1 = AbfsClientThrottlingInterceptFactory.getInstance(accountName, true, true); + String accountName1 = config.get(FS_AZURE_ABFS_ACCOUNT1_NAME); + AbfsClientThrottlingIntercept instance2 = AbfsClientThrottlingInterceptFactory.getInstance(accountName1, true, true); + //if singleton is enabled, for different accounts both the instances should return same value + assertEquals(instance1, instance2); + + AbfsClientThrottlingIntercept instance3 = AbfsClientThrottlingInterceptFactory.getInstance(accountName, true, false); + AbfsClientThrottlingIntercept instance4 = AbfsClientThrottlingInterceptFactory.getInstance(accountName1, true, false); + AbfsClientThrottlingIntercept instance5 = AbfsClientThrottlingInterceptFactory.getInstance(accountName, true, false); + + //if singleton is not enabled, for different accounts instances should return different value + assertNotEquals(instance3, instance4); + + //if singleton is not enabled, for same accounts instances should return same value + assertEquals(instance3, instance5); + } + private void validateCreateFileException(final Class exceptionClass, final AzureBlobFileSystemStore abfsStore) throws Exception { FsPermission permission = new FsPermission(FsAction.ALL, FsAction.ALL, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java index 565eb38c4f70a..2c659e1399a4f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java @@ -24,6 +24,7 @@ public final class TestConfigurationKeys { public static final String FS_AZURE_ACCOUNT_NAME = "fs.azure.account.name"; public static final String FS_AZURE_ABFS_ACCOUNT_NAME = "fs.azure.abfs.account.name"; + public static final String FS_AZURE_ABFS_ACCOUNT1_NAME = "fs.azure.abfs.account1.name"; public static final String FS_AZURE_ACCOUNT_KEY = "fs.azure.account.key"; public static final String FS_AZURE_CONTRACT_TEST_URI = "fs.contract.test.fs.abfs"; public static final String FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT = "fs.azure.test.namespace.enabled"; From d1104082763833912b995afabbe3e67b86324f0d Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Wed, 2 Mar 2022 11:17:15 +0530 Subject: [PATCH 02/53] ABFS: Added changes for expect hundred continue --- .../AbfsClientThrottlingInterceptFactory.java | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java index 31edada643a04..2a5895d09127a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.fs.azurebfs.services; import org.apache.hadoop.fs.azurebfs.Abfs; @@ -21,7 +39,7 @@ public static synchronized AbfsClientThrottlingIntercept getInstance(String acco return null; } if (instanceMapping.get(accountName) == null) { - LOG.debug("The accountNameis: {} ", accountName); + LOG.debug("The accountName is: {} ", accountName); instance = new AbfsClientThrottlingIntercept(accountName); instanceMapping.put(accountName, instance); } else { From a29faa863ee3c515b92388b97608ff35996bf913 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Wed, 2 Mar 2022 11:40:02 +0530 Subject: [PATCH 03/53] ABFS: Added changes for expect hundred continue --- .../org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java | 8 ++++++-- .../apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java | 1 - .../apache/hadoop/fs/azurebfs/services/AbfsClient.java | 2 +- .../services/AbfsClientThrottlingInterceptFactory.java | 1 - .../hadoop/fs/azurebfs/services/AbfsRestOperation.java | 2 +- .../fs/azurebfs/ITestAzureBlobFileSystemCreate.java | 6 +++++- 6 files changed, 13 insertions(+), 7 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index d7c5c64e31e0e..64c05b35e1805 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -693,9 +693,13 @@ public String getAppendBlobDirs() { return this.azureAppendBlobDirs; } - public boolean isExpectHeaderEnabled() { return this.isExpectHeaderEnabled; } + public boolean isExpectHeaderEnabled() { + return this.isExpectHeaderEnabled; + } - public boolean isSingletonEnabled() {return this.isSingletonEnabled; } + public boolean isSingletonEnabled() { + return this.isSingletonEnabled; + } public String getAzureInfiniteLeaseDirs() { return this.azureInfiniteLeaseDirs; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 474ee846fcb4e..dfdcf847fecbb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -50,7 +50,6 @@ import org.apache.commons.lang3.ArrayUtils; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; -import org.apache.hadoop.fs.azurebfs.services.AbfsClientThrottlingIntercept; import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.classification.InterfaceStability; 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 7c6734acaad65..c7cc2573d2ff0 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 @@ -519,7 +519,7 @@ public AbfsRestOperation append(final String path, final byte[] buffer, addCustomerProvidedKeyHeaders(requestHeaders); // 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. - if (isExpectHeaderEnabled == true) { + if (isExpectHeaderEnabled) { requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); } requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java index 2a5895d09127a..2931f211443e5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java @@ -18,7 +18,6 @@ package org.apache.hadoop.fs.azurebfs.services; -import org.apache.hadoop.fs.azurebfs.Abfs; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 67fa7653bd213..3a51d201d6901 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 @@ -317,7 +317,7 @@ private boolean executeHttpOperation(final int retryCount, } if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) { - throw new InvalidAbfsRestOperationException(ex,retryCount); + throw new InvalidAbfsRestOperationException(ex, retryCount); } return false; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index df7ccab0bd4d8..76e783ddf7325 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -37,7 +37,11 @@ import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.test.GenericTestUtils; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*; + +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONTAINER_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT1_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_NAME; import static org.junit.Assume.assumeTrue; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; From dacfde02fbd9b68a80470e73f0b7b0e6a6104ceb Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Wed, 2 Mar 2022 15:50:02 +0530 Subject: [PATCH 04/53] ABFS: Added changes for expect hundred continue --- .../fs/azurebfs/services/AbfsHttpOperation.java | 5 +++-- .../fs/azurebfs/ITestAzureBlobFileSystemDelete.java | 11 ++++++++--- .../hadoop/fs/azurebfs/services/TestAbfsClient.java | 2 +- 3 files changed, 12 insertions(+), 6 deletions(-) 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 d5da47847c2cd..70659f9101395 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 @@ -323,7 +323,9 @@ public void sendRequest(byte[] buffer, int offset, int length) throws IOExceptio } catch (Exception e) { this.statusCode = this.connection.getResponseCode(); this.bytesSent = length; - throw new IOException(e); + if (this.statusCode != -1 && this.statusCode >= HttpURLConnection.HTTP_INTERNAL_ERROR) { + throw new IOException(e); + } } finally { if (this.isTraceEnabled) { @@ -348,7 +350,6 @@ public void processResponse(final byte[] buffer, final int offset, final int len if (this.isTraceEnabled) { startTime = System.nanoTime(); } - this.statusCode = this.connection.getResponseCode(); if (this.isTraceEnabled) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index db181fb5dd660..35426980ccdcd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -221,11 +221,16 @@ public void testDeleteIdempotency() throws Exception { @Test public void testDeleteIdempotencyTriggerHttp404() throws Exception { + AbfsConfiguration abfsConfig + = TestAbfsConfigurationFieldsValidation.updateRetryConfigs( + getConfiguration(), + REDUCED_RETRY_COUNT, REDUCED_MAX_BACKOFF_INTERVALS_MS); final AzureBlobFileSystem fs = getFileSystem(); + AbfsClient abfsClient = fs.getAbfsStore().getClient(); AbfsClient client = TestAbfsClient.createTestClientFromCurrentContext( - fs.getAbfsStore().getClient(), - this.getConfiguration()); + abfsClient, + abfsConfig); // Case 1: Not a retried case should throw error back // Add asserts at AzureBlobFileSystemStore and AbfsClient levels @@ -244,7 +249,7 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { // mock idempotency check to mimic retried case AbfsClient mockClient = TestAbfsClient.getMockAbfsClient( fs.getAbfsStore().getClient(), - this.getConfiguration()); + abfsConfig); AzureBlobFileSystemStore mockStore = mock(AzureBlobFileSystemStore.class); mockStore = TestMockHelpers.setClassField(AzureBlobFileSystemStore.class, mockStore, "client", mockClient); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index a725bf3175a5c..68907d20b02fb 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -306,7 +306,7 @@ public static AbfsClient getMockAbfsClient(AbfsClient baseAbfsClientInstance, when(client.getAccessToken()).thenCallRealMethod(); when(client.getSharedKeyCredentials()).thenCallRealMethod(); when(client.createDefaultHeaders()).thenCallRealMethod(); - + when(client.getAbfsConfiguration()).thenReturn(abfsConfig); // override baseurl client = TestAbfsClient.setAbfsClientField(client, "abfsConfiguration", abfsConfig); From c14f4586f49606e2896b1b357fffaed2ce0cc14b Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Fri, 4 Mar 2022 10:45:29 +0530 Subject: [PATCH 05/53] Added retry mechanism for certain HTTP errors --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 1 + .../services/AppendRequestParameters.java | 12 +++++- .../fs/azurebfs/services/AbfsClient.java | 11 ++++-- .../azurebfs/services/AbfsHttpOperation.java | 8 +--- .../azurebfs/services/AbfsOutputStream.java | 6 ++- .../services/AbfsOutputStreamContext.java | 11 ++++++ .../azurebfs/services/AbfsRestOperation.java | 1 - .../fs/azurebfs/ITestCustomerProvidedKey.java | 4 +- .../services/TestAbfsOutputStream.java | 37 ++++++++++++------- 9 files changed, 61 insertions(+), 30 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index f4f895996447c..770d9449e3b12 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -691,6 +691,7 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext( return new AbfsOutputStreamContext(abfsConfiguration.getSasTokenRenewPeriodForStreamsInSeconds()) .withWriteBufferSize(bufferSize) .enableFlush(abfsConfiguration.isFlushEnabled()) + .enableExpectHeader(abfsConfiguration.isExpectHeaderEnabled()) .enableSmallWriteOptimization(abfsConfiguration.isSmallWriteOptimizationEnabled()) .disableOutputStreamFlush(abfsConfiguration.isOutputStreamFlushDisabled()) .withStreamStatistics(new AbfsOutputStreamStatisticsImpl()) 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 7369bfaf56422..579535a16f926 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,21 @@ public enum Mode { private final Mode mode; private final boolean isAppendBlob; private final String leaseId; + private boolean isExpectHeaderEnabled; public AppendRequestParameters(final long position, final int offset, final int length, final Mode mode, final boolean isAppendBlob, - final String leaseId) { + final String leaseId, boolean isExpectHeaderEnabled) { this.position = position; this.offset = offset; this.length = length; this.mode = mode; this.isAppendBlob = isAppendBlob; this.leaseId = leaseId; + this.isExpectHeaderEnabled = isExpectHeaderEnabled; } public long getPosition() { @@ -72,4 +74,12 @@ public boolean isAppendBlob() { public String getLeaseId() { return this.leaseId; } + + public boolean getIsExpectHeaderEnabled() { + return isExpectHeaderEnabled; + } + + public void setExpectHeaderEnabled(boolean expectHeaderEnabled) { + isExpectHeaderEnabled = expectHeaderEnabled; + } } 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 c7cc2573d2ff0..f6844c20ae46e 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 @@ -38,6 +38,7 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback; @@ -96,7 +97,6 @@ public class AbfsClient implements Closeable { private AccessTokenProvider tokenProvider; private SASTokenProvider sasTokenProvider; private final AbfsCounters abfsCounters; - private boolean isExpectHeaderEnabled; private final ListeningScheduledExecutorService executorService; @@ -112,7 +112,6 @@ 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.isExpectHeaderEnabled = abfsConfiguration.isExpectHeaderEnabled(); String encryptionKey = this.abfsConfiguration .getClientProvidedEncryptionKey(); @@ -519,7 +518,7 @@ public AbfsRestOperation append(final String path, final byte[] buffer, addCustomerProvidedKeyHeaders(requestHeaders); // 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. - if (isExpectHeaderEnabled) { + if (reqParams.getIsExpectHeaderEnabled()) { requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); } requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, @@ -559,6 +558,12 @@ public AbfsRestOperation append(final String path, final byte[] buffer, op.execute(tracingContext); } catch (AzureBlobFileSystemException e) { // If we have no HTTP response, throw the original exception. + if ((((AbfsRestOperationException) e).getStatusCode() >= HttpURLConnection.HTTP_BAD_REQUEST && + ((AbfsRestOperationException) e).getStatusCode() < HttpURLConnection.HTTP_INTERNAL_ERROR) && + reqParams.getIsExpectHeaderEnabled()) { + reqParams.setExpectHeaderEnabled(false); + this.append(path, buffer, reqParams, cachedSasToken, tracingContext); + } if (!op.hasResult()) { throw e; } 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 70659f9101395..b3b0a9fed1298 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 @@ -266,7 +266,6 @@ public AbfsHttpOperation(final URL url, final String method, final List= HttpURLConnection.HTTP_INTERNAL_ERROR) { - throw new IOException(e); - } - } - finally { + } finally { if (this.isTraceEnabled) { this.sendRequestTimeMs = elapsedTimeMs(startTime); } 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 620616b993fe2..82e20ce5b7684 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 @@ -80,6 +80,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; @@ -133,6 +134,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 @@ -327,7 +329,7 @@ private void uploadBlockAsync(DataBlocks.DataBlock blockToUpload, * leaseId - The AbfsLeaseId for this request. */ AppendRequestParameters reqParams = new AppendRequestParameters( - offset, 0, bytesLength, mode, false, leaseId); + offset, 0, bytesLength, mode, false, leaseId, isExpectHeaderEnabled); AbfsRestOperation op = client.append(path, blockUploadData.toByteArray(), reqParams, cachedSasToken.get(), new TracingContext(tracingContext)); @@ -573,7 +575,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()); 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 ad303823e0c68..ed89733036741 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; @@ -78,6 +80,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; @@ -184,6 +191,10 @@ public boolean isEnableFlush() { return enableFlush; } + public boolean isExpectHeaderEnabled() { + return enableExpectHeader; + } + public boolean isDisableOutputStreamFlush() { return disableOutputStreamFlush; } 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 3a51d201d6901..5e69dd3a553b6 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 @@ -333,7 +333,6 @@ private boolean executeHttpOperation(final int retryCount, } result = httpOperation; - return true; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java index 2198a6ab35852..a6e4c5cc340a9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java @@ -203,7 +203,7 @@ public void testAppendWithCPK() throws Exception { // Trying to append with correct CPK headers AppendRequestParameters appendRequestParameters = new AppendRequestParameters( - 0, 0, 5, Mode.APPEND_MODE, false, null); + 0, 0, 5, Mode.APPEND_MODE, false, null, true); byte[] buffer = getRandomBytesArray(5); AbfsClient abfsClient = fs.getAbfsClient(); AbfsRestOperation abfsRestOperation = abfsClient @@ -248,7 +248,7 @@ public void testAppendWithoutCPK() throws Exception { // Trying to append without CPK headers AppendRequestParameters appendRequestParameters = new AppendRequestParameters( - 0, 0, 5, Mode.APPEND_MODE, false, null); + 0, 0, 5, Mode.APPEND_MODE, false, null, true); byte[] buffer = getRandomBytesArray(5); AbfsClient abfsClient = fs.getAbfsClient(); AbfsRestOperation abfsRestOperation = abfsClient diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index 0673e387bfbf0..e26ba938cf5db 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -72,6 +72,7 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext( boolean isFlushEnabled, boolean disableOutputStreamFlush, boolean isAppendBlob, + boolean isExpectHeaderEnabled, AbfsClient client, String path, TracingContext tracingContext, @@ -89,6 +90,7 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext( return new AbfsOutputStreamContext(2) .withWriteBufferSize(writeBufferSize) + .enableExpectHeader(isExpectHeaderEnabled) .enableFlush(isFlushEnabled) .disableOutputStreamFlush(disableOutputStreamFlush) .withStreamStatistics(new AbfsOutputStreamStatisticsImpl()) @@ -129,6 +131,7 @@ public void verifyShortWriteRequest() throws Exception { true, false, false, + true, client, PATH, new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", @@ -149,9 +152,9 @@ public void verifyShortWriteRequest() throws Exception { out.hsync(); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, WRITE_SIZE, APPEND_MODE, false, null); + 0, 0, WRITE_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - WRITE_SIZE, 0, 2 * WRITE_SIZE, APPEND_MODE, false, null); + WRITE_SIZE, 0, 2 * WRITE_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), @@ -190,6 +193,7 @@ public void verifyWriteRequest() throws Exception { true, false, false, + true, client, PATH, tracingContext, @@ -203,9 +207,9 @@ public void verifyWriteRequest() throws Exception { out.close(); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, 5*WRITE_SIZE-BUFFER_SIZE, APPEND_MODE, false, null); + BUFFER_SIZE, 0, 5*WRITE_SIZE-BUFFER_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), @@ -264,6 +268,7 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { true, false, false, + true, client, PATH, tracingContext, @@ -277,9 +282,9 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { out.close(); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); @@ -335,6 +340,7 @@ public void verifyWriteRequestOfBufferSize() throws Exception { true, false, false, + true, client, PATH, new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", @@ -350,9 +356,9 @@ public void verifyWriteRequestOfBufferSize() throws Exception { Thread.sleep(1000); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); @@ -390,6 +396,7 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { true, false, true, + true, client, PATH, new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", @@ -405,9 +412,9 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { Thread.sleep(1000); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, true, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, true, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, true, null); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, true, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); @@ -449,6 +456,7 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { true, false, false, + true, client, PATH, new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", @@ -464,9 +472,9 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { out.hflush(); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); @@ -518,6 +526,7 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { true, false, false, + true, client, PATH, new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", @@ -535,9 +544,9 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { Thread.sleep(1000); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); From 7e40f0789616d5e2b78c66d4ef7afa270066f2d8 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Fri, 4 Mar 2022 11:08:15 +0530 Subject: [PATCH 06/53] Added retry mechanism for certain HTTP errors --- .../hadoop/fs/azurebfs/services/AbfsClient.java | 5 ++++- .../fs/azurebfs/ITestAzureBlobFileSystemDelete.java | 12 +++--------- 2 files changed, 7 insertions(+), 10 deletions(-) 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 f6844c20ae46e..9bb8b32b2d492 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 @@ -557,13 +557,16 @@ public AbfsRestOperation append(final String path, final byte[] buffer, try { op.execute(tracingContext); } catch (AzureBlobFileSystemException e) { - // If we have no HTTP response, throw the original exception. + /* + If the http response code indicates a user error we retry the same append request with expect header disabled + */ if ((((AbfsRestOperationException) e).getStatusCode() >= HttpURLConnection.HTTP_BAD_REQUEST && ((AbfsRestOperationException) e).getStatusCode() < HttpURLConnection.HTTP_INTERNAL_ERROR) && reqParams.getIsExpectHeaderEnabled()) { reqParams.setExpectHeaderEnabled(false); this.append(path, buffer, reqParams, cachedSasToken, tracingContext); } + // If we have no HTTP response, throw the original exception. if (!op.hasResult()) { throw e; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index 35426980ccdcd..cf47f5e8d9293 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -221,16 +221,10 @@ public void testDeleteIdempotency() throws Exception { @Test public void testDeleteIdempotencyTriggerHttp404() throws Exception { - AbfsConfiguration abfsConfig - = TestAbfsConfigurationFieldsValidation.updateRetryConfigs( - getConfiguration(), - REDUCED_RETRY_COUNT, REDUCED_MAX_BACKOFF_INTERVALS_MS); - final AzureBlobFileSystem fs = getFileSystem(); - AbfsClient abfsClient = fs.getAbfsStore().getClient(); AbfsClient client = TestAbfsClient.createTestClientFromCurrentContext( - abfsClient, - abfsConfig); + fs.getAbfsStore().getClient(), + this.getConfiguration()); // Case 1: Not a retried case should throw error back // Add asserts at AzureBlobFileSystemStore and AbfsClient levels @@ -249,7 +243,7 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { // mock idempotency check to mimic retried case AbfsClient mockClient = TestAbfsClient.getMockAbfsClient( fs.getAbfsStore().getClient(), - abfsConfig); + this.getConfiguration()); AzureBlobFileSystemStore mockStore = mock(AzureBlobFileSystemStore.class); mockStore = TestMockHelpers.setClassField(AzureBlobFileSystemStore.class, mockStore, "client", mockClient); From 93a77a73351ba4e563931c6b98dd5a60a8f0376d Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Wed, 30 Mar 2022 14:41:46 +0530 Subject: [PATCH 07/53] Added retry mechanism for certain HTTP errors --- .../java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 9bb8b32b2d492..a611566204964 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 @@ -564,7 +564,7 @@ public AbfsRestOperation append(final String path, final byte[] buffer, ((AbfsRestOperationException) e).getStatusCode() < HttpURLConnection.HTTP_INTERNAL_ERROR) && reqParams.getIsExpectHeaderEnabled()) { reqParams.setExpectHeaderEnabled(false); - this.append(path, buffer, reqParams, cachedSasToken, tracingContext); + return this.append(path, buffer, reqParams, cachedSasToken, tracingContext); } // If we have no HTTP response, throw the original exception. if (!op.hasResult()) { From 9b43316e04c05f0dcd72a878371339bab59744a3 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Mon, 28 Feb 2022 14:24:37 +0530 Subject: [PATCH 08/53] ABFS: Added changes for expect hundred continue header with append requests --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 13 ++++ .../fs/azurebfs/AzureBlobFileSystem.java | 3 + .../azurebfs/constants/AbfsHttpConstants.java | 1 + .../azurebfs/constants/ConfigurationKeys.java | 2 + .../constants/FileSystemConfigurations.java | 3 +- .../constants/HttpHeaderConfigurations.java | 1 + .../InvalidAbfsRestOperationException.java | 30 ++++++--- .../fs/azurebfs/services/AbfsClient.java | 13 ++++ .../AbfsClientThrottlingAnalyzer.java | 2 +- .../AbfsClientThrottlingIntercept.java | 27 +++++--- .../AbfsClientThrottlingInterceptFactory.java | 33 ++++++++++ .../azurebfs/services/AbfsHttpOperation.java | 7 +- .../azurebfs/services/AbfsRestOperation.java | 14 ++-- .../ITestAzureBlobFileSystemCreate.java | 65 +++++++++++++++++++ .../constants/TestConfigurationKeys.java | 1 + 15 files changed, 190 insertions(+), 25 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index fafc30372b4a5..6ef7776c5a07c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -117,6 +117,15 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_OPTIMIZE_FOOTER_READ) private boolean optimizeFooterRead; + @BooleanConfigurationValidatorAnnotation( + ConfigurationKey = FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED, + DefaultValue = DEFAULT_FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED) + private boolean isExpectHeaderEnabled; + + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ACCOUNT_IS_SINGLETON_ENABLED, + DefaultValue = DEFAULT_FS_AZURE_ACCOUNT_IS_SINGLETON_ENABLED) + private boolean isSingletonEnabled; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_READ_BUFFER_SIZE, MinValue = MIN_BUFFER_SIZE, MaxValue = MAX_BUFFER_SIZE, @@ -689,6 +698,10 @@ public String getAppendBlobDirs() { return this.azureAppendBlobDirs; } + public boolean isExpectHeaderEnabled() { return this.isExpectHeaderEnabled; } + + public boolean isSingletonEnabled() {return this.isSingletonEnabled; } + public String getAzureInfiniteLeaseDirs() { return this.azureInfiniteLeaseDirs; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 46141e7c4a838..ba0b9c79d82e0 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -225,8 +225,11 @@ public void initialize(URI uri, Configuration configuration) } } +<<<<<<< HEAD AbfsClientThrottlingIntercept.initializeSingleton(abfsConfiguration.isAutoThrottlingEnabled()); rateLimiting = RateLimitingFactory.create(abfsConfiguration.getRateLimit()); +======= +>>>>>>> ABFS: Added changes for expect hundred continue header with append requests LOG.debug("Initializing AzureBlobFileSystem for {} complete", uri); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java index 5cf7ec565b59e..e82f38bd5f068 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java @@ -103,6 +103,7 @@ public final class AbfsHttpConstants { public static final String DEFAULT_SCOPE = "default:"; public static final String PERMISSION_FORMAT = "%04d"; public static final String SUPER_USER = "$superuser"; + public static final String HUNDRED_CONTINUE = "100-continue"; public static final char CHAR_FORWARD_SLASH = '/'; public static final char CHAR_EXCLAMATION_POINT = '!'; 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 9d3b2d5e82c6e..aff5b66332582 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,6 +35,8 @@ public final class ConfigurationKeys { * path to determine HNS status. */ public static final String FS_AZURE_ACCOUNT_IS_HNS_ENABLED = "fs.azure.account.hns.enabled"; + public static final String FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED = "fs.azure.account.expect.header.enabled"; + public static final String FS_AZURE_ACCOUNT_IS_SINGLETON_ENABLED = "fs.azure.account.singleton.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"; 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 63d62a33b1819..ff915eaef2e10 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,8 @@ 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 boolean DEFAULT_FS_AZURE_ACCOUNT_IS_SINGLETON_ENABLED = false; public static final String USER_HOME_DIRECTORY_PREFIX = "/user"; private static final int SIXTY_SECONDS = 60 * 1000; 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 d4065ac2836d0..ec68f1f923680 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 @@ -60,6 +60,7 @@ public final class HttpHeaderConfigurations { public static final String X_MS_UMASK = "x-ms-umask"; public static final String X_MS_NAMESPACE_ENABLED = "x-ms-namespace-enabled"; public static final String X_MS_ABFS_CLIENT_LATENCY = "x-ms-abfs-client-latency"; + public static final String EXPECT = "Expect"; public static final String X_MS_ENCRYPTION_KEY = "x-ms-encryption-key"; public static final String X_MS_ENCRYPTION_KEY_SHA256 = "x-ms-encryption-key-sha256"; public static final String X_MS_ENCRYPTION_ALGORITHM = "x-ms-encryption-algorithm"; 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 196202121343a..cce2ad1d2ca90 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 @@ -30,14 +30,24 @@ @InterfaceAudience.Public @InterfaceStability.Evolving public class InvalidAbfsRestOperationException extends AbfsRestOperationException { - public InvalidAbfsRestOperationException( - final Exception innerException) { - super( - AzureServiceErrorCode.UNKNOWN.getStatusCode(), - AzureServiceErrorCode.UNKNOWN.getErrorCode(), - innerException != null - ? innerException.toString() - : "InvalidAbfsRestOperationException", - innerException); - } + public InvalidAbfsRestOperationException( + final Exception innerException) { + super( + AzureServiceErrorCode.UNKNOWN.getStatusCode(), + AzureServiceErrorCode.UNKNOWN.getErrorCode(), + innerException != null + ? innerException.toString() + : "InvalidAbfsRestOperationException", + innerException); + } + + public InvalidAbfsRestOperationException(final Exception innerException, int retryCount) { + super( + AzureServiceErrorCode.UNKNOWN.getStatusCode(), + AzureServiceErrorCode.UNKNOWN.getErrorCode(), + innerException != null + ? innerException.toString() + : "InvalidAbfsRestOperationException" + "RetryCount: " + String.valueOf(retryCount), + innerException); + } } 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 b701037d0fc41..e84008f866139 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 @@ -99,6 +99,7 @@ public class AbfsClient implements Closeable { private AccessTokenProvider tokenProvider; private SASTokenProvider sasTokenProvider; private final AbfsCounters abfsCounters; + private boolean isExpectHeaderEnabled; private final ListeningScheduledExecutorService executorService; @@ -114,6 +115,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.isExpectHeaderEnabled = abfsConfiguration.isExpectHeaderEnabled(); String encryptionKey = this.abfsConfiguration .getClientProvidedEncryptionKey(); @@ -606,6 +608,9 @@ public AbfsRestOperation append(final String path, final byte[] buffer, addCustomerProvidedKeyHeaders(requestHeaders); // 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. + if (isExpectHeaderEnabled == true) { + requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); + } requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); if (reqParams.getLeaseId() != null) { @@ -1227,6 +1232,14 @@ protected AbfsCounters getAbfsCounters() { return abfsCounters; } + /** + * Getter for abfsConfiguration from AbfsClient. + * @return AbfsConfiguration instance + */ + protected AbfsConfiguration getAbfsConfiguration() { + return abfsConfiguration; + } + public int getNumLeaseThreads() { return abfsConfiguration.getNumLeaseThreads(); } 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 6dfd352954d34..e44ae052158d7 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 @@ -256,7 +256,7 @@ public void run() { /** * Stores Abfs operation metrics during each analysis period. */ - static class AbfsOperationMetrics { + class AbfsOperationMetrics { private AtomicLong bytesFailed; private AtomicLong bytesSuccessful; private AtomicLong operationsFailed; 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 7303e833418db..bf83bdc6df32e 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 @@ -46,6 +46,16 @@ public final class AbfsClientThrottlingIntercept { private AbfsClientThrottlingAnalyzer readThrottler = null; private AbfsClientThrottlingAnalyzer writeThrottler = null; private static boolean isAutoThrottlingEnabled = false; + private String accountName = ""; + + // Hide default constructor + public AbfsClientThrottlingIntercept(String accountName) { + isAutoThrottlingEnabled = true; + LOG.debug("Client-side throttling is enabled for the ABFS file system."); + this.readThrottler = new AbfsClientThrottlingAnalyzer("read"); + this.writeThrottler = new AbfsClientThrottlingAnalyzer("write"); + this.accountName = accountName; + } // Hide default constructor private AbfsClientThrottlingIntercept() { @@ -53,18 +63,19 @@ private AbfsClientThrottlingIntercept() { writeThrottler = new AbfsClientThrottlingAnalyzer("write"); } - public static synchronized void initializeSingleton(boolean enableAutoThrottling) { + public static synchronized AbfsClientThrottlingIntercept initializeSingleton(boolean enableAutoThrottling) { if (!enableAutoThrottling) { - return; + return null; } if (singleton == null) { singleton = new AbfsClientThrottlingIntercept(); isAutoThrottlingEnabled = true; LOG.debug("Client-side throttling is enabled for the ABFS file system."); } + return singleton; } - static void updateMetrics(AbfsRestOperationType operationType, + void updateMetrics(AbfsRestOperationType operationType, AbfsHttpOperation abfsHttpOperation) { if (!isAutoThrottlingEnabled || abfsHttpOperation == null) { return; @@ -82,7 +93,7 @@ static void updateMetrics(AbfsRestOperationType operationType, case Append: contentLength = abfsHttpOperation.getBytesSent(); if (contentLength > 0) { - singleton.writeThrottler.addBytesTransferred(contentLength, + this.writeThrottler.addBytesTransferred(contentLength, isFailedOperation); } break; @@ -90,7 +101,7 @@ static void updateMetrics(AbfsRestOperationType operationType, String range = abfsHttpOperation.getConnection().getRequestProperty(HttpHeaderConfigurations.RANGE); contentLength = getContentLengthIfKnown(range); if (contentLength > 0) { - singleton.readThrottler.addBytesTransferred(contentLength, + this.readThrottler.addBytesTransferred(contentLength, isFailedOperation); } break; @@ -104,7 +115,7 @@ static void updateMetrics(AbfsRestOperationType operationType, * uses this to suspend the request, if necessary, to minimize errors and * maximize throughput. */ - static void sendingRequest(AbfsRestOperationType operationType, + void sendingRequest(AbfsRestOperationType operationType, AbfsCounters abfsCounters) { if (!isAutoThrottlingEnabled) { return; @@ -112,13 +123,13 @@ static void sendingRequest(AbfsRestOperationType operationType, switch (operationType) { case ReadFile: - if (singleton.readThrottler.suspendIfNecessary() + if (this.readThrottler.suspendIfNecessary() && abfsCounters != null) { abfsCounters.incrementCounter(AbfsStatistic.READ_THROTTLES, 1); } break; case Append: - if (singleton.writeThrottler.suspendIfNecessary() + if (this.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/AbfsClientThrottlingInterceptFactory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java new file mode 100644 index 0000000000000..31edada643a04 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java @@ -0,0 +1,33 @@ +package org.apache.hadoop.fs.azurebfs.services; + +import org.apache.hadoop.fs.azurebfs.Abfs; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class AbfsClientThrottlingInterceptFactory { + private static final Logger LOG = LoggerFactory.getLogger(AbfsClientThrottlingInterceptFactory.class); + private static Map instanceMapping = new ConcurrentHashMap<>(); + + public static synchronized AbfsClientThrottlingIntercept getInstance(String accountName, boolean isAutoThrottlingEnabled, + boolean isSingletonEnabled) { + AbfsClientThrottlingIntercept instance; + if (isSingletonEnabled) { + instance = AbfsClientThrottlingIntercept.initializeSingleton(isAutoThrottlingEnabled); + } else { + if (!isAutoThrottlingEnabled) { + return null; + } + if (instanceMapping.get(accountName) == null) { + LOG.debug("The accountNameis: {} ", accountName); + instance = new AbfsClientThrottlingIntercept(accountName); + instanceMapping.put(accountName, instance); + } else { + instance = instanceMapping.get(accountName); + } + } + return instance; + } +} 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 413bf3686898b..d5da47847c2cd 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 @@ -320,7 +320,12 @@ public void sendRequest(byte[] buffer, int offset, int length) throws IOExceptio // accompanying statusCode this.bytesSent = length; outputStream.write(buffer, offset, length); - } finally { + } catch (Exception e) { + this.statusCode = this.connection.getResponseCode(); + this.bytesSent = length; + throw new IOException(e); + } + finally { if (this.isTraceEnabled) { this.sendRequestTimeMs = elapsedTimeMs(startTime); } 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 74b267d563eb2..67fa7653bd213 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 @@ -242,6 +242,10 @@ private void completeExecute(TracingContext tracingContext) private boolean executeHttpOperation(final int retryCount, TracingContext tracingContext) throws AzureBlobFileSystemException { AbfsHttpOperation httpOperation = null; + String accountName = this.client.getAbfsConfiguration().getAccountName(); + boolean isAutoThrottlingEnabled = this.client.getAbfsConfiguration().isAutoThrottlingEnabled(); + boolean isSingletonEnabled = this.client.getAbfsConfiguration().isSingletonEnabled(); + AbfsClientThrottlingIntercept intercept; try { // initialize the HTTP request and open the connection httpOperation = new AbfsHttpOperation(url, method, requestHeaders); @@ -278,7 +282,8 @@ private boolean executeHttpOperation(final int retryCount, // dump the headers AbfsIoUtils.dumpHeadersToDebugLog("Request Headers", httpOperation.getConnection().getRequestProperties()); - AbfsClientThrottlingIntercept.sendingRequest(operationType, abfsCounters); + intercept = AbfsClientThrottlingInterceptFactory.getInstance(accountName, isAutoThrottlingEnabled, isSingletonEnabled); + intercept.sendingRequest(operationType, abfsCounters); if (hasRequestBody) { // HttpUrlConnection requires @@ -303,7 +308,7 @@ private boolean executeHttpOperation(final int retryCount, 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) { @@ -312,12 +317,13 @@ private boolean executeHttpOperation(final int retryCount, } if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) { - throw new InvalidAbfsRestOperationException(ex); + throw new InvalidAbfsRestOperationException(ex,retryCount); } return false; } finally { - AbfsClientThrottlingIntercept.updateMetrics(operationType, httpOperation); + AbfsClientThrottlingIntercept instance = AbfsClientThrottlingInterceptFactory.getInstance(accountName, isAutoThrottlingEnabled, isSingletonEnabled); + instance.updateMetrics(operationType, httpOperation); } LOG.debug("HttpRequest: {}: {}", operationType, httpOperation); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 2f23ac5c5c708..df7ccab0bd4d8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -22,6 +22,7 @@ import java.io.FilterOutputStream; import java.io.IOException; import java.lang.reflect.Field; +import java.net.URI; import java.util.EnumSet; import java.util.UUID; @@ -36,6 +37,8 @@ import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.test.GenericTestUtils; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*; +import static org.junit.Assume.assumeTrue; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; @@ -43,6 +46,8 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientThrottlingIntercept; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientThrottlingInterceptFactory; import org.apache.hadoop.fs.azurebfs.services.TestAbfsClient; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; @@ -475,6 +480,66 @@ private AzureBlobFileSystemStore setAzureBlobSystemStoreField( return abfsStore; } + @Test + public void testCreatMultipleAccountThrottling() + throws Throwable { + final AzureBlobFileSystem currentFs = getFileSystem(); + Configuration config = new Configuration(this.getRawConfiguration()); + String fileSystemName = TEST_CONTAINER_PREFIX + UUID.randomUUID().toString(); + String accountName = config.get(FS_AZURE_ACCOUNT_NAME); + if (accountName == null) { + // check if accountName is set using different config key + accountName = config.get(FS_AZURE_ABFS_ACCOUNT1_NAME); + } + assumeTrue("Not set: " + FS_AZURE_ABFS_ACCOUNT1_NAME, + accountName != null && !accountName.isEmpty()); + + final String abfsUrl = fileSystemName + "@" + accountName; + URI defaultUri = null; + String abfsScheme = "abfss"; + try { + defaultUri = new URI(abfsScheme, abfsUrl, null, null, null); + } catch (Exception ex) { + throw new AssertionError(ex); + } + + Configuration rawConfig1 = new Configuration(); + rawConfig1.addResource(TEST_CONFIGURATION_FILE_NAME); + + AbfsConfiguration abfsConfig = new AbfsConfiguration(rawConfig1, accountName); + + final AzureBlobFileSystem fs = + (AzureBlobFileSystem) FileSystem.newInstance(defaultUri, + rawConfig1); + System.out.println(fs.toString()); + + AbfsClient mockClient = TestAbfsClient.getMockAbfsClient(fs.getAbfsStore().getClient(), fs.getAbfsStore().getAbfsConfiguration()); + + AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); + abfsStore = setAzureBlobSystemStoreField(abfsStore, "client", mockClient); + + AbfsRestOperation successOp = mock(AbfsRestOperation.class); + AbfsHttpOperation http500Op = mock(AbfsHttpOperation.class); + when(http500Op.getStatusCode()).thenReturn(HTTP_INTERNAL_ERROR); + when(successOp.getResult()).thenReturn(http500Op); + + AbfsClientThrottlingIntercept instance1 = AbfsClientThrottlingInterceptFactory.getInstance(accountName, true, true); + String accountName1 = config.get(FS_AZURE_ABFS_ACCOUNT1_NAME); + AbfsClientThrottlingIntercept instance2 = AbfsClientThrottlingInterceptFactory.getInstance(accountName1, true, true); + //if singleton is enabled, for different accounts both the instances should return same value + assertEquals(instance1, instance2); + + AbfsClientThrottlingIntercept instance3 = AbfsClientThrottlingInterceptFactory.getInstance(accountName, true, false); + AbfsClientThrottlingIntercept instance4 = AbfsClientThrottlingInterceptFactory.getInstance(accountName1, true, false); + AbfsClientThrottlingIntercept instance5 = AbfsClientThrottlingInterceptFactory.getInstance(accountName, true, false); + + //if singleton is not enabled, for different accounts instances should return different value + assertNotEquals(instance3, instance4); + + //if singleton is not enabled, for same accounts instances should return same value + assertEquals(instance3, instance5); + } + private void validateCreateFileException(final Class exceptionClass, final AzureBlobFileSystemStore abfsStore) throws Exception { FsPermission permission = new FsPermission(FsAction.ALL, FsAction.ALL, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java index 565eb38c4f70a..2c659e1399a4f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java @@ -24,6 +24,7 @@ public final class TestConfigurationKeys { public static final String FS_AZURE_ACCOUNT_NAME = "fs.azure.account.name"; public static final String FS_AZURE_ABFS_ACCOUNT_NAME = "fs.azure.abfs.account.name"; + public static final String FS_AZURE_ABFS_ACCOUNT1_NAME = "fs.azure.abfs.account1.name"; public static final String FS_AZURE_ACCOUNT_KEY = "fs.azure.account.key"; public static final String FS_AZURE_CONTRACT_TEST_URI = "fs.contract.test.fs.abfs"; public static final String FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT = "fs.azure.test.namespace.enabled"; From 899b40be7000bd10c7682b0f561335fe24d86834 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Wed, 2 Mar 2022 11:17:15 +0530 Subject: [PATCH 09/53] ABFS: Added changes for expect hundred continue --- .../AbfsClientThrottlingInterceptFactory.java | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java index 31edada643a04..2a5895d09127a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.fs.azurebfs.services; import org.apache.hadoop.fs.azurebfs.Abfs; @@ -21,7 +39,7 @@ public static synchronized AbfsClientThrottlingIntercept getInstance(String acco return null; } if (instanceMapping.get(accountName) == null) { - LOG.debug("The accountNameis: {} ", accountName); + LOG.debug("The accountName is: {} ", accountName); instance = new AbfsClientThrottlingIntercept(accountName); instanceMapping.put(accountName, instance); } else { From 2af317dc8434fb71305ee04c48c129eb1ee8a467 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Wed, 2 Mar 2022 11:40:02 +0530 Subject: [PATCH 10/53] ABFS: Added changes for expect hundred continue --- .../org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java | 8 ++++++-- .../apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java | 1 - .../apache/hadoop/fs/azurebfs/services/AbfsClient.java | 2 +- .../services/AbfsClientThrottlingInterceptFactory.java | 1 - .../hadoop/fs/azurebfs/services/AbfsRestOperation.java | 2 +- .../fs/azurebfs/ITestAzureBlobFileSystemCreate.java | 6 +++++- 6 files changed, 13 insertions(+), 7 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 6ef7776c5a07c..c7b293d5ee256 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -698,9 +698,13 @@ public String getAppendBlobDirs() { return this.azureAppendBlobDirs; } - public boolean isExpectHeaderEnabled() { return this.isExpectHeaderEnabled; } + public boolean isExpectHeaderEnabled() { + return this.isExpectHeaderEnabled; + } - public boolean isSingletonEnabled() {return this.isSingletonEnabled; } + public boolean isSingletonEnabled() { + return this.isSingletonEnabled; + } public String getAzureInfiniteLeaseDirs() { return this.azureInfiniteLeaseDirs; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index ba0b9c79d82e0..6380790d64a10 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -55,7 +55,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.azurebfs.commit.ResilientCommitByRename; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; -import org.apache.hadoop.fs.azurebfs.services.AbfsClientThrottlingIntercept; import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.classification.InterfaceStability; 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 e84008f866139..7094e374d07cb 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 @@ -608,7 +608,7 @@ public AbfsRestOperation append(final String path, final byte[] buffer, addCustomerProvidedKeyHeaders(requestHeaders); // 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. - if (isExpectHeaderEnabled == true) { + if (isExpectHeaderEnabled) { requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); } requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java index 2a5895d09127a..2931f211443e5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java @@ -18,7 +18,6 @@ package org.apache.hadoop.fs.azurebfs.services; -import org.apache.hadoop.fs.azurebfs.Abfs; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 67fa7653bd213..3a51d201d6901 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 @@ -317,7 +317,7 @@ private boolean executeHttpOperation(final int retryCount, } if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) { - throw new InvalidAbfsRestOperationException(ex,retryCount); + throw new InvalidAbfsRestOperationException(ex, retryCount); } return false; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index df7ccab0bd4d8..76e783ddf7325 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -37,7 +37,11 @@ import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.test.GenericTestUtils; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.*; + +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONTAINER_PREFIX; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT1_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_NAME; import static org.junit.Assume.assumeTrue; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; From cc9fcdb4c6a801d5907abe0f1114261941f53c77 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Wed, 2 Mar 2022 15:50:02 +0530 Subject: [PATCH 11/53] ABFS: Added changes for expect hundred continue --- .../fs/azurebfs/services/AbfsHttpOperation.java | 5 +++-- .../fs/azurebfs/ITestAzureBlobFileSystemDelete.java | 11 ++++++++--- .../hadoop/fs/azurebfs/services/TestAbfsClient.java | 2 +- 3 files changed, 12 insertions(+), 6 deletions(-) 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 d5da47847c2cd..70659f9101395 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 @@ -323,7 +323,9 @@ public void sendRequest(byte[] buffer, int offset, int length) throws IOExceptio } catch (Exception e) { this.statusCode = this.connection.getResponseCode(); this.bytesSent = length; - throw new IOException(e); + if (this.statusCode != -1 && this.statusCode >= HttpURLConnection.HTTP_INTERNAL_ERROR) { + throw new IOException(e); + } } finally { if (this.isTraceEnabled) { @@ -348,7 +350,6 @@ public void processResponse(final byte[] buffer, final int offset, final int len if (this.isTraceEnabled) { startTime = System.nanoTime(); } - this.statusCode = this.connection.getResponseCode(); if (this.isTraceEnabled) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index db181fb5dd660..35426980ccdcd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -221,11 +221,16 @@ public void testDeleteIdempotency() throws Exception { @Test public void testDeleteIdempotencyTriggerHttp404() throws Exception { + AbfsConfiguration abfsConfig + = TestAbfsConfigurationFieldsValidation.updateRetryConfigs( + getConfiguration(), + REDUCED_RETRY_COUNT, REDUCED_MAX_BACKOFF_INTERVALS_MS); final AzureBlobFileSystem fs = getFileSystem(); + AbfsClient abfsClient = fs.getAbfsStore().getClient(); AbfsClient client = TestAbfsClient.createTestClientFromCurrentContext( - fs.getAbfsStore().getClient(), - this.getConfiguration()); + abfsClient, + abfsConfig); // Case 1: Not a retried case should throw error back // Add asserts at AzureBlobFileSystemStore and AbfsClient levels @@ -244,7 +249,7 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { // mock idempotency check to mimic retried case AbfsClient mockClient = TestAbfsClient.getMockAbfsClient( fs.getAbfsStore().getClient(), - this.getConfiguration()); + abfsConfig); AzureBlobFileSystemStore mockStore = mock(AzureBlobFileSystemStore.class); mockStore = TestMockHelpers.setClassField(AzureBlobFileSystemStore.class, mockStore, "client", mockClient); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index a725bf3175a5c..68907d20b02fb 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -306,7 +306,7 @@ public static AbfsClient getMockAbfsClient(AbfsClient baseAbfsClientInstance, when(client.getAccessToken()).thenCallRealMethod(); when(client.getSharedKeyCredentials()).thenCallRealMethod(); when(client.createDefaultHeaders()).thenCallRealMethod(); - + when(client.getAbfsConfiguration()).thenReturn(abfsConfig); // override baseurl client = TestAbfsClient.setAbfsClientField(client, "abfsConfiguration", abfsConfig); From 9fc9c99446517666fdeb673a40913b1630974d83 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Fri, 4 Mar 2022 10:45:29 +0530 Subject: [PATCH 12/53] Added retry mechanism for certain HTTP errors --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 1 + .../services/AppendRequestParameters.java | 12 +++++- .../fs/azurebfs/services/AbfsClient.java | 11 ++++-- .../azurebfs/services/AbfsHttpOperation.java | 8 +--- .../azurebfs/services/AbfsOutputStream.java | 6 ++- .../services/AbfsOutputStreamContext.java | 11 ++++++ .../azurebfs/services/AbfsRestOperation.java | 1 - .../fs/azurebfs/ITestCustomerProvidedKey.java | 4 +- .../services/TestAbfsOutputStream.java | 37 ++++++++++++------- 9 files changed, 61 insertions(+), 30 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 09b48a855f00b..6f79b27e5f715 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -692,6 +692,7 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext( return new AbfsOutputStreamContext(abfsConfiguration.getSasTokenRenewPeriodForStreamsInSeconds()) .withWriteBufferSize(bufferSize) .enableFlush(abfsConfiguration.isFlushEnabled()) + .enableExpectHeader(abfsConfiguration.isExpectHeaderEnabled()) .enableSmallWriteOptimization(abfsConfiguration.isSmallWriteOptimizationEnabled()) .disableOutputStreamFlush(abfsConfiguration.isOutputStreamFlushDisabled()) .withStreamStatistics(new AbfsOutputStreamStatisticsImpl()) 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 7369bfaf56422..579535a16f926 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,21 @@ public enum Mode { private final Mode mode; private final boolean isAppendBlob; private final String leaseId; + private boolean isExpectHeaderEnabled; public AppendRequestParameters(final long position, final int offset, final int length, final Mode mode, final boolean isAppendBlob, - final String leaseId) { + final String leaseId, boolean isExpectHeaderEnabled) { this.position = position; this.offset = offset; this.length = length; this.mode = mode; this.isAppendBlob = isAppendBlob; this.leaseId = leaseId; + this.isExpectHeaderEnabled = isExpectHeaderEnabled; } public long getPosition() { @@ -72,4 +74,12 @@ public boolean isAppendBlob() { public String getLeaseId() { return this.leaseId; } + + public boolean getIsExpectHeaderEnabled() { + return isExpectHeaderEnabled; + } + + public void setExpectHeaderEnabled(boolean expectHeaderEnabled) { + isExpectHeaderEnabled = expectHeaderEnabled; + } } 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 7094e374d07cb..ab1909f594f4c 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 @@ -38,6 +38,7 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback; @@ -99,7 +100,6 @@ public class AbfsClient implements Closeable { private AccessTokenProvider tokenProvider; private SASTokenProvider sasTokenProvider; private final AbfsCounters abfsCounters; - private boolean isExpectHeaderEnabled; private final ListeningScheduledExecutorService executorService; @@ -115,7 +115,6 @@ 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.isExpectHeaderEnabled = abfsConfiguration.isExpectHeaderEnabled(); String encryptionKey = this.abfsConfiguration .getClientProvidedEncryptionKey(); @@ -608,7 +607,7 @@ public AbfsRestOperation append(final String path, final byte[] buffer, addCustomerProvidedKeyHeaders(requestHeaders); // 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. - if (isExpectHeaderEnabled) { + if (reqParams.getIsExpectHeaderEnabled()) { requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); } requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, @@ -648,6 +647,12 @@ public AbfsRestOperation append(final String path, final byte[] buffer, op.execute(tracingContext); } catch (AzureBlobFileSystemException e) { // If we have no HTTP response, throw the original exception. + if ((((AbfsRestOperationException) e).getStatusCode() >= HttpURLConnection.HTTP_BAD_REQUEST && + ((AbfsRestOperationException) e).getStatusCode() < HttpURLConnection.HTTP_INTERNAL_ERROR) && + reqParams.getIsExpectHeaderEnabled()) { + reqParams.setExpectHeaderEnabled(false); + this.append(path, buffer, reqParams, cachedSasToken, tracingContext); + } if (!op.hasResult()) { throw e; } 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 70659f9101395..b3b0a9fed1298 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 @@ -266,7 +266,6 @@ public AbfsHttpOperation(final URL url, final String method, final List= HttpURLConnection.HTTP_INTERNAL_ERROR) { - throw new IOException(e); - } - } - finally { + } finally { if (this.isTraceEnabled) { this.sendRequestTimeMs = elapsedTimeMs(startTime); } 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 620616b993fe2..82e20ce5b7684 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 @@ -80,6 +80,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; @@ -133,6 +134,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 @@ -327,7 +329,7 @@ private void uploadBlockAsync(DataBlocks.DataBlock blockToUpload, * leaseId - The AbfsLeaseId for this request. */ AppendRequestParameters reqParams = new AppendRequestParameters( - offset, 0, bytesLength, mode, false, leaseId); + offset, 0, bytesLength, mode, false, leaseId, isExpectHeaderEnabled); AbfsRestOperation op = client.append(path, blockUploadData.toByteArray(), reqParams, cachedSasToken.get(), new TracingContext(tracingContext)); @@ -573,7 +575,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()); 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 ad303823e0c68..ed89733036741 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; @@ -78,6 +80,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; @@ -184,6 +191,10 @@ public boolean isEnableFlush() { return enableFlush; } + public boolean isExpectHeaderEnabled() { + return enableExpectHeader; + } + public boolean isDisableOutputStreamFlush() { return disableOutputStreamFlush; } 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 3a51d201d6901..5e69dd3a553b6 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 @@ -333,7 +333,6 @@ private boolean executeHttpOperation(final int retryCount, } result = httpOperation; - return true; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java index 02260310bb813..f70e0109df6cc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java @@ -203,7 +203,7 @@ public void testAppendWithCPK() throws Exception { // Trying to append with correct CPK headers AppendRequestParameters appendRequestParameters = new AppendRequestParameters( - 0, 0, 5, Mode.APPEND_MODE, false, null); + 0, 0, 5, Mode.APPEND_MODE, false, null, true); byte[] buffer = getRandomBytesArray(5); AbfsClient abfsClient = fs.getAbfsClient(); AbfsRestOperation abfsRestOperation = abfsClient @@ -248,7 +248,7 @@ public void testAppendWithoutCPK() throws Exception { // Trying to append without CPK headers AppendRequestParameters appendRequestParameters = new AppendRequestParameters( - 0, 0, 5, Mode.APPEND_MODE, false, null); + 0, 0, 5, Mode.APPEND_MODE, false, null, true); byte[] buffer = getRandomBytesArray(5); AbfsClient abfsClient = fs.getAbfsClient(); AbfsRestOperation abfsRestOperation = abfsClient diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index 0673e387bfbf0..e26ba938cf5db 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -72,6 +72,7 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext( boolean isFlushEnabled, boolean disableOutputStreamFlush, boolean isAppendBlob, + boolean isExpectHeaderEnabled, AbfsClient client, String path, TracingContext tracingContext, @@ -89,6 +90,7 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext( return new AbfsOutputStreamContext(2) .withWriteBufferSize(writeBufferSize) + .enableExpectHeader(isExpectHeaderEnabled) .enableFlush(isFlushEnabled) .disableOutputStreamFlush(disableOutputStreamFlush) .withStreamStatistics(new AbfsOutputStreamStatisticsImpl()) @@ -129,6 +131,7 @@ public void verifyShortWriteRequest() throws Exception { true, false, false, + true, client, PATH, new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", @@ -149,9 +152,9 @@ public void verifyShortWriteRequest() throws Exception { out.hsync(); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, WRITE_SIZE, APPEND_MODE, false, null); + 0, 0, WRITE_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - WRITE_SIZE, 0, 2 * WRITE_SIZE, APPEND_MODE, false, null); + WRITE_SIZE, 0, 2 * WRITE_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), @@ -190,6 +193,7 @@ public void verifyWriteRequest() throws Exception { true, false, false, + true, client, PATH, tracingContext, @@ -203,9 +207,9 @@ public void verifyWriteRequest() throws Exception { out.close(); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, 5*WRITE_SIZE-BUFFER_SIZE, APPEND_MODE, false, null); + BUFFER_SIZE, 0, 5*WRITE_SIZE-BUFFER_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), @@ -264,6 +268,7 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { true, false, false, + true, client, PATH, tracingContext, @@ -277,9 +282,9 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { out.close(); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); @@ -335,6 +340,7 @@ public void verifyWriteRequestOfBufferSize() throws Exception { true, false, false, + true, client, PATH, new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", @@ -350,9 +356,9 @@ public void verifyWriteRequestOfBufferSize() throws Exception { Thread.sleep(1000); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); @@ -390,6 +396,7 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { true, false, true, + true, client, PATH, new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", @@ -405,9 +412,9 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { Thread.sleep(1000); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, true, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, true, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, true, null); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, true, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); @@ -449,6 +456,7 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { true, false, false, + true, client, PATH, new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", @@ -464,9 +472,9 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { out.hflush(); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); @@ -518,6 +526,7 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { true, false, false, + true, client, PATH, new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", @@ -535,9 +544,9 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { Thread.sleep(1000); AppendRequestParameters firstReqParameters = new AppendRequestParameters( - 0, 0, BUFFER_SIZE, APPEND_MODE, false, null); + 0, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); AppendRequestParameters secondReqParameters = new AppendRequestParameters( - BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); + BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null, true); verify(client, times(1)).append( eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); From 56eda26aacf7d9ee5964978944925c4c2a590bcd Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Fri, 4 Mar 2022 11:08:15 +0530 Subject: [PATCH 13/53] Added retry mechanism for certain HTTP errors --- .../hadoop/fs/azurebfs/services/AbfsClient.java | 5 ++++- .../fs/azurebfs/ITestAzureBlobFileSystemDelete.java | 12 +++--------- 2 files changed, 7 insertions(+), 10 deletions(-) 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 ab1909f594f4c..b9393dce3a42a 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 @@ -646,13 +646,16 @@ public AbfsRestOperation append(final String path, final byte[] buffer, try { op.execute(tracingContext); } catch (AzureBlobFileSystemException e) { - // If we have no HTTP response, throw the original exception. + /* + If the http response code indicates a user error we retry the same append request with expect header disabled + */ if ((((AbfsRestOperationException) e).getStatusCode() >= HttpURLConnection.HTTP_BAD_REQUEST && ((AbfsRestOperationException) e).getStatusCode() < HttpURLConnection.HTTP_INTERNAL_ERROR) && reqParams.getIsExpectHeaderEnabled()) { reqParams.setExpectHeaderEnabled(false); this.append(path, buffer, reqParams, cachedSasToken, tracingContext); } + // If we have no HTTP response, throw the original exception. if (!op.hasResult()) { throw e; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index 35426980ccdcd..cf47f5e8d9293 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -221,16 +221,10 @@ public void testDeleteIdempotency() throws Exception { @Test public void testDeleteIdempotencyTriggerHttp404() throws Exception { - AbfsConfiguration abfsConfig - = TestAbfsConfigurationFieldsValidation.updateRetryConfigs( - getConfiguration(), - REDUCED_RETRY_COUNT, REDUCED_MAX_BACKOFF_INTERVALS_MS); - final AzureBlobFileSystem fs = getFileSystem(); - AbfsClient abfsClient = fs.getAbfsStore().getClient(); AbfsClient client = TestAbfsClient.createTestClientFromCurrentContext( - abfsClient, - abfsConfig); + fs.getAbfsStore().getClient(), + this.getConfiguration()); // Case 1: Not a retried case should throw error back // Add asserts at AzureBlobFileSystemStore and AbfsClient levels @@ -249,7 +243,7 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { // mock idempotency check to mimic retried case AbfsClient mockClient = TestAbfsClient.getMockAbfsClient( fs.getAbfsStore().getClient(), - abfsConfig); + this.getConfiguration()); AzureBlobFileSystemStore mockStore = mock(AzureBlobFileSystemStore.class); mockStore = TestMockHelpers.setClassField(AzureBlobFileSystemStore.class, mockStore, "client", mockClient); From 091f1d45719e85091995620d4f6380235701dcc7 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Wed, 30 Mar 2022 14:41:46 +0530 Subject: [PATCH 14/53] Added retry mechanism for certain HTTP errors --- .../java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 b9393dce3a42a..79bcc9f8d612d 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 @@ -653,7 +653,7 @@ public AbfsRestOperation append(final String path, final byte[] buffer, ((AbfsRestOperationException) e).getStatusCode() < HttpURLConnection.HTTP_INTERNAL_ERROR) && reqParams.getIsExpectHeaderEnabled()) { reqParams.setExpectHeaderEnabled(false); - this.append(path, buffer, reqParams, cachedSasToken, tracingContext); + return this.append(path, buffer, reqParams, cachedSasToken, tracingContext); } // If we have no HTTP response, throw the original exception. if (!op.hasResult()) { From 3709619362195c6e35f5c45f95de892a19e84717 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Thu, 12 May 2022 15:44:51 +0530 Subject: [PATCH 15/53] Fix trunk conflict --- .../org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 6380790d64a10..295edb8ad580e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -224,11 +224,7 @@ public void initialize(URI uri, Configuration configuration) } } -<<<<<<< HEAD - AbfsClientThrottlingIntercept.initializeSingleton(abfsConfiguration.isAutoThrottlingEnabled()); rateLimiting = RateLimitingFactory.create(abfsConfiguration.getRateLimit()); -======= ->>>>>>> ABFS: Added changes for expect hundred continue header with append requests LOG.debug("Initializing AzureBlobFileSystem for {} complete", uri); } From 94397c74918f1e7aab18697ab6477b58f23b413e Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Wed, 20 Jul 2022 12:24:07 +0530 Subject: [PATCH 16/53] Added config details in md file --- .../hadoop/fs/azurebfs/services/AbfsClient.java | 11 +++++++---- .../services/AbfsClientThrottlingIntercept.java | 6 +++++- .../fs/azurebfs/services/AbfsHttpOperation.java | 2 +- hadoop-tools/hadoop-azure/src/site/markdown/abfs.md | 12 ++++++++++++ 4 files changed, 25 insertions(+), 6 deletions(-) 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 79bcc9f8d612d..a7cc5191970e9 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 @@ -649,11 +649,14 @@ public AbfsRestOperation append(final String path, final byte[] buffer, /* If the http response code indicates a user error we retry the same append request with expect header disabled */ - if ((((AbfsRestOperationException) e).getStatusCode() >= HttpURLConnection.HTTP_BAD_REQUEST && - ((AbfsRestOperationException) e).getStatusCode() < HttpURLConnection.HTTP_INTERNAL_ERROR) && - reqParams.getIsExpectHeaderEnabled()) { + if ((((AbfsRestOperationException) e).getStatusCode() + >= HttpURLConnection.HTTP_BAD_REQUEST && + ((AbfsRestOperationException) e).getStatusCode() + < HttpURLConnection.HTTP_INTERNAL_ERROR) && + reqParams.getIsExpectHeaderEnabled()) { reqParams.setExpectHeaderEnabled(false); - return this.append(path, buffer, reqParams, cachedSasToken, tracingContext); + return this.append(path, buffer, reqParams, cachedSasToken, + tracingContext); } // If we have no HTTP response, throw the original exception. if (!op.hasResult()) { 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 bf83bdc6df32e..1da2c7edd12c7 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 @@ -48,9 +48,13 @@ public final class AbfsClientThrottlingIntercept { private static boolean isAutoThrottlingEnabled = false; private String accountName = ""; + private synchronized static void setIsAutoThrottlingEnabled(boolean autoThrottlingEnabled) { + isAutoThrottlingEnabled = autoThrottlingEnabled; + } + // Hide default constructor public AbfsClientThrottlingIntercept(String accountName) { - isAutoThrottlingEnabled = true; + setIsAutoThrottlingEnabled(true); LOG.debug("Client-side throttling is enabled for the ABFS file system."); this.readThrottler = new AbfsClientThrottlingAnalyzer("read"); this.writeThrottler = new AbfsClientThrottlingAnalyzer("write"); 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 b3b0a9fed1298..f1fd769bc61f9 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 @@ -319,7 +319,7 @@ public void sendRequest(byte[] buffer, int offset, int length) throws IOExceptio // accompanying statusCode this.bytesSent = length; outputStream.write(buffer, offset, length); - } catch (Exception e) { + } catch (IOException e) { this.bytesSent = length; } finally { if (this.isTraceEnabled) { diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md index 35d360556047e..0c858441d0348 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md @@ -767,6 +767,18 @@ Hflush() being the only documented API that can provide persistent data transfer, Flush() also attempting to persist buffered data will lead to performance issues. +### 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. + +`fs.azure.account.singleton.enabled`: This config is used to specify whether you +want to enable throttling at account level or not. Otherwise a single throttling +class level instance is created. It is configured to true by default. We need to +configure it to false if we want account level throttling. + ### 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, From 5f2606199032b27dbe5331bd6cf23bdbb648688f Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Wed, 20 Jul 2022 14:24:55 +0530 Subject: [PATCH 17/53] Changing class modifier --- .../fs/azurebfs/services/AbfsClientThrottlingAnalyzer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 e44ae052158d7..3e6a34bc92f02 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 @@ -256,7 +256,7 @@ public void run() { /** * Stores Abfs operation metrics during each analysis period. */ - class AbfsOperationMetrics { + private class AbfsOperationMetrics { private AtomicLong bytesFailed; private AtomicLong bytesSuccessful; private AtomicLong operationsFailed; From 0f18d947eb63062dbde2499a5fab334e16d04d93 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Tue, 2 Aug 2022 09:40:49 +0530 Subject: [PATCH 18/53] Spot bugs and checkstyle fixes --- .../src/config/checkstyle-suppressions.xml | 2 + .../fs/azurebfs/services/AbfsClient.java | 8 +- .../AbfsClientThrottlingAnalyzer.java | 42 +++-------- .../AbfsClientThrottlingIntercept.java | 2 +- .../AbfsClientThrottlingInterceptFactory.java | 53 +++++++------ .../services/AbfsOperationMetrics.java | 74 +++++++++++++++++++ .../ITestAzureBlobFileSystemCreate.java | 62 ---------------- .../services/TestExponentialRetryPolicy.java | 66 ++++++++++++++++- 8 files changed, 188 insertions(+), 121 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOperationMetrics.java diff --git a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml index fd2a7c210e706..e9ca716400e4e 100644 --- a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml +++ b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml @@ -51,4 +51,6 @@ + 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 10aa4bd7efd90..33d0a2b3288e4 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 @@ -696,10 +696,10 @@ public AbfsRestOperation append(final String path, final byte[] buffer, If the http response code indicates a user error we retry the same append request with expect header disabled */ if ((((AbfsRestOperationException) e).getStatusCode() - >= HttpURLConnection.HTTP_BAD_REQUEST && - ((AbfsRestOperationException) e).getStatusCode() - < HttpURLConnection.HTTP_INTERNAL_ERROR) && - reqParams.getIsExpectHeaderEnabled()) { + >= HttpURLConnection.HTTP_BAD_REQUEST + && ((AbfsRestOperationException) e).getStatusCode() + < HttpURLConnection.HTTP_INTERNAL_ERROR) + && reqParams.getIsExpectHeaderEnabled()) { reqParams.setExpectHeaderEnabled(false); return this.append(path, buffer, reqParams, cachedSasToken, tracingContext); 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 3e6a34bc92f02..84abc5df37ec4 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 @@ -21,7 +21,6 @@ import java.util.Timer; import java.util.TimerTask; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.classification.VisibleForTesting; @@ -104,11 +103,11 @@ private AbfsClientThrottlingAnalyzer() { public void addBytesTransferred(long count, boolean isFailedOperation) { AbfsOperationMetrics metrics = blobMetrics.get(); if (isFailedOperation) { - metrics.bytesFailed.addAndGet(count); - metrics.operationsFailed.incrementAndGet(); + metrics.getBytesFailed().addAndGet(count); + metrics.getOperationsFailed().incrementAndGet(); } else { - metrics.bytesSuccessful.addAndGet(count); - metrics.operationsSuccessful.incrementAndGet(); + metrics.getBytesSuccessful().addAndGet(count); + metrics.getOperationsSuccessful().incrementAndGet(); } } @@ -137,16 +136,16 @@ int getSleepDuration() { 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; + long periodMs = metrics.getEndTime() - metrics.getStartTime(); double newSleepDuration; @@ -238,10 +237,10 @@ public void run() { } long now = System.currentTimeMillis(); - if (now - blobMetrics.get().startTime >= analysisPeriodMs) { + if (now - blobMetrics.get().getStartTime() >= analysisPeriodMs) { AbfsOperationMetrics oldMetrics = blobMetrics.getAndSet( new AbfsOperationMetrics(now)); - oldMetrics.endTime = now; + oldMetrics.setEndTime(now); sleepDuration = analyzeMetricsAndUpdateSleepDuration(oldMetrics, sleepDuration); } @@ -253,23 +252,4 @@ public void run() { } } - /** - * Stores Abfs operation metrics during each analysis period. - */ - private 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 1da2c7edd12c7..60570cf81fb00 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 @@ -48,7 +48,7 @@ public final class AbfsClientThrottlingIntercept { private static boolean isAutoThrottlingEnabled = false; private String accountName = ""; - private synchronized static void setIsAutoThrottlingEnabled(boolean autoThrottlingEnabled) { + private synchronized void setIsAutoThrottlingEnabled(boolean autoThrottlingEnabled) { isAutoThrottlingEnabled = autoThrottlingEnabled; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java index 2931f211443e5..2fc8a69e8c585 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java @@ -24,27 +24,36 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -public class AbfsClientThrottlingInterceptFactory { - private static final Logger LOG = LoggerFactory.getLogger(AbfsClientThrottlingInterceptFactory.class); - private static Map instanceMapping = new ConcurrentHashMap<>(); - - public static synchronized AbfsClientThrottlingIntercept getInstance(String accountName, boolean isAutoThrottlingEnabled, - boolean isSingletonEnabled) { - AbfsClientThrottlingIntercept instance; - if (isSingletonEnabled) { - instance = AbfsClientThrottlingIntercept.initializeSingleton(isAutoThrottlingEnabled); - } else { - if (!isAutoThrottlingEnabled) { - return null; - } - if (instanceMapping.get(accountName) == null) { - LOG.debug("The accountName is: {} ", accountName); - instance = new AbfsClientThrottlingIntercept(accountName); - instanceMapping.put(accountName, instance); - } else { - instance = instanceMapping.get(accountName); - } - } - return instance; +class AbfsClientThrottlingInterceptFactory { + + private AbfsClientThrottlingInterceptFactory() { + } + + private static final Logger LOG = LoggerFactory.getLogger( + AbfsClientThrottlingInterceptFactory.class); + + private static Map instanceMapping + = new ConcurrentHashMap<>(); + + static synchronized AbfsClientThrottlingIntercept getInstance(String accountName, + boolean isAutoThrottlingEnabled, + boolean isSingletonEnabled) { + AbfsClientThrottlingIntercept instance; + if (isSingletonEnabled) { + instance = AbfsClientThrottlingIntercept.initializeSingleton( + isAutoThrottlingEnabled); + } else { + if (!isAutoThrottlingEnabled) { + return null; + } + if (instanceMapping.get(accountName) == null) { + LOG.debug("The accountName is: {} ", accountName); + instance = new AbfsClientThrottlingIntercept(accountName); + instanceMapping.put(accountName, instance); + } else { + instance = instanceMapping.get(accountName); + } } + return instance; + } } 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 0000000000000..0a631b8127cbd --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOperationMetrics.java @@ -0,0 +1,74 @@ +/** + * 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 { + + 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(); + } + + AtomicLong getBytesFailed() { + return bytesFailed; + } + + AtomicLong getBytesSuccessful() { + return bytesSuccessful; + } + + AtomicLong getOperationsFailed() { + return operationsFailed; + } + + AtomicLong getOperationsSuccessful() { + return operationsSuccessful; + } + + long getEndTime() { + return endTime; + } + + void setEndTime(final long endTime) { + this.endTime = endTime; + } + + long getStartTime() { + return startTime; + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 76e783ddf7325..737b4ea0eb3ef 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -50,8 +50,6 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; -import org.apache.hadoop.fs.azurebfs.services.AbfsClientThrottlingIntercept; -import org.apache.hadoop.fs.azurebfs.services.AbfsClientThrottlingInterceptFactory; import org.apache.hadoop.fs.azurebfs.services.TestAbfsClient; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; @@ -484,66 +482,6 @@ private AzureBlobFileSystemStore setAzureBlobSystemStoreField( return abfsStore; } - @Test - public void testCreatMultipleAccountThrottling() - throws Throwable { - final AzureBlobFileSystem currentFs = getFileSystem(); - Configuration config = new Configuration(this.getRawConfiguration()); - String fileSystemName = TEST_CONTAINER_PREFIX + UUID.randomUUID().toString(); - String accountName = config.get(FS_AZURE_ACCOUNT_NAME); - if (accountName == null) { - // check if accountName is set using different config key - accountName = config.get(FS_AZURE_ABFS_ACCOUNT1_NAME); - } - assumeTrue("Not set: " + FS_AZURE_ABFS_ACCOUNT1_NAME, - accountName != null && !accountName.isEmpty()); - - final String abfsUrl = fileSystemName + "@" + accountName; - URI defaultUri = null; - String abfsScheme = "abfss"; - try { - defaultUri = new URI(abfsScheme, abfsUrl, null, null, null); - } catch (Exception ex) { - throw new AssertionError(ex); - } - - Configuration rawConfig1 = new Configuration(); - rawConfig1.addResource(TEST_CONFIGURATION_FILE_NAME); - - AbfsConfiguration abfsConfig = new AbfsConfiguration(rawConfig1, accountName); - - final AzureBlobFileSystem fs = - (AzureBlobFileSystem) FileSystem.newInstance(defaultUri, - rawConfig1); - System.out.println(fs.toString()); - - AbfsClient mockClient = TestAbfsClient.getMockAbfsClient(fs.getAbfsStore().getClient(), fs.getAbfsStore().getAbfsConfiguration()); - - AzureBlobFileSystemStore abfsStore = fs.getAbfsStore(); - abfsStore = setAzureBlobSystemStoreField(abfsStore, "client", mockClient); - - AbfsRestOperation successOp = mock(AbfsRestOperation.class); - AbfsHttpOperation http500Op = mock(AbfsHttpOperation.class); - when(http500Op.getStatusCode()).thenReturn(HTTP_INTERNAL_ERROR); - when(successOp.getResult()).thenReturn(http500Op); - - AbfsClientThrottlingIntercept instance1 = AbfsClientThrottlingInterceptFactory.getInstance(accountName, true, true); - String accountName1 = config.get(FS_AZURE_ABFS_ACCOUNT1_NAME); - AbfsClientThrottlingIntercept instance2 = AbfsClientThrottlingInterceptFactory.getInstance(accountName1, true, true); - //if singleton is enabled, for different accounts both the instances should return same value - assertEquals(instance1, instance2); - - AbfsClientThrottlingIntercept instance3 = AbfsClientThrottlingInterceptFactory.getInstance(accountName, true, false); - AbfsClientThrottlingIntercept instance4 = AbfsClientThrottlingInterceptFactory.getInstance(accountName1, true, false); - AbfsClientThrottlingIntercept instance5 = AbfsClientThrottlingInterceptFactory.getInstance(accountName, true, false); - - //if singleton is not enabled, for different accounts instances should return different value - assertNotEquals(instance3, instance4); - - //if singleton is not enabled, for same accounts instances should return same value - assertEquals(instance3, instance5); - } - private void validateCreateFileException(final Class exceptionClass, final AzureBlobFileSystemStore abfsStore) throws Exception { FsPermission permission = new FsPermission(FsAction.ALL, FsAction.ALL, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java index 0f8dc55aa14a4..894b293f3c773 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java @@ -18,10 +18,20 @@ package org.apache.hadoop.fs.azurebfs.services; +import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_BACKOFF_INTERVAL; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_BACKOFF_INTERVAL; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_IO_RETRIES; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MIN_BACKOFF_INTERVAL; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT1_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONTAINER_PREFIX; +import static org.junit.Assume.assumeTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; import java.util.Random; @@ -29,9 +39,10 @@ import org.junit.Test; import org.apache.hadoop.conf.Configuration; - +import java.lang.reflect.Field; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; +import org.apache.hadoop.fs.FileSystem; /** * Unit test TestExponentialRetryPolicy. @@ -67,6 +78,59 @@ public void testDefaultMaxIORetryCount() throws Exception { testMaxIOConfig(abfsConfig); } + @Test + public void testCreateMultipleAccountThrottling() throws Throwable { + final AzureBlobFileSystem currentFs = getFileSystem(); + Configuration config = new Configuration(this.getRawConfiguration()); + String fileSystemName = TEST_CONTAINER_PREFIX + java.util.UUID.randomUUID().toString(); + String accountName = config.get(FS_AZURE_ACCOUNT_NAME); + if (accountName == null) { + // check if accountName is set using different config key + accountName = config.get(FS_AZURE_ABFS_ACCOUNT1_NAME); + } + assumeTrue("Not set: " + FS_AZURE_ABFS_ACCOUNT1_NAME, + accountName != null && !accountName.isEmpty()); + + final String abfsUrl = fileSystemName + "@" + accountName; + java.net.URI defaultUri = null; + String abfsScheme = "abfss"; + try { + defaultUri = new java.net.URI(abfsScheme, abfsUrl, null, null, null); + } catch (Exception ex) { + throw new AssertionError(ex); + } + + Configuration rawConfig1 = new Configuration(); + rawConfig1.addResource(TEST_CONFIGURATION_FILE_NAME); + + AbfsConfiguration abfsConfig = new AbfsConfiguration(rawConfig1, accountName); + + final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(defaultUri, rawConfig1); + + AbfsClient mockClient = TestAbfsClient.getMockAbfsClient(fs.getAbfsStore().getClient(), fs.getAbfsStore().getAbfsConfiguration()); + + AbfsRestOperation successOp = mock(AbfsRestOperation.class); + AbfsHttpOperation http500Op = mock(AbfsHttpOperation.class); + when(http500Op.getStatusCode()).thenReturn(HTTP_INTERNAL_ERROR); + when(successOp.getResult()).thenReturn(http500Op); + + AbfsClientThrottlingIntercept instance1 = AbfsClientThrottlingInterceptFactory.getInstance(accountName, true, true); + String accountName1 = config.get(FS_AZURE_ABFS_ACCOUNT1_NAME); + AbfsClientThrottlingIntercept instance2 = AbfsClientThrottlingInterceptFactory.getInstance(accountName1, true, true); + //if singleton is enabled, for different accounts both the instances should return same value + assertEquals(instance1, instance2); + + AbfsClientThrottlingIntercept instance3 = AbfsClientThrottlingInterceptFactory.getInstance(accountName, true, false); + AbfsClientThrottlingIntercept instance4 = AbfsClientThrottlingInterceptFactory.getInstance(accountName1, true, false); + AbfsClientThrottlingIntercept instance5 = AbfsClientThrottlingInterceptFactory.getInstance(accountName, true, false); + + //if singleton is not enabled, for different accounts instances should return different value + assertNotEquals(instance3, instance4); + + //if singleton is not enabled, for same accounts instances should return same value + assertEquals(instance3, instance5); + } + @Test public void testAbfsConfigConstructor() throws Exception { // Ensure we choose expected values that are not defaults From 58c112345c753eed670dd55d0c2526e027e2133e Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Tue, 2 Aug 2022 11:50:05 +0530 Subject: [PATCH 19/53] remove unused imports --- .../services/AbfsClientThrottlingInterceptFactory.java | 2 +- .../fs/azurebfs/ITestAzureBlobFileSystemCreate.java | 8 -------- .../fs/azurebfs/services/TestExponentialRetryPolicy.java | 3 +-- 3 files changed, 2 insertions(+), 11 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java index 2fc8a69e8c585..618c25eac3bde 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java @@ -24,7 +24,7 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -class AbfsClientThrottlingInterceptFactory { +final class AbfsClientThrottlingInterceptFactory { private AbfsClientThrottlingInterceptFactory() { } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 737b4ea0eb3ef..6291ab0275a11 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -22,7 +22,6 @@ import java.io.FilterOutputStream; import java.io.IOException; import java.lang.reflect.Field; -import java.net.URI; import java.util.EnumSet; import java.util.UUID; @@ -37,13 +36,6 @@ import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.test.GenericTestUtils; - -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONTAINER_PREFIX; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT1_NAME; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_NAME; -import static org.junit.Assume.assumeTrue; - import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConcurrentWriteOperationDetectedException; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java index 894b293f3c773..521993f86a3ac 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java @@ -31,7 +31,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; -import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; + import java.util.Random; @@ -39,7 +39,6 @@ import org.junit.Test; import org.apache.hadoop.conf.Configuration; -import java.lang.reflect.Field; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; import org.apache.hadoop.fs.FileSystem; From aab31282886c5e90f15af5fd83982f73005957e7 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Mon, 22 Aug 2022 15:29:15 +0530 Subject: [PATCH 20/53] Fix imports --- .../services/TestExponentialRetryPolicy.java | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java index 521993f86a3ac..0c10184c13810 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java @@ -31,17 +31,15 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; - - +import java.net.URI; import java.util.Random; - import org.junit.Assert; import org.junit.Test; - +import java.util.UUID; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; -import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileSystem; /** * Unit test TestExponentialRetryPolicy. @@ -81,7 +79,7 @@ public void testDefaultMaxIORetryCount() throws Exception { public void testCreateMultipleAccountThrottling() throws Throwable { final AzureBlobFileSystem currentFs = getFileSystem(); Configuration config = new Configuration(this.getRawConfiguration()); - String fileSystemName = TEST_CONTAINER_PREFIX + java.util.UUID.randomUUID().toString(); + String fileSystemName = TEST_CONTAINER_PREFIX + UUID.randomUUID().toString(); String accountName = config.get(FS_AZURE_ACCOUNT_NAME); if (accountName == null) { // check if accountName is set using different config key @@ -91,10 +89,10 @@ public void testCreateMultipleAccountThrottling() throws Throwable { accountName != null && !accountName.isEmpty()); final String abfsUrl = fileSystemName + "@" + accountName; - java.net.URI defaultUri = null; + URI defaultUri = null; String abfsScheme = "abfss"; try { - defaultUri = new java.net.URI(abfsScheme, abfsUrl, null, null, null); + defaultUri = new URI(abfsScheme, abfsUrl, null, null, null); } catch (Exception ex) { throw new AssertionError(ex); } From 478021ad03dfab088575386246769648bfeef0cf Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Mon, 19 Sep 2022 15:21:07 +0530 Subject: [PATCH 21/53] Separate out account throttling --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 8 -- .../fs/azurebfs/AzureBlobFileSystem.java | 3 + .../azurebfs/constants/ConfigurationKeys.java | 1 - .../constants/FileSystemConfigurations.java | 1 - .../AbfsClientThrottlingAnalyzer.java | 44 ++++++++--- .../AbfsClientThrottlingIntercept.java | 31 ++------ .../AbfsClientThrottlingInterceptFactory.java | 59 --------------- .../services/AbfsOperationMetrics.java | 74 ------------------- .../azurebfs/services/AbfsRestOperation.java | 10 +-- .../hadoop-azure/src/site/markdown/abfs.md | 5 -- .../ITestAzureBlobFileSystemCreate.java | 1 + .../ITestAzureBlobFileSystemDelete.java | 4 +- .../constants/TestConfigurationKeys.java | 1 - .../services/TestExponentialRetryPolicy.java | 65 ---------------- 14 files changed, 48 insertions(+), 259 deletions(-) delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOperationMetrics.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index c7b293d5ee256..fd183052f2be4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -122,10 +122,6 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED) private boolean isExpectHeaderEnabled; - @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ACCOUNT_IS_SINGLETON_ENABLED, - DefaultValue = DEFAULT_FS_AZURE_ACCOUNT_IS_SINGLETON_ENABLED) - private boolean isSingletonEnabled; - @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_READ_BUFFER_SIZE, MinValue = MIN_BUFFER_SIZE, MaxValue = MAX_BUFFER_SIZE, @@ -702,10 +698,6 @@ public boolean isExpectHeaderEnabled() { return this.isExpectHeaderEnabled; } - public boolean isSingletonEnabled() { - return this.isSingletonEnabled; - } - public String getAzureInfiniteLeaseDirs() { return this.azureInfiniteLeaseDirs; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 1d2d9589b7fa2..d0bdd9818db24 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -55,6 +55,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.azurebfs.commit.ResilientCommitByRename; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientThrottlingIntercept; import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.classification.InterfaceStability; @@ -223,6 +224,8 @@ public void initialize(URI uri, Configuration configuration) LOG.debug("Created DelegationTokenManager {}", delegationTokenManager); } } + + AbfsClientThrottlingIntercept.initializeSingleton(abfsConfiguration.isAutoThrottlingEnabled()); rateLimiting = RateLimitingFactory.create(abfsConfiguration.getRateLimit()); LOG.debug("Initializing AzureBlobFileSystem for {} complete", uri); } 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 aff5b66332582..268542b47a5b6 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 @@ -36,7 +36,6 @@ public final class ConfigurationKeys { */ public static final String FS_AZURE_ACCOUNT_IS_HNS_ENABLED = "fs.azure.account.hns.enabled"; public static final String FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED = "fs.azure.account.expect.header.enabled"; - public static final String FS_AZURE_ACCOUNT_IS_SINGLETON_ENABLED = "fs.azure.account.singleton.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"; 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 ff915eaef2e10..c3a0ca9f52d7e 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 @@ -33,7 +33,6 @@ 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 boolean DEFAULT_FS_AZURE_ACCOUNT_IS_SINGLETON_ENABLED = false; public static final String USER_HOME_DIRECTORY_PREFIX = "/user"; private static final int SIXTY_SECONDS = 60 * 1000; 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 84abc5df37ec4..8a33b8f20ab37 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 @@ -22,7 +22,7 @@ import java.util.TimerTask; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; - +import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.util.Preconditions; import org.apache.commons.lang3.StringUtils; @@ -103,11 +103,11 @@ private AbfsClientThrottlingAnalyzer() { public void addBytesTransferred(long count, boolean isFailedOperation) { AbfsOperationMetrics metrics = blobMetrics.get(); if (isFailedOperation) { - metrics.getBytesFailed().addAndGet(count); - metrics.getOperationsFailed().incrementAndGet(); + metrics.bytesFailed.addAndGet(count); + metrics.operationsFailed.incrementAndGet(); } else { - metrics.getBytesSuccessful().addAndGet(count); - metrics.getOperationsSuccessful().incrementAndGet(); + metrics.bytesSuccessful.addAndGet(count); + metrics.operationsSuccessful.incrementAndGet(); } } @@ -136,16 +136,16 @@ int getSleepDuration() { private int analyzeMetricsAndUpdateSleepDuration(AbfsOperationMetrics metrics, int sleepDuration) { final double percentageConversionFactor = 100; - double bytesFailed = metrics.getBytesFailed().get(); - double bytesSuccessful = metrics.getBytesSuccessful().get(); - double operationsFailed = metrics.getOperationsFailed().get(); - double operationsSuccessful = metrics.getOperationsSuccessful().get(); + double bytesFailed = metrics.bytesFailed.get(); + double bytesSuccessful = metrics.bytesSuccessful.get(); + double operationsFailed = metrics.operationsFailed.get(); + double operationsSuccessful = metrics.operationsSuccessful.get(); double errorPercentage = (bytesFailed <= 0) ? 0 : (percentageConversionFactor * bytesFailed / (bytesFailed + bytesSuccessful)); - long periodMs = metrics.getEndTime() - metrics.getStartTime(); + long periodMs = metrics.endTime - metrics.startTime; double newSleepDuration; @@ -237,10 +237,10 @@ public void run() { } long now = System.currentTimeMillis(); - if (now - blobMetrics.get().getStartTime() >= analysisPeriodMs) { + if (now - blobMetrics.get().startTime >= analysisPeriodMs) { AbfsOperationMetrics oldMetrics = blobMetrics.getAndSet( new AbfsOperationMetrics(now)); - oldMetrics.setEndTime(now); + oldMetrics.endTime = now; sleepDuration = analyzeMetricsAndUpdateSleepDuration(oldMetrics, sleepDuration); } @@ -252,4 +252,24 @@ 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 60570cf81fb00..7303e833418db 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 @@ -46,20 +46,6 @@ public final class AbfsClientThrottlingIntercept { private AbfsClientThrottlingAnalyzer readThrottler = null; private AbfsClientThrottlingAnalyzer writeThrottler = null; private static boolean isAutoThrottlingEnabled = false; - private String accountName = ""; - - private synchronized void setIsAutoThrottlingEnabled(boolean autoThrottlingEnabled) { - isAutoThrottlingEnabled = autoThrottlingEnabled; - } - - // Hide default constructor - public AbfsClientThrottlingIntercept(String accountName) { - setIsAutoThrottlingEnabled(true); - LOG.debug("Client-side throttling is enabled for the ABFS file system."); - this.readThrottler = new AbfsClientThrottlingAnalyzer("read"); - this.writeThrottler = new AbfsClientThrottlingAnalyzer("write"); - this.accountName = accountName; - } // Hide default constructor private AbfsClientThrottlingIntercept() { @@ -67,19 +53,18 @@ private AbfsClientThrottlingIntercept() { writeThrottler = new AbfsClientThrottlingAnalyzer("write"); } - public static synchronized AbfsClientThrottlingIntercept initializeSingleton(boolean enableAutoThrottling) { + public static synchronized void initializeSingleton(boolean enableAutoThrottling) { if (!enableAutoThrottling) { - return null; + return; } if (singleton == null) { singleton = new AbfsClientThrottlingIntercept(); isAutoThrottlingEnabled = true; LOG.debug("Client-side throttling is enabled for the ABFS file system."); } - return singleton; } - void updateMetrics(AbfsRestOperationType operationType, + static void updateMetrics(AbfsRestOperationType operationType, AbfsHttpOperation abfsHttpOperation) { if (!isAutoThrottlingEnabled || abfsHttpOperation == null) { return; @@ -97,7 +82,7 @@ void updateMetrics(AbfsRestOperationType operationType, case Append: contentLength = abfsHttpOperation.getBytesSent(); if (contentLength > 0) { - this.writeThrottler.addBytesTransferred(contentLength, + singleton.writeThrottler.addBytesTransferred(contentLength, isFailedOperation); } break; @@ -105,7 +90,7 @@ void updateMetrics(AbfsRestOperationType operationType, String range = abfsHttpOperation.getConnection().getRequestProperty(HttpHeaderConfigurations.RANGE); contentLength = getContentLengthIfKnown(range); if (contentLength > 0) { - this.readThrottler.addBytesTransferred(contentLength, + singleton.readThrottler.addBytesTransferred(contentLength, isFailedOperation); } break; @@ -119,7 +104,7 @@ void updateMetrics(AbfsRestOperationType operationType, * uses this to suspend the request, if necessary, to minimize errors and * maximize throughput. */ - void sendingRequest(AbfsRestOperationType operationType, + static void sendingRequest(AbfsRestOperationType operationType, AbfsCounters abfsCounters) { if (!isAutoThrottlingEnabled) { return; @@ -127,13 +112,13 @@ void sendingRequest(AbfsRestOperationType operationType, switch (operationType) { case ReadFile: - if (this.readThrottler.suspendIfNecessary() + if (singleton.readThrottler.suspendIfNecessary() && abfsCounters != null) { abfsCounters.incrementCounter(AbfsStatistic.READ_THROTTLES, 1); } break; case Append: - if (this.writeThrottler.suspendIfNecessary() + if (singleton.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/AbfsClientThrottlingInterceptFactory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java deleted file mode 100644 index 618c25eac3bde..0000000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingInterceptFactory.java +++ /dev/null @@ -1,59 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.services; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -final class AbfsClientThrottlingInterceptFactory { - - private AbfsClientThrottlingInterceptFactory() { - } - - private static final Logger LOG = LoggerFactory.getLogger( - AbfsClientThrottlingInterceptFactory.class); - - private static Map instanceMapping - = new ConcurrentHashMap<>(); - - static synchronized AbfsClientThrottlingIntercept getInstance(String accountName, - boolean isAutoThrottlingEnabled, - boolean isSingletonEnabled) { - AbfsClientThrottlingIntercept instance; - if (isSingletonEnabled) { - instance = AbfsClientThrottlingIntercept.initializeSingleton( - isAutoThrottlingEnabled); - } else { - if (!isAutoThrottlingEnabled) { - return null; - } - if (instanceMapping.get(accountName) == null) { - LOG.debug("The accountName is: {} ", accountName); - instance = new AbfsClientThrottlingIntercept(accountName); - instanceMapping.put(accountName, instance); - } else { - instance = instanceMapping.get(accountName); - } - } - return instance; - } -} 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 deleted file mode 100644 index 0a631b8127cbd..0000000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOperationMetrics.java +++ /dev/null @@ -1,74 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.services; - -import java.util.concurrent.atomic.AtomicLong; -/** - * Stores Abfs operation metrics during each analysis period. - */ -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(); - } - - AtomicLong getBytesFailed() { - return bytesFailed; - } - - AtomicLong getBytesSuccessful() { - return bytesSuccessful; - } - - AtomicLong getOperationsFailed() { - return operationsFailed; - } - - AtomicLong getOperationsSuccessful() { - return operationsSuccessful; - } - - long getEndTime() { - return endTime; - } - - void setEndTime(final long endTime) { - this.endTime = endTime; - } - - long getStartTime() { - return startTime; - } -} 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 5e69dd3a553b6..a98055459141b 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 @@ -242,10 +242,6 @@ private void completeExecute(TracingContext tracingContext) private boolean executeHttpOperation(final int retryCount, TracingContext tracingContext) throws AzureBlobFileSystemException { AbfsHttpOperation httpOperation = null; - String accountName = this.client.getAbfsConfiguration().getAccountName(); - boolean isAutoThrottlingEnabled = this.client.getAbfsConfiguration().isAutoThrottlingEnabled(); - boolean isSingletonEnabled = this.client.getAbfsConfiguration().isSingletonEnabled(); - AbfsClientThrottlingIntercept intercept; try { // initialize the HTTP request and open the connection httpOperation = new AbfsHttpOperation(url, method, requestHeaders); @@ -282,8 +278,7 @@ private boolean executeHttpOperation(final int retryCount, // dump the headers AbfsIoUtils.dumpHeadersToDebugLog("Request Headers", httpOperation.getConnection().getRequestProperties()); - intercept = AbfsClientThrottlingInterceptFactory.getInstance(accountName, isAutoThrottlingEnabled, isSingletonEnabled); - intercept.sendingRequest(operationType, abfsCounters); + AbfsClientThrottlingIntercept.sendingRequest(operationType, abfsCounters); if (hasRequestBody) { // HttpUrlConnection requires @@ -322,8 +317,7 @@ private boolean executeHttpOperation(final int retryCount, return false; } finally { - AbfsClientThrottlingIntercept instance = AbfsClientThrottlingInterceptFactory.getInstance(accountName, isAutoThrottlingEnabled, isSingletonEnabled); - instance.updateMetrics(operationType, httpOperation); + AbfsClientThrottlingIntercept.updateMetrics(operationType, httpOperation); } LOG.debug("HttpRequest: {}: {}", operationType, httpOperation); diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md index 0c858441d0348..0fd02c230fd59 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md @@ -774,11 +774,6 @@ 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. -`fs.azure.account.singleton.enabled`: This config is used to specify whether you -want to enable throttling at account level or not. Otherwise a single throttling -class level instance is created. It is configured to true by default. We need to -configure it to false if we want account level throttling. - ### 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, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 6291ab0275a11..2f23ac5c5c708 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -36,6 +36,7 @@ import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.test.GenericTestUtils; + import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConcurrentWriteOperationDetectedException; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index cf47f5e8d9293..d6cf2a83fd348 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -223,8 +223,8 @@ public void testDeleteIdempotency() throws Exception { public void testDeleteIdempotencyTriggerHttp404() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); AbfsClient client = TestAbfsClient.createTestClientFromCurrentContext( - fs.getAbfsStore().getClient(), - this.getConfiguration()); + fs.getAbfsStore().getClient(), + this.getConfiguration()); // Case 1: Not a retried case should throw error back // Add asserts at AzureBlobFileSystemStore and AbfsClient levels diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java index 2c659e1399a4f..565eb38c4f70a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java @@ -24,7 +24,6 @@ public final class TestConfigurationKeys { public static final String FS_AZURE_ACCOUNT_NAME = "fs.azure.account.name"; public static final String FS_AZURE_ABFS_ACCOUNT_NAME = "fs.azure.abfs.account.name"; - public static final String FS_AZURE_ABFS_ACCOUNT1_NAME = "fs.azure.abfs.account1.name"; public static final String FS_AZURE_ACCOUNT_KEY = "fs.azure.account.key"; public static final String FS_AZURE_CONTRACT_TEST_URI = "fs.contract.test.fs.abfs"; public static final String FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT = "fs.azure.test.namespace.enabled"; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java index 0c10184c13810..3df1143f73ec6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java @@ -18,28 +18,16 @@ package org.apache.hadoop.fs.azurebfs.services; -import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_BACKOFF_INTERVAL; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_BACKOFF_INTERVAL; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_IO_RETRIES; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MIN_BACKOFF_INTERVAL; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT1_NAME; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_NAME; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONTAINER_PREFIX; -import static org.junit.Assume.assumeTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; -import java.net.URI; import java.util.Random; import org.junit.Assert; import org.junit.Test; -import java.util.UUID; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; -import org.apache.hadoop.fs.FileSystem; /** * Unit test TestExponentialRetryPolicy. @@ -75,59 +63,6 @@ public void testDefaultMaxIORetryCount() throws Exception { testMaxIOConfig(abfsConfig); } - @Test - public void testCreateMultipleAccountThrottling() throws Throwable { - final AzureBlobFileSystem currentFs = getFileSystem(); - Configuration config = new Configuration(this.getRawConfiguration()); - String fileSystemName = TEST_CONTAINER_PREFIX + UUID.randomUUID().toString(); - String accountName = config.get(FS_AZURE_ACCOUNT_NAME); - if (accountName == null) { - // check if accountName is set using different config key - accountName = config.get(FS_AZURE_ABFS_ACCOUNT1_NAME); - } - assumeTrue("Not set: " + FS_AZURE_ABFS_ACCOUNT1_NAME, - accountName != null && !accountName.isEmpty()); - - final String abfsUrl = fileSystemName + "@" + accountName; - URI defaultUri = null; - String abfsScheme = "abfss"; - try { - defaultUri = new URI(abfsScheme, abfsUrl, null, null, null); - } catch (Exception ex) { - throw new AssertionError(ex); - } - - Configuration rawConfig1 = new Configuration(); - rawConfig1.addResource(TEST_CONFIGURATION_FILE_NAME); - - AbfsConfiguration abfsConfig = new AbfsConfiguration(rawConfig1, accountName); - - final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(defaultUri, rawConfig1); - - AbfsClient mockClient = TestAbfsClient.getMockAbfsClient(fs.getAbfsStore().getClient(), fs.getAbfsStore().getAbfsConfiguration()); - - AbfsRestOperation successOp = mock(AbfsRestOperation.class); - AbfsHttpOperation http500Op = mock(AbfsHttpOperation.class); - when(http500Op.getStatusCode()).thenReturn(HTTP_INTERNAL_ERROR); - when(successOp.getResult()).thenReturn(http500Op); - - AbfsClientThrottlingIntercept instance1 = AbfsClientThrottlingInterceptFactory.getInstance(accountName, true, true); - String accountName1 = config.get(FS_AZURE_ABFS_ACCOUNT1_NAME); - AbfsClientThrottlingIntercept instance2 = AbfsClientThrottlingInterceptFactory.getInstance(accountName1, true, true); - //if singleton is enabled, for different accounts both the instances should return same value - assertEquals(instance1, instance2); - - AbfsClientThrottlingIntercept instance3 = AbfsClientThrottlingInterceptFactory.getInstance(accountName, true, false); - AbfsClientThrottlingIntercept instance4 = AbfsClientThrottlingInterceptFactory.getInstance(accountName1, true, false); - AbfsClientThrottlingIntercept instance5 = AbfsClientThrottlingInterceptFactory.getInstance(accountName, true, false); - - //if singleton is not enabled, for different accounts instances should return different value - assertNotEquals(instance3, instance4); - - //if singleton is not enabled, for same accounts instances should return same value - assertEquals(instance3, instance5); - } - @Test public void testAbfsConfigConstructor() throws Exception { // Ensure we choose expected values that are not defaults From 9fbb4ded5d5924fa487308cd54a68a6b75ec9276 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Tue, 25 Oct 2022 00:49:32 -0700 Subject: [PATCH 22/53] Documentation added --- .../fs/azurebfs/services/AbfsClient.java | 23 ++++++++++++++----- .../AbfsClientThrottlingAnalyzer.java | 4 ++-- .../ITestAzureBlobFileSystemDelete.java | 1 + .../services/TestExponentialRetryPolicy.java | 4 ++++ 4 files changed, 24 insertions(+), 8 deletions(-) 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 bd4f8159707c1..f61e172c324e3 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 @@ -693,13 +693,12 @@ public AbfsRestOperation append(final String path, final byte[] buffer, op.execute(tracingContext); } catch (AzureBlobFileSystemException e) { /* - If the http response code indicates a user error we retry the same append request with expect header disabled + If the http response code indicates a user error we retry the same append request with expect header disabled. + When "100-continue" header is enabled but a non Http 100 response comes, JDK fails to provide all response headers. + 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. */ - if ((((AbfsRestOperationException) e).getStatusCode() - >= HttpURLConnection.HTTP_BAD_REQUEST - && ((AbfsRestOperationException) e).getStatusCode() - < HttpURLConnection.HTTP_INTERNAL_ERROR) - && reqParams.getIsExpectHeaderEnabled()) { + if (checkUserError(e) && reqParams.getIsExpectHeaderEnabled()) { reqParams.setExpectHeaderEnabled(false); return this.append(path, buffer, reqParams, cachedSasToken, tracingContext); @@ -730,6 +729,18 @@ && appendSuccessCheckOp(op, path, return op; } + /** + * Returns true if the status code lies in the range of user error + * @param e Exception caught + * @return True or False + */ + private boolean checkUserError(AzureBlobFileSystemException e) { + return ((AbfsRestOperationException) e).getStatusCode() + >= HttpURLConnection.HTTP_BAD_REQUEST + && ((AbfsRestOperationException) e).getStatusCode() + < HttpURLConnection.HTTP_INTERNAL_ERROR; + } + // 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). 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 8a33b8f20ab37..6dfd352954d34 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 @@ -21,8 +21,9 @@ import java.util.Timer; import java.util.TimerTask; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.util.Preconditions; import org.apache.commons.lang3.StringUtils; @@ -271,5 +272,4 @@ static class AbfsOperationMetrics { this.operationsSuccessful = new AtomicLong(); } } - } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index d6cf2a83fd348..db181fb5dd660 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -221,6 +221,7 @@ public void testDeleteIdempotency() throws Exception { @Test public void testDeleteIdempotencyTriggerHttp404() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); AbfsClient client = TestAbfsClient.createTestClientFromCurrentContext( fs.getAbfsStore().getClient(), diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java index 3df1143f73ec6..0f8dc55aa14a4 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java @@ -22,10 +22,14 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_BACKOFF_INTERVAL; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_IO_RETRIES; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MIN_BACKOFF_INTERVAL; + import java.util.Random; + import org.junit.Assert; import org.junit.Test; + import org.apache.hadoop.conf.Configuration; + import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; From 7c43202841c8b13ed420ce611ffd305b1c792deb Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Tue, 25 Oct 2022 00:58:13 -0700 Subject: [PATCH 23/53] Formatting --- .../hadoop/fs/azurebfs/services/AbfsHttpOperation.java | 2 ++ .../hadoop/fs/azurebfs/services/AbfsRestOperation.java | 1 + hadoop-tools/hadoop-azure/src/site/markdown/abfs.md | 5 ++--- 3 files changed, 5 insertions(+), 3 deletions(-) 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 f1fd769bc61f9..f63cf0b0e6751 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 @@ -266,6 +266,7 @@ public AbfsHttpOperation(final URL url, final String method, final List 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. +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. ### HNS Check Options Config `fs.azure.account.hns.enabled` provides an option to specify whether From 3fc18b97de71dd932b0003e858179606cc8e1672 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Fri, 28 Oct 2022 04:24:33 -0700 Subject: [PATCH 24/53] Addressed PR comments --- .../testrun-scripts/testsupport.sh | 44 ++++++++++--------- .../fs/azurebfs/AzureBlobFileSystemStore.java | 2 +- .../azurebfs/constants/AbfsHttpConstants.java | 2 + .../constants/HttpHeaderConfigurations.java | 2 +- .../services/AppendRequestParameters.java | 3 +- .../fs/azurebfs/services/AbfsClient.java | 34 +++++++------- .../azurebfs/services/AbfsHttpOperation.java | 3 ++ .../src/test/resources/log4j.properties | 1 + 8 files changed, 51 insertions(+), 40 deletions(-) diff --git a/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh b/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh index 28f96edd27354..e051fe0f146a7 100644 --- a/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh +++ b/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh @@ -75,29 +75,29 @@ ENDOFFILE testlogfilename="$testOutputLogFolder/Test-Logs-$combination.txt" touch "$testlogfilename" - if [ "$runTest" == true ] - then - STARTTIME=$(date +%s) - echo "Running test for combination $combination on account $accountName [ProcessCount=$processcount]" - logOutput "Test run report can be seen in $testlogfilename" - mvn -T 1C -Dparallel-tests=abfs -Dscale -DtestsThreadCount="$processcount" verify >> "$testlogfilename" || true - ENDTIME=$(date +%s) - summary - fi - - if [ "$cleanUpTestContainers" == true ] - then - mvn test -Dtest=org.apache.hadoop.fs.azurebfs.utils.CleanupTestContainers >> "$testlogfilename" || true - if grep -q "There are test failures" "$testlogfilename"; - then logOutput "ERROR: All test containers could not be deleted. Detailed error cause in $testlogfilename" - pcregrep -M "$testresultsregex" "$testlogfilename" - exit 0 + if [ "$runTest" == true ] + then + STARTTIME=$(date +%s) + echo "Running test for combination $combination on account $accountName [ProcessCount=$processcount]" + logOutput "Test run report can be seen in $testlogfilename" + mvn -T 1C -Dparallel-tests=abfs -Dscale -DtestsThreadCount="$processcount" verify >> "$testlogfilename" || true + ENDTIME=$(date +%s) + summary fi - logOutput "Delete test containers - complete. Test run logs in - $testlogfilename" - fi + if [ "$cleanUpTestContainers" == true ] + then + mvn test -Dtest=org.apache.hadoop.fs.azurebfs.utils.CleanupTestContainers >> "$testlogfilename" || true + if grep -q "There are test failures" "$testlogfilename"; + then logOutput "ERROR: All test containers could not be deleted. Detailed error cause in $testlogfilename" + pcregrep -M "$testresultsregex" "$testlogfilename" + exit 0 + fi -} + logOutput "Delete test containers - complete. Test run logs in - $testlogfilename" + fi + + } summary() { { @@ -169,4 +169,8 @@ init() { logOutput() { echo -e "$outputFormatOn" "$1" "$outputFormatOff" +<<<<<<< Updated upstream +} +======= } +>>>>>>> Stashed changes diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 2c22b80a5bf8d..6936ac0d51670 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -693,8 +693,8 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext( } return new AbfsOutputStreamContext(abfsConfiguration.getSasTokenRenewPeriodForStreamsInSeconds()) .withWriteBufferSize(bufferSize) - .enableFlush(abfsConfiguration.isFlushEnabled()) .enableExpectHeader(abfsConfiguration.isExpectHeaderEnabled()) + .enableFlush(abfsConfiguration.isFlushEnabled()) .enableSmallWriteOptimization(abfsConfiguration.isSmallWriteOptimizationEnabled()) .disableOutputStreamFlush(abfsConfiguration.isOutputStreamFlushDisabled()) .withStreamStatistics(new AbfsOutputStreamStatisticsImpl()) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java index e82f38bd5f068..04551be270943 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java @@ -103,6 +103,8 @@ public final class AbfsHttpConstants { public static final String DEFAULT_SCOPE = "default:"; public static final String PERMISSION_FORMAT = "%04d"; public static final String SUPER_USER = "$superuser"; + //The HTTP 100 Continue informational status response code indicates that everything so far + // is OK and that the client should continue with the request or ignore it if it is already finished. public static final String HUNDRED_CONTINUE = "100-continue"; public static final char CHAR_FORWARD_SLASH = '/'; 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 ec68f1f923680..b123e90170e69 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 @@ -60,7 +60,6 @@ public final class HttpHeaderConfigurations { public static final String X_MS_UMASK = "x-ms-umask"; public static final String X_MS_NAMESPACE_ENABLED = "x-ms-namespace-enabled"; public static final String X_MS_ABFS_CLIENT_LATENCY = "x-ms-abfs-client-latency"; - public static final String EXPECT = "Expect"; public static final String X_MS_ENCRYPTION_KEY = "x-ms-encryption-key"; public static final String X_MS_ENCRYPTION_KEY_SHA256 = "x-ms-encryption-key-sha256"; public static final String X_MS_ENCRYPTION_ALGORITHM = "x-ms-encryption-algorithm"; @@ -71,6 +70,7 @@ public final class HttpHeaderConfigurations { public static final String X_MS_LEASE_ID = "x-ms-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 EXPECT = "Expect"; private HttpHeaderConfigurations() {} } 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 579535a16f926..e308f13a6b9a0 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 @@ -41,7 +41,8 @@ public AppendRequestParameters(final long position, final int length, final Mode mode, final boolean isAppendBlob, - final String leaseId, boolean isExpectHeaderEnabled) { + final String leaseId, + final boolean isExpectHeaderEnabled) { this.position = position; this.offset = offset; this.length = length; 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 f61e172c324e3..b8c4d03d25c46 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 @@ -651,11 +651,11 @@ public AbfsRestOperation append(final String path, final byte[] buffer, throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); addCustomerProvidedKeyHeaders(requestHeaders); - // 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. if (reqParams.getIsExpectHeaderEnabled()) { 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, HTTP_METHOD_PATCH)); if (reqParams.getLeaseId() != null) { @@ -692,13 +692,15 @@ public AbfsRestOperation append(final String path, final byte[] buffer, 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 disabled. - When "100-continue" header is enabled but a non Http 100 response comes, JDK fails to provide all response headers. - 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. - */ - if (checkUserError(e) && reqParams.getIsExpectHeaderEnabled()) { + // If the http response code indicates a user error we retry + // the same append request with expect header disabled. + // When "100-continue" header is enabled but a non Http 100 response comes, + // JDK fails to provide all response headers. + // 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.getIsExpectHeaderEnabled()) { reqParams.setExpectHeaderEnabled(false); return this.append(path, buffer, reqParams, cachedSasToken, tracingContext); @@ -730,15 +732,13 @@ && appendSuccessCheckOp(op, path, } /** - * Returns true if the status code lies in the range of user error - * @param e Exception caught - * @return True or False + * 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(AzureBlobFileSystemException e) { - return ((AbfsRestOperationException) e).getStatusCode() - >= HttpURLConnection.HTTP_BAD_REQUEST - && ((AbfsRestOperationException) e).getStatusCode() - < HttpURLConnection.HTTP_INTERNAL_ERROR; + private boolean checkUserError(int responseStatusCode) { + return (responseStatusCode >= HttpURLConnection.HTTP_BAD_REQUEST + && responseStatusCode < HttpURLConnection.HTTP_INTERNAL_ERROR); } // For AppendBlob its possible that the append succeeded in the backend but the request failed. 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 f63cf0b0e6751..d3db6c2fbfab8 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 @@ -321,6 +321,9 @@ public void sendRequest(byte[] buffer, int offset, int length) throws IOExceptio this.bytesSent = length; outputStream.write(buffer, offset, length); } catch (IOException e) { + // If getOutputStream fails with an exception due to 100-continue + // enabled, we update the bytes sent before they are sent + // in the catch block. this.bytesSent = length; } finally { if (this.isTraceEnabled) { diff --git a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties index 9f72d03653306..daca5870dd5b2 100644 --- a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties @@ -26,6 +26,7 @@ log4j.logger.org.apache.hadoop.fs.azure.AzureFileSystemThreadPoolExecutor=DEBUG log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG log4j.logger.org.apache.hadoop.fs.azurebfs.contracts.services.TracingService=TRACE log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsClient=DEBUG +log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsIoUtils=DEBUG # after here: turn off log messages from other parts of the system # which only clutter test reports. From aaca1c1d22bd50b542907565bedc3e0c1f00767b Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Fri, 28 Oct 2022 04:27:43 -0700 Subject: [PATCH 25/53] Addressed PR comments --- .../testrun-scripts/testsupport.sh | 46 +++++++++---------- .../src/test/resources/log4j.properties | 1 - 2 files changed, 21 insertions(+), 26 deletions(-) diff --git a/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh b/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh index e051fe0f146a7..a54c0af4fa305 100644 --- a/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh +++ b/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh @@ -75,29 +75,29 @@ ENDOFFILE testlogfilename="$testOutputLogFolder/Test-Logs-$combination.txt" touch "$testlogfilename" - if [ "$runTest" == true ] - then - STARTTIME=$(date +%s) - echo "Running test for combination $combination on account $accountName [ProcessCount=$processcount]" - logOutput "Test run report can be seen in $testlogfilename" - mvn -T 1C -Dparallel-tests=abfs -Dscale -DtestsThreadCount="$processcount" verify >> "$testlogfilename" || true - ENDTIME=$(date +%s) - summary - fi - - if [ "$cleanUpTestContainers" == true ] - then - mvn test -Dtest=org.apache.hadoop.fs.azurebfs.utils.CleanupTestContainers >> "$testlogfilename" || true - if grep -q "There are test failures" "$testlogfilename"; - then logOutput "ERROR: All test containers could not be deleted. Detailed error cause in $testlogfilename" - pcregrep -M "$testresultsregex" "$testlogfilename" - exit 0 - fi + if [ "$runTest" == true ] + then + STARTTIME=$(date +%s) + echo "Running test for combination $combination on account $accountName [ProcessCount=$processcount]" + logOutput "Test run report can be seen in $testlogfilename" + mvn -T 1C -Dparallel-tests=abfs -Dscale -DtestsThreadCount="$processcount" verify >> "$testlogfilename" || true + ENDTIME=$(date +%s) + summary + fi - logOutput "Delete test containers - complete. Test run logs in - $testlogfilename" + if [ "$cleanUpTestContainers" == true ] + then + mvn test -Dtest=org.apache.hadoop.fs.azurebfs.utils.CleanupTestContainers >> "$testlogfilename" || true + if grep -q "There are test failures" "$testlogfilename"; + then logOutput "ERROR: All test containers could not be deleted. Detailed error cause in $testlogfilename" + pcregrep -M "$testresultsregex" "$testlogfilename" + exit 0 fi - } + logOutput "Delete test containers - complete. Test run logs in - $testlogfilename" + fi + +} summary() { { @@ -169,8 +169,4 @@ init() { logOutput() { echo -e "$outputFormatOn" "$1" "$outputFormatOff" -<<<<<<< Updated upstream -} -======= -} ->>>>>>> Stashed changes +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties index daca5870dd5b2..9f72d03653306 100644 --- a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties @@ -26,7 +26,6 @@ log4j.logger.org.apache.hadoop.fs.azure.AzureFileSystemThreadPoolExecutor=DEBUG log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG log4j.logger.org.apache.hadoop.fs.azurebfs.contracts.services.TracingService=TRACE log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsClient=DEBUG -log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsIoUtils=DEBUG # after here: turn off log messages from other parts of the system # which only clutter test reports. From 135e04f29366db626f5dea06b21cf2cf750b103a Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Fri, 28 Oct 2022 04:29:21 -0700 Subject: [PATCH 26/53] Addressed PR comments --- .../hadoop-azure/dev-support/testrun-scripts/testsupport.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh b/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh index a54c0af4fa305..28f96edd27354 100644 --- a/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh +++ b/hadoop-tools/hadoop-azure/dev-support/testrun-scripts/testsupport.sh @@ -169,4 +169,4 @@ init() { logOutput() { echo -e "$outputFormatOn" "$1" "$outputFormatOff" -} \ No newline at end of file +} From 2a1834f379af1f232f0310e59fc6e77758cb68a2 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Tue, 29 Nov 2022 03:01:28 -0800 Subject: [PATCH 27/53] Addressed PR comments --- .../services/AppendRequestParameters.java | 2 +- .../hadoop/fs/azurebfs/services/AbfsClient.java | 5 +++-- .../fs/azurebfs/services/AbfsHttpOperation.java | 17 +++++++++++------ 3 files changed, 15 insertions(+), 9 deletions(-) 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 e308f13a6b9a0..57e559a60ec84 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 @@ -76,7 +76,7 @@ public String getLeaseId() { return this.leaseId; } - public boolean getIsExpectHeaderEnabled() { + public boolean isExpectHeaderEnabled() { return isExpectHeaderEnabled; } 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 b8c4d03d25c46..0a2049423ff9e 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 @@ -651,7 +651,7 @@ public AbfsRestOperation append(final String path, final byte[] buffer, throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); addCustomerProvidedKeyHeaders(requestHeaders); - if (reqParams.getIsExpectHeaderEnabled()) { + if (reqParams.isExpectHeaderEnabled()) { requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); } // JDK7 does not support PATCH, so to workaround the issue we will use @@ -700,7 +700,8 @@ public AbfsRestOperation append(final String path, final byte[] buffer, // 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.getIsExpectHeaderEnabled()) { + if (checkUserError(responseStatusCode) && reqParams.isExpectHeaderEnabled()) { + LOG.debug("User error, retrying without 100 continue enabled"); reqParams.setExpectHeaderEnabled(false); return this.append(path, buffer, reqParams, cachedSasToken, tracingContext); 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 d3db6c2fbfab8..1a5ebf4bf219a 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 @@ -314,17 +314,22 @@ public void sendRequest(byte[] buffer, int offset, int length) throws IOExceptio if (this.isTraceEnabled) { startTime = System.nanoTime(); } - try (OutputStream outputStream = this.connection.getOutputStream()) { + OutputStream outputStream; + try { + try { + outputStream = this.connection.getOutputStream(); + } catch (IOException e) { + // If getOutputStream fails with an exception due to 100-continue + // enabled, we update the bytes sent before they are sent + // in the catch block. + this.bytesSent = length; + throw e; + } // update bytes sent before they are sent so we may observe // attempted sends as well as successful sends via the // accompanying statusCode this.bytesSent = length; outputStream.write(buffer, offset, length); - } catch (IOException e) { - // If getOutputStream fails with an exception due to 100-continue - // enabled, we update the bytes sent before they are sent - // in the catch block. - this.bytesSent = length; } finally { if (this.isTraceEnabled) { this.sendRequestTimeMs = elapsedTimeMs(startTime); From 443e263bf47f3a3ed406dee23d3f913988babf03 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Tue, 29 Nov 2022 08:47:18 -0800 Subject: [PATCH 28/53] Fix for exception --- .../fs/azurebfs/services/AbfsHttpOperation.java | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) 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 1a5ebf4bf219a..d225afa7402a7 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 @@ -320,17 +320,15 @@ public void sendRequest(byte[] buffer, int offset, int length) throws IOExceptio outputStream = this.connection.getOutputStream(); } catch (IOException e) { // If getOutputStream fails with an exception due to 100-continue - // enabled, we update the bytes sent before they are sent - // in the catch block. - this.bytesSent = length; - throw e; + // enabled, we return back without throwing an exception. + return; } - // update bytes sent before they are sent so we may observe - // attempted sends as well as successful sends via the - // accompanying statusCode - this.bytesSent = length; + // This will normally throw an IOException. outputStream.write(buffer, offset, length); } finally { + // update bytes sent for successful as well as failed attempts via the + // accompanying statusCode. + this.bytesSent = length; if (this.isTraceEnabled) { this.sendRequestTimeMs = elapsedTimeMs(startTime); } From 05df41cd10167924d513eb04abb0128b51f05bda Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Mon, 5 Dec 2022 14:26:11 +0530 Subject: [PATCH 29/53] Update AbfsConfiguration.java --- .../java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 5ffd715eb1874..756a6de44af7f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -713,7 +713,8 @@ public String getAppendBlobDirs() { public boolean isExpectHeaderEnabled() { return this.isExpectHeaderEnabled; - + } + public boolean accountThrottlingEnabled() { return accountThrottlingEnabled; } From 457fda0e5d4834b5687170023e502854d3e25b24 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Tue, 6 Dec 2022 02:35:15 -0800 Subject: [PATCH 30/53] Changes for exception handling --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 8 ++++---- .../azurebfs/services/AbfsHttpOperation.java | 19 +++++++++++++++---- 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 756a6de44af7f..124c4d9de722b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -118,10 +118,10 @@ public class AbfsConfiguration{ private boolean optimizeFooterRead; @BooleanConfigurationValidatorAnnotation( - ConfigurationKey = FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED, - DefaultValue = DEFAULT_FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED) + ConfigurationKey = FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED, + DefaultValue = DEFAULT_FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED) private boolean isExpectHeaderEnabled; - + @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED, DefaultValue = DEFAULT_FS_AZURE_ACCOUNT_LEVEL_THROTTLING_ENABLED) private boolean accountThrottlingEnabled; @@ -714,7 +714,7 @@ public String getAppendBlobDirs() { public boolean isExpectHeaderEnabled() { return this.isExpectHeaderEnabled; } - + public boolean accountThrottlingEnabled() { return accountThrottlingEnabled; } 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 d225afa7402a7..16c68583c912a 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 @@ -314,18 +314,29 @@ public void sendRequest(byte[] buffer, int offset, int length) throws IOExceptio if (this.isTraceEnabled) { startTime = System.nanoTime(); } - OutputStream outputStream; + OutputStream outputStream = null; try { try { outputStream = this.connection.getOutputStream(); } catch (IOException e) { - // If getOutputStream fails with an exception due to 100-continue - // enabled, we return back without throwing an exception. - return; + // If getOutputStream fails with an exception and 100-continue + // is enabled, we return back without throwing an exception + // because processResponse will give the correct status code + // based on which the retry logic will come into place. + String expectHeader = this.connection.getRequestProperty("Expect"); + if (expectHeader != null && expectHeader.equals("100-continue")) { + return; + } else { + throw e; + } } // This will normally throw an IOException. outputStream.write(buffer, offset, length); } finally { + // Closing the opened output stream + if (outputStream != null) { + outputStream.close(); + } // update bytes sent for successful as well as failed attempts via the // accompanying statusCode. this.bytesSent = length; From f2e6f522fb3d170cbf33abf0d0cdb348f925c43a Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Tue, 6 Dec 2022 21:46:34 -0800 Subject: [PATCH 31/53] String correction --- .../hadoop/fs/azurebfs/services/AbfsHttpOperation.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) 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 16c68583c912a..2457f8d094d4d 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 @@ -43,6 +43,9 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsPerfLoggable; import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; +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. */ @@ -323,8 +326,8 @@ public void sendRequest(byte[] buffer, int offset, int length) throws IOExceptio // is enabled, we return back without throwing an exception // because processResponse will give the correct status code // based on which the retry logic will come into place. - String expectHeader = this.connection.getRequestProperty("Expect"); - if (expectHeader != null && expectHeader.equals("100-continue")) { + String expectHeader = this.connection.getRequestProperty(EXPECT); + if (expectHeader != null && expectHeader.equals(HUNDRED_CONTINUE)) { return; } else { throw e; From fd006bdc2bac3b57966b88ab1ddd2448f1fa66e9 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Wed, 14 Dec 2022 21:56:56 -0800 Subject: [PATCH 32/53] Tests for hundred continue --- .../azurebfs/constants/AbfsHttpConstants.java | 2 +- .../InvalidAbfsRestOperationException.java | 45 ++++--- .../fs/azurebfs/services/AbfsClient.java | 2 +- .../azurebfs/services/AbfsHttpOperation.java | 7 +- .../azurebfs/services/AbfsRestOperation.java | 7 +- .../hadoop-azure/src/site/markdown/abfs.md | 2 + .../fs/azurebfs/services/TestAbfsClient.java | 20 ++++ .../services/TestAbfsHttpOperation.java | 11 ++ .../services/TestAbfsRestOperation.java | 110 ++++++++++++++++++ .../src/test/resources/log4j.properties | 1 + 10 files changed, 185 insertions(+), 22 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java index 04551be270943..3d8d5f8e78d38 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java @@ -103,7 +103,7 @@ public final class AbfsHttpConstants { public static final String DEFAULT_SCOPE = "default:"; public static final String PERMISSION_FORMAT = "%04d"; public static final String SUPER_USER = "$superuser"; - //The HTTP 100 Continue informational status response code indicates that everything so far + // The HTTP 100 Continue informational status response code indicates that everything so far // is OK and that the client should continue with the request or ignore it if it is already finished. public static final String HUNDRED_CONTINUE = "100-continue"; 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 cce2ad1d2ca90..9cc7b2389f0f8 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 @@ -30,24 +30,33 @@ @InterfaceAudience.Public @InterfaceStability.Evolving public class InvalidAbfsRestOperationException extends AbfsRestOperationException { - public InvalidAbfsRestOperationException( - final Exception innerException) { - super( - AzureServiceErrorCode.UNKNOWN.getStatusCode(), - AzureServiceErrorCode.UNKNOWN.getErrorCode(), - innerException != null - ? innerException.toString() - : "InvalidAbfsRestOperationException", - innerException); - } - public InvalidAbfsRestOperationException(final Exception innerException, int retryCount) { - super( - AzureServiceErrorCode.UNKNOWN.getStatusCode(), - AzureServiceErrorCode.UNKNOWN.getErrorCode(), - innerException != null - ? innerException.toString() - : "InvalidAbfsRestOperationException" + "RetryCount: " + String.valueOf(retryCount), - innerException); + private final static String errorMessage = "InvalidAbfsRestOperationException"; + + public InvalidAbfsRestOperationException( + final Exception innerException) { + super( + AzureServiceErrorCode.UNKNOWN.getStatusCode(), + AzureServiceErrorCode.UNKNOWN.getErrorCode(), + innerException != null + ? innerException.toString() + : errorMessage, + 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() + : errorMessage + " RetryCount: " + retryCount, + innerException); } } 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 aad935aabfe0e..579627a67a605 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 @@ -38,7 +38,6 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.store.LogExactlyOnce; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; @@ -78,6 +77,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME; 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; import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND; /** 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 2457f8d094d4d..b134de911b2f0 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 @@ -28,6 +28,7 @@ import javax.net.ssl.HttpsURLConnection; import javax.net.ssl.SSLSocketFactory; +import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.fs.azurebfs.utils.UriUtils; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; @@ -113,6 +114,10 @@ protected HttpURLConnection getConnection() { return connection; } + void setConnection(final HttpURLConnection connection) { + this.connection = connection; + } + public String getMethod() { return method; } @@ -454,7 +459,7 @@ public void setRequestProperty(String key, String value) { * * @throws IOException if an error occurs. */ - private HttpURLConnection openConnection() throws IOException { + HttpURLConnection openConnection() throws IOException { if (!isTraceEnabled) { return (HttpURLConnection) url.openConnection(); } 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 8dce474d9926e..3673e5110214c 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 @@ -237,6 +237,11 @@ private void completeExecute(TracingContext tracingContext) LOG.trace("{} REST operation complete", operationType); } + AbfsHttpOperation getHttpOperation(final URL url, final String method, + final List requestHeaders) throws IOException { + return new AbfsHttpOperation(url, method, requestHeaders); + } + /** * Executes a single HTTP operation to complete the REST operation. If it * fails, there may be a retry. The retryCount is incremented with each @@ -248,7 +253,7 @@ private boolean executeHttpOperation(final int retryCount, try { // initialize the HTTP request and open the connection - httpOperation = new AbfsHttpOperation(url, method, requestHeaders); + httpOperation = getHttpOperation(url, method, requestHeaders); incrementCounter(AbfsStatistic.CONNECTIONS_MADE, 1); tracingContext.constructHeader(httpOperation); diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md index ba8a00ccd2cc7..52a5ac99a3976 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md @@ -768,11 +768,13 @@ transfer, Flush() also attempting to persist buffered data will lead to performance issues. ### 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. ### 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. diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index d765250b27aac..5c9c06812fe59 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -400,6 +400,26 @@ public static AbfsRestOperation getRestOp(AbfsRestOperationType type, requestHeaders); } + public static AbfsRestOperation getRestOp1(AbfsRestOperationType type, + AbfsClient client, + String method, + URL url, + List requestHeaders, + byte[] buffer, + int bufferOffset, + int bufferLength, + String sasToken) { + return new AbfsRestOperation(type, + client, + method, + url, + requestHeaders, + buffer, + bufferOffset, + bufferLength, + sasToken); + } + public static AccessTokenProvider getAccessTokenProvider(AbfsClient client) { return client.getTokenProvider(); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsHttpOperation.java index 36914a4e4f365..8af314a74957d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsHttpOperation.java @@ -18,13 +18,16 @@ package org.apache.hadoop.fs.azurebfs.services; +import java.io.IOException; import java.io.UnsupportedEncodingException; +import java.net.HttpURLConnection; import java.net.MalformedURLException; import java.net.URL; import java.net.URLEncoder; import org.assertj.core.api.Assertions; import org.junit.Test; +import org.mockito.Mockito; import org.apache.hadoop.fs.azurebfs.utils.UriUtils; @@ -113,4 +116,12 @@ private void testIfMaskAndEncodeSuccessful(final String scenario, .isEqualTo(expectedMaskedEncodedUrl); } + public static void setHttpURLConnection(HttpURLConnection connection, AbfsHttpOperation abfsHttpOperation) throws + IOException { + Mockito.doReturn(connection).when(abfsHttpOperation).openConnection(); + } + + public static void setHttpOperation(AbfsHttpOperation abfsHttpOperation, AbfsRestOperation abfsRestOperation) throws IOException { + Mockito.doReturn(abfsHttpOperation).when(abfsRestOperation).getHttpOperation(Mockito.any(), Mockito.any(), Mockito.any()); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java new file mode 100644 index 0000000000000..39f36565bc80e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java @@ -0,0 +1,110 @@ +package org.apache.hadoop.fs.azurebfs.services; + +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.List; +import java.util.Random; + +import org.junit.Test; +import org.mockito.Mockito; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.TestAbfsConfigurationFieldsValidation; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; +import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; + +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME; + +public class TestAbfsRestOperation extends AbstractAbfsIntegrationTest { + + public TestAbfsRestOperation() throws Exception { + super(); + } + + private byte[] getRandomBytesArray(int length) { + final byte[] b = new byte[length]; + new Random().nextBytes(b); + return b; + } + + @Test + public void testExpectHundredContinue() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Configuration configuration = new Configuration(); + configuration.addResource(TEST_CONFIGURATION_FILE_NAME); + configuration.setBoolean(FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED, true); + AbfsClient abfsClient = fs.getAbfsStore().getClient(); + + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, + "dummy.dfs.core.windows.net"); + + AbfsConfiguration abfsConfig + = TestAbfsConfigurationFieldsValidation.updateRetryConfigs( + abfsConfiguration, + 5, 100); + + AbfsClient testClient = TestAbfsClient.createTestClientFromCurrentContext( + abfsClient, + abfsConfig); + + AppendRequestParameters appendRequestParameters = new AppendRequestParameters( + 0, 0, 5, AppendRequestParameters.Mode.APPEND_MODE, false, null, true); + byte[] buffer = getRandomBytesArray(5); + + // Mock instance of AbfsRestOperation + + final String TEST_PATH = "/testfile"; + + Path testPath = path(TEST_PATH); + fs.create(testPath); + String newString = testPath.toString().substring(testPath.toString().lastIndexOf("/"), + testPath.toString().length()); + + final List requestHeaders = TestAbfsClient.getTestRequestHeaders(testClient); +// if (appendRequestParameters.isExpectHeaderEnabled()) { +// requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); +// } + AbfsRestOperation op = Mockito.spy(new AbfsRestOperation( + AbfsRestOperationType.Append, + testClient, + HTTP_METHOD_PUT, + TestAbfsClient.getTestUrl(testClient, newString), + requestHeaders, buffer, + appendRequestParameters.getoffset(), + appendRequestParameters.getLength(), null)); + + HttpURLConnection urlConnection = Mockito.spy((HttpURLConnection) TestAbfsClient.getTestUrl(testClient, newString).openConnection()); + final int CONNECT_TIMEOUT = 30 * 1000; + final int READ_TIMEOUT = 30 * 1000; + + urlConnection.setConnectTimeout(CONNECT_TIMEOUT); + urlConnection.setReadTimeout(READ_TIMEOUT); + urlConnection.setRequestMethod(HTTP_METHOD_PUT); + + for (AbfsHttpHeader header : requestHeaders) { + urlConnection.setRequestProperty(header.getName(), header.getValue()); + } + AbfsHttpOperation abfsHttpOperation = new AbfsHttpOperation(TestAbfsClient.getTestUrl(testClient, newString), HTTP_METHOD_PUT, + requestHeaders); + abfsHttpOperation.setConnection(urlConnection); + //Mockito.doThrow(new IOException()).when(urlConnection).getOutputStream(); + Mockito.doReturn(abfsHttpOperation).when(op).getHttpOperation(Mockito.any(), Mockito.any(), Mockito.any()); + TracingContext tracingContext = new TracingContext("abcd", + "abcde", FSOperationType.APPEND, + TracingHeaderFormat.ALL_ID_FORMAT, null); + op.execute(tracingContext); + } + + private class HttpsURLConnection {} +} diff --git a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties index 9f72d03653306..daca5870dd5b2 100644 --- a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties @@ -26,6 +26,7 @@ log4j.logger.org.apache.hadoop.fs.azure.AzureFileSystemThreadPoolExecutor=DEBUG log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG log4j.logger.org.apache.hadoop.fs.azurebfs.contracts.services.TracingService=TRACE log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsClient=DEBUG +log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsIoUtils=DEBUG # after here: turn off log messages from other parts of the system # which only clutter test reports. From baf9ec7e7a0a7dad72e8723b05a7679a20194bb3 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Thu, 15 Dec 2022 04:21:36 -0800 Subject: [PATCH 33/53] Add tests for 100 continue --- .../services/TestAbfsRestOperation.java | 38 ++++++++++++------- 1 file changed, 25 insertions(+), 13 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java index 39f36565bc80e..e19f411fe4546 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java @@ -16,14 +16,22 @@ import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; import org.apache.hadoop.fs.azurebfs.TestAbfsConfigurationFieldsValidation; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PATCH; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_POSITION; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME; public class TestAbfsRestOperation extends AbstractAbfsIntegrationTest { @@ -44,10 +52,11 @@ public void testExpectHundredContinue() throws Exception { final Configuration configuration = new Configuration(); configuration.addResource(TEST_CONFIGURATION_FILE_NAME); configuration.setBoolean(FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED, true); + //configuration.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey.name()); AbfsClient abfsClient = fs.getAbfsStore().getClient(); AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, - "dummy.dfs.core.windows.net"); + configuration.get(FS_AZURE_ABFS_ACCOUNT_NAME)); AbfsConfiguration abfsConfig = TestAbfsConfigurationFieldsValidation.updateRetryConfigs( @@ -65,26 +74,31 @@ public void testExpectHundredContinue() throws Exception { // Mock instance of AbfsRestOperation final String TEST_PATH = "/testfile"; - Path testPath = path(TEST_PATH); fs.create(testPath); - String newString = testPath.toString().substring(testPath.toString().lastIndexOf("/"), - testPath.toString().length()); + String newString = testPath.toString().substring(testPath.toString().lastIndexOf("/")); final List requestHeaders = TestAbfsClient.getTestRequestHeaders(testClient); -// if (appendRequestParameters.isExpectHeaderEnabled()) { -// requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); -// } + requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); + if (appendRequestParameters.isExpectHeaderEnabled()) { + requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); + } + + final AbfsUriQueryBuilder abfsUriQueryBuilder = testClient.createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, APPEND_ACTION); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(appendRequestParameters.getPosition())); + URL url = testClient.createRequestUrl(newString, abfsUriQueryBuilder.toString()); + AbfsRestOperation op = Mockito.spy(new AbfsRestOperation( AbfsRestOperationType.Append, testClient, HTTP_METHOD_PUT, - TestAbfsClient.getTestUrl(testClient, newString), + url, requestHeaders, buffer, appendRequestParameters.getoffset(), appendRequestParameters.getLength(), null)); - HttpURLConnection urlConnection = Mockito.spy((HttpURLConnection) TestAbfsClient.getTestUrl(testClient, newString).openConnection()); + HttpURLConnection urlConnection = Mockito.spy((HttpURLConnection) url.openConnection()); final int CONNECT_TIMEOUT = 30 * 1000; final int READ_TIMEOUT = 30 * 1000; @@ -95,16 +109,14 @@ public void testExpectHundredContinue() throws Exception { for (AbfsHttpHeader header : requestHeaders) { urlConnection.setRequestProperty(header.getName(), header.getValue()); } - AbfsHttpOperation abfsHttpOperation = new AbfsHttpOperation(TestAbfsClient.getTestUrl(testClient, newString), HTTP_METHOD_PUT, + AbfsHttpOperation abfsHttpOperation = new AbfsHttpOperation(url, HTTP_METHOD_PUT, requestHeaders); abfsHttpOperation.setConnection(urlConnection); - //Mockito.doThrow(new IOException()).when(urlConnection).getOutputStream(); + Mockito.doThrow(new IOException()).when(urlConnection).getOutputStream(); Mockito.doReturn(abfsHttpOperation).when(op).getHttpOperation(Mockito.any(), Mockito.any(), Mockito.any()); TracingContext tracingContext = new TracingContext("abcd", "abcde", FSOperationType.APPEND, TracingHeaderFormat.ALL_ID_FORMAT, null); op.execute(tracingContext); } - - private class HttpsURLConnection {} } From fe8deea515d63df7953cab339778b3f969293ed4 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Thu, 15 Dec 2022 06:33:14 -0800 Subject: [PATCH 34/53] Add tests for hundred continue --- .../services/TestAbfsRestOperation.java | 102 ++++++++++++++++-- 1 file changed, 95 insertions(+), 7 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java index e19f411fe4546..a4a1faf54a23d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java @@ -1,7 +1,11 @@ package org.apache.hadoop.fs.azurebfs.services; +import java.io.ByteArrayOutputStream; +import java.io.FileOutputStream; import java.io.IOException; +import java.io.OutputStream; import java.net.HttpURLConnection; +import java.net.ProtocolException; import java.net.URL; import java.util.List; import java.util.Random; @@ -33,6 +37,8 @@ import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_POSITION; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.mockito.Mockito.mock; public class TestAbfsRestOperation extends AbstractAbfsIntegrationTest { @@ -51,8 +57,6 @@ public void testExpectHundredContinue() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); final Configuration configuration = new Configuration(); configuration.addResource(TEST_CONFIGURATION_FILE_NAME); - configuration.setBoolean(FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED, true); - //configuration.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey.name()); AbfsClient abfsClient = fs.getAbfsStore().getClient(); AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, @@ -98,9 +102,10 @@ public void testExpectHundredContinue() throws Exception { appendRequestParameters.getoffset(), appendRequestParameters.getLength(), null)); + AbfsHttpOperation abfsHttpOperation = new AbfsHttpOperation(url, HTTP_METHOD_PUT, requestHeaders); HttpURLConnection urlConnection = Mockito.spy((HttpURLConnection) url.openConnection()); - final int CONNECT_TIMEOUT = 30 * 1000; - final int READ_TIMEOUT = 30 * 1000; + final int CONNECT_TIMEOUT = 1 * 1000; + final int READ_TIMEOUT = 1 * 1000; urlConnection.setConnectTimeout(CONNECT_TIMEOUT); urlConnection.setReadTimeout(READ_TIMEOUT); @@ -109,14 +114,97 @@ public void testExpectHundredContinue() throws Exception { for (AbfsHttpHeader header : requestHeaders) { urlConnection.setRequestProperty(header.getName(), header.getValue()); } - AbfsHttpOperation abfsHttpOperation = new AbfsHttpOperation(url, HTTP_METHOD_PUT, - requestHeaders); abfsHttpOperation.setConnection(urlConnection); - Mockito.doThrow(new IOException()).when(urlConnection).getOutputStream(); + + Mockito.doThrow(new ProtocolException("Server rejected Operation")).when(urlConnection).getOutputStream(); Mockito.doReturn(abfsHttpOperation).when(op).getHttpOperation(Mockito.any(), Mockito.any(), Mockito.any()); TracingContext tracingContext = new TracingContext("abcd", "abcde", FSOperationType.APPEND, TracingHeaderFormat.ALL_ID_FORMAT, null); + + Mockito.doReturn(503).when(urlConnection).getResponseCode(); + Mockito.doReturn("Server busy").when(urlConnection).getResponseMessage(); + op.execute(tracingContext); } + + @Test + public void testExpectHundredContinueWriteException() throws Exception { + final AzureBlobFileSystem fs = getFileSystem(); + final Configuration configuration = new Configuration(); + configuration.addResource(TEST_CONFIGURATION_FILE_NAME); + AbfsClient abfsClient = fs.getAbfsStore().getClient(); + + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, + configuration.get(FS_AZURE_ABFS_ACCOUNT_NAME)); + + AbfsConfiguration abfsConfig + = TestAbfsConfigurationFieldsValidation.updateRetryConfigs( + abfsConfiguration, + 0, 100); + + AbfsClient testClient = TestAbfsClient.createTestClientFromCurrentContext( + abfsClient, + abfsConfig); + + AppendRequestParameters appendRequestParameters = new AppendRequestParameters( + 0, 0, 5, AppendRequestParameters.Mode.APPEND_MODE, false, null, true); + byte[] buffer = getRandomBytesArray(5); + + // Mock instance of AbfsRestOperation + + final String TEST_PATH = "/testfile"; + Path testPath = path(TEST_PATH); + fs.create(testPath); + String newString = testPath.toString().substring(testPath.toString().lastIndexOf("/")); + + final List requestHeaders = TestAbfsClient.getTestRequestHeaders(testClient); + requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); + if (appendRequestParameters.isExpectHeaderEnabled()) { + requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); + } + + final AbfsUriQueryBuilder abfsUriQueryBuilder = testClient.createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, APPEND_ACTION); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(appendRequestParameters.getPosition())); + URL url = testClient.createRequestUrl(newString, abfsUriQueryBuilder.toString()); + + AbfsRestOperation op = Mockito.spy(new AbfsRestOperation( + AbfsRestOperationType.Append, + testClient, + HTTP_METHOD_PUT, + url, + requestHeaders, buffer, + appendRequestParameters.getoffset(), + appendRequestParameters.getLength(), null)); + + AbfsHttpOperation abfsHttpOperation = new AbfsHttpOperation(url, HTTP_METHOD_PUT, requestHeaders); + HttpURLConnection urlConnection = Mockito.spy((HttpURLConnection) url.openConnection()); + final int CONNECT_TIMEOUT = 1 * 1000; + final int READ_TIMEOUT = 1 * 1000; + + urlConnection.setConnectTimeout(CONNECT_TIMEOUT); + urlConnection.setReadTimeout(READ_TIMEOUT); + urlConnection.setRequestMethod(HTTP_METHOD_PUT); + + for (AbfsHttpHeader header : requestHeaders) { + urlConnection.setRequestProperty(header.getName(), header.getValue()); + } + OutputStream outputStream = Mockito.spy(new OutputStream() { + @Override + public void write(final int i) throws IOException { + } + }); + Mockito.doReturn(outputStream).when(urlConnection).getOutputStream(); + abfsHttpOperation.setConnection(urlConnection); + Mockito.doThrow(new IOException()).when(outputStream).write(buffer, appendRequestParameters.getoffset(), appendRequestParameters.getLength()); + Mockito.doReturn(abfsHttpOperation).when(op).getHttpOperation(Mockito.any(), Mockito.any(), Mockito.any()); + TracingContext tracingContext = new TracingContext("abcd", + "abcde", FSOperationType.APPEND, + TracingHeaderFormat.ALL_ID_FORMAT, null); + intercept(IOException.class, + () -> op.execute(tracingContext)); + } } + + From cafd4093345b700a494e870a1aabef2f63e1cf09 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Sun, 18 Dec 2022 21:45:40 -0800 Subject: [PATCH 35/53] Parameters for test --- .../fs/azurebfs/utils/TracingContext.java | 5 + .../fs/azurebfs/services/TestAbfsClient.java | 20 -- .../services/TestAbfsRestOperation.java | 229 +++++++++--------- 3 files changed, 123 insertions(+), 131 deletions(-) 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 5a115451df159..c56ab673819c1 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 @@ -23,6 +23,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; @@ -139,6 +140,10 @@ public void setOperation(FSOperationType operation) { this.opType = operation; } + public int getRetryCount() { + return retryCount; + } + public void setRetryCount(int retryCount) { this.retryCount = retryCount; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index 5c9c06812fe59..d765250b27aac 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -400,26 +400,6 @@ public static AbfsRestOperation getRestOp(AbfsRestOperationType type, requestHeaders); } - public static AbfsRestOperation getRestOp1(AbfsRestOperationType type, - AbfsClient client, - String method, - URL url, - List requestHeaders, - byte[] buffer, - int bufferOffset, - int bufferLength, - String sasToken) { - return new AbfsRestOperation(type, - client, - method, - url, - requestHeaders, - buffer, - bufferOffset, - bufferLength, - sasToken); - } - public static AccessTokenProvider getAccessTokenProvider(AbfsClient client) { return client.getTokenProvider(); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java index a4a1faf54a23d..cff95be7da45f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java @@ -1,16 +1,18 @@ package org.apache.hadoop.fs.azurebfs.services; -import java.io.ByteArrayOutputStream; -import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; import java.net.HttpURLConnection; import java.net.ProtocolException; import java.net.URL; +import java.util.Arrays; import java.util.List; import java.util.Random; +import org.assertj.core.api.Assertions; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.mockito.Mockito; import org.apache.hadoop.conf.Configuration; @@ -29,8 +31,6 @@ import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PATCH; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE; import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_ACTION; @@ -40,8 +40,35 @@ import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.mockito.Mockito.mock; +@RunWith(Parameterized.class) public class TestAbfsRestOperation extends AbstractAbfsIntegrationTest { + public enum error {OUTPUTSTREAM, WRITE}; + + @Parameterized.Parameter + public boolean expectHeaderEnabled; + + @Parameterized.Parameter(1) + public int responseCode; + + @Parameterized.Parameter(2) + public String responseMessage; + + @Parameterized.Parameter(3) + public error errorType; + + @Parameterized.Parameters(name = "expect={0}-code={1}-error={3}") + public static Iterable params() { + return Arrays.asList(new Object[][]{ + {true, 200, "OK", error.WRITE}, + {false, 200, "OK", error.WRITE}, + {true, 503, "ServerBusy", error.OUTPUTSTREAM}, + {false, 503, "ServerBusy", error.OUTPUTSTREAM}, + {true, 404, "Resource Not Found", error.OUTPUTSTREAM}, + {true, 417, "Expectation Failed", error.OUTPUTSTREAM} + }); + } + public TestAbfsRestOperation() throws Exception { super(); } @@ -52,84 +79,7 @@ private byte[] getRandomBytesArray(int length) { return b; } - @Test - public void testExpectHundredContinue() throws Exception { - final AzureBlobFileSystem fs = getFileSystem(); - final Configuration configuration = new Configuration(); - configuration.addResource(TEST_CONFIGURATION_FILE_NAME); - AbfsClient abfsClient = fs.getAbfsStore().getClient(); - - AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, - configuration.get(FS_AZURE_ABFS_ACCOUNT_NAME)); - - AbfsConfiguration abfsConfig - = TestAbfsConfigurationFieldsValidation.updateRetryConfigs( - abfsConfiguration, - 5, 100); - - AbfsClient testClient = TestAbfsClient.createTestClientFromCurrentContext( - abfsClient, - abfsConfig); - - AppendRequestParameters appendRequestParameters = new AppendRequestParameters( - 0, 0, 5, AppendRequestParameters.Mode.APPEND_MODE, false, null, true); - byte[] buffer = getRandomBytesArray(5); - - // Mock instance of AbfsRestOperation - - final String TEST_PATH = "/testfile"; - Path testPath = path(TEST_PATH); - fs.create(testPath); - String newString = testPath.toString().substring(testPath.toString().lastIndexOf("/")); - - final List requestHeaders = TestAbfsClient.getTestRequestHeaders(testClient); - requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); - if (appendRequestParameters.isExpectHeaderEnabled()) { - requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); - } - - final AbfsUriQueryBuilder abfsUriQueryBuilder = testClient.createDefaultUriQueryBuilder(); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, APPEND_ACTION); - abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(appendRequestParameters.getPosition())); - URL url = testClient.createRequestUrl(newString, abfsUriQueryBuilder.toString()); - - AbfsRestOperation op = Mockito.spy(new AbfsRestOperation( - AbfsRestOperationType.Append, - testClient, - HTTP_METHOD_PUT, - url, - requestHeaders, buffer, - appendRequestParameters.getoffset(), - appendRequestParameters.getLength(), null)); - - AbfsHttpOperation abfsHttpOperation = new AbfsHttpOperation(url, HTTP_METHOD_PUT, requestHeaders); - HttpURLConnection urlConnection = Mockito.spy((HttpURLConnection) url.openConnection()); - final int CONNECT_TIMEOUT = 1 * 1000; - final int READ_TIMEOUT = 1 * 1000; - - urlConnection.setConnectTimeout(CONNECT_TIMEOUT); - urlConnection.setReadTimeout(READ_TIMEOUT); - urlConnection.setRequestMethod(HTTP_METHOD_PUT); - - for (AbfsHttpHeader header : requestHeaders) { - urlConnection.setRequestProperty(header.getName(), header.getValue()); - } - abfsHttpOperation.setConnection(urlConnection); - - Mockito.doThrow(new ProtocolException("Server rejected Operation")).when(urlConnection).getOutputStream(); - Mockito.doReturn(abfsHttpOperation).when(op).getHttpOperation(Mockito.any(), Mockito.any(), Mockito.any()); - TracingContext tracingContext = new TracingContext("abcd", - "abcde", FSOperationType.APPEND, - TracingHeaderFormat.ALL_ID_FORMAT, null); - - Mockito.doReturn(503).when(urlConnection).getResponseCode(); - Mockito.doReturn("Server busy").when(urlConnection).getResponseMessage(); - - op.execute(tracingContext); - } - - @Test - public void testExpectHundredContinueWriteException() throws Exception { + private AbfsRestOperation getRestOperation() throws Exception{ final AzureBlobFileSystem fs = getFileSystem(); final Configuration configuration = new Configuration(); configuration.addResource(TEST_CONFIGURATION_FILE_NAME); @@ -141,34 +91,28 @@ public void testExpectHundredContinueWriteException() throws Exception { AbfsConfiguration abfsConfig = TestAbfsConfigurationFieldsValidation.updateRetryConfigs( abfsConfiguration, - 0, 100); + 2, 100); AbfsClient testClient = TestAbfsClient.createTestClientFromCurrentContext( abfsClient, abfsConfig); - AppendRequestParameters appendRequestParameters = new AppendRequestParameters( - 0, 0, 5, AppendRequestParameters.Mode.APPEND_MODE, false, null, true); + 0, 0, 5, AppendRequestParameters.Mode.APPEND_MODE, false, null, expectHeaderEnabled); byte[] buffer = getRandomBytesArray(5); - // Mock instance of AbfsRestOperation - final String TEST_PATH = "/testfile"; Path testPath = path(TEST_PATH); fs.create(testPath); String newString = testPath.toString().substring(testPath.toString().lastIndexOf("/")); - final List requestHeaders = TestAbfsClient.getTestRequestHeaders(testClient); requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); if (appendRequestParameters.isExpectHeaderEnabled()) { requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); } - final AbfsUriQueryBuilder abfsUriQueryBuilder = testClient.createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, APPEND_ACTION); abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(appendRequestParameters.getPosition())); URL url = testClient.createRequestUrl(newString, abfsUriQueryBuilder.toString()); - AbfsRestOperation op = Mockito.spy(new AbfsRestOperation( AbfsRestOperationType.Append, testClient, @@ -179,31 +123,94 @@ public void testExpectHundredContinueWriteException() throws Exception { appendRequestParameters.getLength(), null)); AbfsHttpOperation abfsHttpOperation = new AbfsHttpOperation(url, HTTP_METHOD_PUT, requestHeaders); - HttpURLConnection urlConnection = Mockito.spy((HttpURLConnection) url.openConnection()); - final int CONNECT_TIMEOUT = 1 * 1000; - final int READ_TIMEOUT = 1 * 1000; - - urlConnection.setConnectTimeout(CONNECT_TIMEOUT); - urlConnection.setReadTimeout(READ_TIMEOUT); - urlConnection.setRequestMethod(HTTP_METHOD_PUT); + HttpURLConnection urlConnection = mock(HttpURLConnection.class); - for (AbfsHttpHeader header : requestHeaders) { - urlConnection.setRequestProperty(header.getName(), header.getValue()); + if (expectHeaderEnabled) { + Mockito.doReturn(HUNDRED_CONTINUE) + .when(urlConnection) + .getRequestProperty(EXPECT); + } + Mockito.doNothing().when(urlConnection).setRequestProperty(Mockito + .any(), Mockito.any()); + Mockito.doReturn(url).when(urlConnection).getURL(); + Mockito.doReturn(HTTP_METHOD_PUT).when(urlConnection).getRequestMethod(); + + switch (errorType) { + case OUTPUTSTREAM: + Mockito.doReturn(responseCode).when(urlConnection).getResponseCode(); + Mockito.doReturn(responseMessage) + .when(urlConnection) + .getResponseMessage(); + Mockito.doThrow(new ProtocolException("Server rejected Operation")) + .when(urlConnection) + .getOutputStream(); + break; + case WRITE: + OutputStream outputStream = Mockito.spy(new OutputStream() { + @Override + public void write(final int i) throws IOException { + } + }); + Mockito.doReturn(outputStream).when(urlConnection).getOutputStream(); + Mockito.doThrow(new IOException()) + .when(outputStream) + .write(buffer, appendRequestParameters.getoffset(), + appendRequestParameters.getLength()); + break; + default: + break; } - OutputStream outputStream = Mockito.spy(new OutputStream() { - @Override - public void write(final int i) throws IOException { - } - }); - Mockito.doReturn(outputStream).when(urlConnection).getOutputStream(); abfsHttpOperation.setConnection(urlConnection); - Mockito.doThrow(new IOException()).when(outputStream).write(buffer, appendRequestParameters.getoffset(), appendRequestParameters.getLength()); - Mockito.doReturn(abfsHttpOperation).when(op).getHttpOperation(Mockito.any(), Mockito.any(), Mockito.any()); - TracingContext tracingContext = new TracingContext("abcd", + Mockito.doReturn(abfsHttpOperation) + .when(op) + .getHttpOperation(Mockito.any(), Mockito.any(), Mockito.any()); + return op; + } + + @Test + public void testExpectHundredContinue() throws Exception { + AbfsRestOperation op = getRestOperation(); + TracingContext tracingContext = Mockito.spy(new TracingContext("abcd", "abcde", FSOperationType.APPEND, - TracingHeaderFormat.ALL_ID_FORMAT, null); - intercept(IOException.class, - () -> op.execute(tracingContext)); + TracingHeaderFormat.ALL_ID_FORMAT, null)); + switch (errorType) { + case WRITE: + intercept(IOException.class, + () -> op.execute(tracingContext)); + Assertions.assertThat(tracingContext.getRetryCount()) + .describedAs("The retry count is incorrect") + .isEqualTo(2); + break; + case OUTPUTSTREAM: + switch (responseCode) { + case 503: + intercept(IOException.class, + () -> op.execute(tracingContext)); + Assertions.assertThat(tracingContext.getRetryCount()) + .describedAs("The retry count is incorrect") + .isEqualTo(2); + break; + case 404: + intercept(AzureBlobFileSystemException.class, + () -> op.execute(tracingContext)); + Assertions.assertThat(tracingContext.getRetryCount()) + .describedAs("The retry count is incorrect") + .isEqualTo(0); + break; + case 417: + intercept(AzureBlobFileSystemException.class, + () -> op.execute(tracingContext)); + Assertions.assertThat(tracingContext.getRetryCount()) + .describedAs("The retry count is incorrect") + .isEqualTo(0); + break; + default: + break; + } + break; + default: + break; + } } } From f17c15af5c4213150a353a492f364129585d5c2c Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Mon, 19 Dec 2022 05:26:43 -0800 Subject: [PATCH 36/53] Tests for expect header --- .../fs/azurebfs/services/AbfsClient.java | 57 +++-- .../azurebfs/services/AbfsRestOperation.java | 9 +- .../fs/azurebfs/services/TestAbfsClient.java | 203 ++++++++++++++++-- .../services/TestAbfsHttpOperation.java | 9 - .../services/TestAbfsRestOperation.java | 132 ++++++++++-- 5 files changed, 347 insertions(+), 63 deletions(-) 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 579627a67a605..2178acee8e7b6 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 @@ -685,16 +685,9 @@ public AbfsRestOperation append(final String path, final byte[] buffer, 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); + final AbfsRestOperation op = getAbfsRestOperationForAppend(AbfsRestOperationType.Append, + HTTP_METHOD_PUT, url, requestHeaders, buffer, reqParams.getoffset(), + reqParams.getLength(), sasTokenForReuse); try { op.execute(tracingContext); } catch (AzureBlobFileSystemException e) { @@ -719,15 +712,9 @@ 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(), + final AbfsRestOperation successOp = getAbfsRestOperationForAppend( + AbfsRestOperationType.Append, HTTP_METHOD_PUT, url, requestHeaders, + buffer, reqParams.getoffset(), reqParams.getLength(), sasTokenForReuse); successOp.hardSetResult(HttpURLConnection.HTTP_OK); return successOp; @@ -738,6 +725,38 @@ && appendSuccessCheckOp(op, path, return op; } + /** + * 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 new AbfsRestOperation( + operationType, + this, + 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. 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 3673e5110214c..19610f8082745 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 @@ -325,7 +325,14 @@ private boolean executeHttpOperation(final int retryCount, return false; } finally { - intercept.updateMetrics(operationType, httpOperation); + int status = httpOperation.getStatusCode(); + // If httpStatusCode lies in the range of 400 to 500, it signifies user error. + boolean userError = (status >= HttpURLConnection.HTTP_BAD_REQUEST + && status < HttpURLConnection.HTTP_INTERNAL_ERROR); + // We should not update the metrics for user error cases. + if (!userError) { + intercept.updateMetrics(operationType, httpOperation); + } } LOG.debug("HttpRequest: {}: {}", operationType, httpOperation); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index d765250b27aac..2bbc61df16ae0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -20,20 +20,43 @@ import java.io.IOException; import java.lang.reflect.Field; +import java.net.HttpURLConnection; +import java.net.ProtocolException; import java.net.URL; import java.util.List; +import java.util.Random; import java.util.regex.Pattern; +import org.assertj.core.api.Assertions; import org.junit.Test; +import org.mockito.Mockito; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; +import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.TestAbfsConfigurationFieldsValidation; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; -import static org.assertj.core.api.Assertions.assertThat; +import static java.net.HttpURLConnection.HTTP_NOT_FOUND; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PATCH; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_HTTP_METHOD_OVERRIDE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_ACTION; +import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.QUERY_PARAM_POSITION; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -59,14 +82,18 @@ * Test useragent of abfs client. * */ -public final class TestAbfsClient { +public final class TestAbfsClient extends AbstractAbfsIntegrationTest { private static final String ACCOUNT_NAME = "bogusAccountName.dfs.core.windows.net"; private static final String FS_AZURE_USER_AGENT_PREFIX = "Partner Service"; + public static final int REDUCED_RETRY_COUNT = 2; + public static final int REDUCED_BACKOFF_INTERVAL = 100; + public static final int BUFFER_LENGTH = 5; + public static final int BUFFER_OFFSET = 0; private final Pattern userAgentStringPattern; - public TestAbfsClient(){ + public TestAbfsClient() throws Exception { StringBuilder regEx = new StringBuilder(); regEx.append("^"); regEx.append(APN_VERSION); @@ -124,7 +151,7 @@ public void verifybBasicInfo() throws Exception { } private void verifybBasicInfo(String userAgentStr) { - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string [" + userAgentStr + "] should be of the pattern: " + this.userAgentStringPattern.pattern()) .matches(this.userAgentStringPattern) @@ -153,7 +180,7 @@ public void verifyUserAgentPrefix() String userAgentStr = getUserAgentString(abfsConfiguration, false); verifybBasicInfo(userAgentStr); - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string should contain " + FS_AZURE_USER_AGENT_PREFIX) .contains(FS_AZURE_USER_AGENT_PREFIX); @@ -163,7 +190,7 @@ public void verifyUserAgentPrefix() userAgentStr = getUserAgentString(abfsConfiguration, false); verifybBasicInfo(userAgentStr); - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string should not contain " + FS_AZURE_USER_AGENT_PREFIX) .doesNotContain(FS_AZURE_USER_AGENT_PREFIX); } @@ -179,14 +206,14 @@ public void verifyUserAgentWithoutSSLProvider() throws Exception { String userAgentStr = getUserAgentString(abfsConfiguration, true); verifybBasicInfo(userAgentStr); - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string should contain sslProvider") .contains(DelegatingSSLSocketFactory.getDefaultFactory().getProviderName()); userAgentStr = getUserAgentString(abfsConfiguration, false); verifybBasicInfo(userAgentStr); - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string should not contain sslProvider") .doesNotContain(DelegatingSSLSocketFactory.getDefaultFactory().getProviderName()); } @@ -202,7 +229,7 @@ public void verifyUserAgentClusterName() throws Exception { String userAgentStr = getUserAgentString(abfsConfiguration, false); verifybBasicInfo(userAgentStr); - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string should contain cluster name") .contains(clusterName); @@ -212,7 +239,7 @@ public void verifyUserAgentClusterName() throws Exception { userAgentStr = getUserAgentString(abfsConfiguration, false); verifybBasicInfo(userAgentStr); - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string should not contain cluster name") .doesNotContain(clusterName) .describedAs("User-Agent string should contain UNKNOWN as cluster name config is absent") @@ -230,7 +257,7 @@ public void verifyUserAgentClusterType() throws Exception { String userAgentStr = getUserAgentString(abfsConfiguration, false); verifybBasicInfo(userAgentStr); - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string should contain cluster type") .contains(clusterType); @@ -240,7 +267,7 @@ public void verifyUserAgentClusterType() throws Exception { userAgentStr = getUserAgentString(abfsConfiguration, false); verifybBasicInfo(userAgentStr); - assertThat(userAgentStr) + Assertions.assertThat(userAgentStr) .describedAs("User-Agent string should not contain cluster type") .doesNotContain(clusterType) .describedAs("User-Agent string should contain UNKNOWN as cluster type config is absent") @@ -403,4 +430,156 @@ public static AbfsRestOperation getRestOp(AbfsRestOperationType type, public static AccessTokenProvider getAccessTokenProvider(AbfsClient client) { return client.getTokenProvider(); } + + /** + * Test helper method to get random bytes array. + * @param length The length of byte buffer. + * @return byte buffer. + */ + private byte[] getRandomBytesArray(int length) { + final byte[] b = new byte[length]; + new Random().nextBytes(b); + return b; + } + + /** + * Test to verify that client retries append request without + * expect header enabled if append with expect header enabled fails + * with 4xx kind of error. + * @throws Exception + */ + @Test + public void testExpectHundredContinue() throws Exception { + // Get the filesystem. + final AzureBlobFileSystem fs = getFileSystem(); + + final Configuration configuration = new Configuration(); + configuration.addResource(TEST_CONFIGURATION_FILE_NAME); + AbfsClient abfsClient = fs.getAbfsStore().getClient(); + + AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, + configuration.get(FS_AZURE_ABFS_ACCOUNT_NAME)); + + // Update the configuration with reduced retry count and reduced backoff interval. + AbfsConfiguration abfsConfig + = TestAbfsConfigurationFieldsValidation.updateRetryConfigs( + abfsConfiguration, + REDUCED_RETRY_COUNT, REDUCED_BACKOFF_INTERVAL); + + // Gets the client. + AbfsClient testClient = Mockito.spy( + TestAbfsClient.createTestClientFromCurrentContext( + abfsClient, + abfsConfig)); + + // Create the append request params with expect header enabled initially. + AppendRequestParameters appendRequestParameters + = new AppendRequestParameters( + BUFFER_OFFSET, BUFFER_OFFSET, BUFFER_LENGTH, + AppendRequestParameters.Mode.APPEND_MODE, false, null, true); + + byte[] buffer = getRandomBytesArray(BUFFER_LENGTH); + + // Create a test container to upload the data. + final String TEST_PATH = "/testfile"; + Path testPath = path(TEST_PATH); + fs.create(testPath); + String finalTestPath = testPath.toString() + .substring(testPath.toString().lastIndexOf("/")); + + // Creates a list of request headers. + final List requestHeaders + = TestAbfsClient.getTestRequestHeaders(testClient); + requestHeaders.add( + new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); + if (appendRequestParameters.isExpectHeaderEnabled()) { + requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); + } + + // Updates the query parameters. + final AbfsUriQueryBuilder abfsUriQueryBuilder + = testClient.createDefaultUriQueryBuilder(); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, APPEND_ACTION); + abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, + Long.toString(appendRequestParameters.getPosition())); + + // Creates the url for the specified path. + URL url = testClient.createRequestUrl(finalTestPath, abfsUriQueryBuilder.toString()); + + // Create a mock of the AbfsRestOperation to set the urlConnection in the corresponding httpOperation. + AbfsRestOperation op = Mockito.spy(new AbfsRestOperation( + AbfsRestOperationType.Append, + testClient, + HTTP_METHOD_PUT, + url, + requestHeaders, buffer, + appendRequestParameters.getoffset(), + appendRequestParameters.getLength(), null)); + + AbfsHttpOperation abfsHttpOperation = new AbfsHttpOperation(url, + HTTP_METHOD_PUT, requestHeaders); + + // Create a mock of UrlConnection class. + HttpURLConnection urlConnection = mock(HttpURLConnection.class); + + // Sets the expect request property if expect header is enabled. + if (appendRequestParameters.isExpectHeaderEnabled()) { + Mockito.doReturn(HUNDRED_CONTINUE).when(urlConnection) + .getRequestProperty(EXPECT); + } + Mockito.doNothing().when(urlConnection).setRequestProperty(Mockito + .any(), Mockito.any()); + Mockito.doReturn(url).when(urlConnection).getURL(); + + // Give user error code 404 when processResponse is called. + Mockito.doReturn(HTTP_METHOD_PUT).when(urlConnection).getRequestMethod(); + Mockito.doReturn(HTTP_NOT_FOUND).when(urlConnection).getResponseCode(); + Mockito.doReturn("Resource Not Found") + .when(urlConnection) + .getResponseMessage(); + + // Make the getOuputStream throw IOException to see it returns from the sendRequest correctly. + Mockito.doThrow(new ProtocolException("Server rejected Operation")) + .when(urlConnection) + .getOutputStream(); + abfsHttpOperation.setConnection(urlConnection); + + // Sets the httpOperation for the rest operation. + Mockito.doReturn(abfsHttpOperation) + .when(op) + .getHttpOperation(Mockito.any(), Mockito.any(), Mockito.any()); + + // Mock the restOperation for the client. + Mockito.doReturn(op).when(testClient).getAbfsRestOperationForAppend( + Mockito.nullable(AbfsRestOperationType.class), + Mockito.nullable(String.class), + Mockito.nullable(URL.class), + Mockito.nullable(List.class), + Mockito.nullable(byte[].class), + Mockito.nullable(int.class), + Mockito.nullable(int.class), + Mockito.nullable(String.class)); + + TracingContext tracingContext = Mockito.spy(new TracingContext("abcd", + "abcde", FSOperationType.APPEND, + TracingHeaderFormat.ALL_ID_FORMAT, null)); + + // Check that expect header is enabled before the append call. + Assertions.assertThat(appendRequestParameters.isExpectHeaderEnabled()) + .describedAs("The expect header is not true before the append call") + .isEqualTo(true); + + intercept(AzureBlobFileSystemException.class, + () -> testClient.append(finalTestPath, buffer, appendRequestParameters, null, tracingContext)); + + // Verify that the request was not exponentially retried because of user error. + Assertions.assertThat(tracingContext.getRetryCount()) + .describedAs("The retry count is incorrect") + .isEqualTo(0); + + // Verify that the same request was retried with expect header disabled. + Assertions.assertThat(appendRequestParameters.isExpectHeaderEnabled()) + .describedAs("The expect header is not false") + .isEqualTo(false); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsHttpOperation.java index 8af314a74957d..cc9661092fe01 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsHttpOperation.java @@ -115,13 +115,4 @@ private void testIfMaskAndEncodeSuccessful(final String scenario, + "be: " + expectedMaskedEncodedUrl) .isEqualTo(expectedMaskedEncodedUrl); } - - public static void setHttpURLConnection(HttpURLConnection connection, AbfsHttpOperation abfsHttpOperation) throws - IOException { - Mockito.doReturn(connection).when(abfsHttpOperation).openConnection(); - } - - public static void setHttpOperation(AbfsHttpOperation abfsHttpOperation, AbfsRestOperation abfsRestOperation) throws IOException { - Mockito.doReturn(abfsHttpOperation).when(abfsRestOperation).getHttpOperation(Mockito.any(), Mockito.any(), Mockito.any()); - } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java index cff95be7da45f..5c4cc2898caf7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.fs.azurebfs.services; import java.io.IOException; @@ -27,6 +45,9 @@ import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; +import static java.net.HttpURLConnection.HTTP_NOT_FOUND; +import static java.net.HttpURLConnection.HTTP_OK; +import static java.net.HttpURLConnection.HTTP_UNAVAILABLE; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_ACTION; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PATCH; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; @@ -43,29 +64,45 @@ @RunWith(Parameterized.class) public class TestAbfsRestOperation extends AbstractAbfsIntegrationTest { + // Specifies whether getOutputStream() or write() throws IOException. public enum error {OUTPUTSTREAM, WRITE}; + public static final int HTTP_EXPECTATION_FAILED = 417; + public static final int REDUCED_RETRY_COUNT = 2; + public static final int REDUCED_BACKOFF_INTERVAL = 100; + public static final int BUFFER_LENGTH = 5; + public static final int BUFFER_OFFSET = 0; + + // Specifies whether the expect header is enabled or not. @Parameterized.Parameter public boolean expectHeaderEnabled; + // Gives the http response code. @Parameterized.Parameter(1) public int responseCode; + // Gives the http response message. @Parameterized.Parameter(2) public String responseMessage; + // Gives the errorType based on the enum. @Parameterized.Parameter(3) public error errorType; + /* + HTTP_OK = 200, + HTTP_UNAVAILABLE = 503, + HTTP_NOT_FOUND = 404, + HTTP_EXPECTATION_FAILED = 417. + */ @Parameterized.Parameters(name = "expect={0}-code={1}-error={3}") public static Iterable params() { return Arrays.asList(new Object[][]{ - {true, 200, "OK", error.WRITE}, - {false, 200, "OK", error.WRITE}, - {true, 503, "ServerBusy", error.OUTPUTSTREAM}, - {false, 503, "ServerBusy", error.OUTPUTSTREAM}, - {true, 404, "Resource Not Found", error.OUTPUTSTREAM}, - {true, 417, "Expectation Failed", error.OUTPUTSTREAM} + {true, HTTP_OK, "OK", error.WRITE}, + {false, HTTP_OK, "OK", error.WRITE}, + {true, HTTP_UNAVAILABLE, "ServerBusy", error.OUTPUTSTREAM}, + {true, HTTP_NOT_FOUND, "Resource Not Found", error.OUTPUTSTREAM}, + {true, HTTP_EXPECTATION_FAILED, "Expectation Failed", error.OUTPUTSTREAM} }); } @@ -73,14 +110,26 @@ public TestAbfsRestOperation() throws Exception { super(); } + /** + * Test helper method to get random bytes array. + * @param length The length of byte buffer + * @return byte buffer + */ private byte[] getRandomBytesArray(int length) { final byte[] b = new byte[length]; new Random().nextBytes(b); return b; } - private AbfsRestOperation getRestOperation() throws Exception{ + /** + * Gives the AbfsRestOperation. + * @return abfsRestOperation. + * @throws Exception + */ + private AbfsRestOperation getRestOperation() throws Exception { + // Get the filesystem. final AzureBlobFileSystem fs = getFileSystem(); + final Configuration configuration = new Configuration(); configuration.addResource(TEST_CONFIGURATION_FILE_NAME); AbfsClient abfsClient = fs.getAbfsStore().getClient(); @@ -88,31 +137,48 @@ private AbfsRestOperation getRestOperation() throws Exception{ AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, configuration.get(FS_AZURE_ABFS_ACCOUNT_NAME)); + // Update the configuration with reduced retry count and reduced backoff interval. AbfsConfiguration abfsConfig = TestAbfsConfigurationFieldsValidation.updateRetryConfigs( abfsConfiguration, - 2, 100); + REDUCED_RETRY_COUNT, REDUCED_BACKOFF_INTERVAL); + // Gets the client. AbfsClient testClient = TestAbfsClient.createTestClientFromCurrentContext( abfsClient, abfsConfig); - AppendRequestParameters appendRequestParameters = new AppendRequestParameters( - 0, 0, 5, AppendRequestParameters.Mode.APPEND_MODE, false, null, expectHeaderEnabled); + + // Expect header is enabled or not based on the parameter. + AppendRequestParameters appendRequestParameters + = new AppendRequestParameters( + BUFFER_OFFSET, BUFFER_OFFSET, BUFFER_LENGTH, + AppendRequestParameters.Mode.APPEND_MODE, false, null, + expectHeaderEnabled); + byte[] buffer = getRandomBytesArray(5); + // Create a test container to upload the data. final String TEST_PATH = "/testfile"; Path testPath = path(TEST_PATH); fs.create(testPath); - String newString = testPath.toString().substring(testPath.toString().lastIndexOf("/")); + String finalTestPath = testPath.toString().substring(testPath.toString().lastIndexOf("/")); + + // Creates a list of request headers. final List requestHeaders = TestAbfsClient.getTestRequestHeaders(testClient); requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); if (appendRequestParameters.isExpectHeaderEnabled()) { requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); } + + // Updates the query parameters. final AbfsUriQueryBuilder abfsUriQueryBuilder = testClient.createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, APPEND_ACTION); abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(appendRequestParameters.getPosition())); - URL url = testClient.createRequestUrl(newString, abfsUriQueryBuilder.toString()); + + // Creates the url for the specified path. + URL url = testClient.createRequestUrl(finalTestPath, abfsUriQueryBuilder.toString()); + + // Create a mock of the AbfsRestOperation to set the urlConnection in the corresponding httpOperation. AbfsRestOperation op = Mockito.spy(new AbfsRestOperation( AbfsRestOperationType.Append, testClient, @@ -123,8 +189,11 @@ private AbfsRestOperation getRestOperation() throws Exception{ appendRequestParameters.getLength(), null)); AbfsHttpOperation abfsHttpOperation = new AbfsHttpOperation(url, HTTP_METHOD_PUT, requestHeaders); + + // Create a mock of UrlConnection class. HttpURLConnection urlConnection = mock(HttpURLConnection.class); + // Sets the expect request property if expect header is enabled. if (expectHeaderEnabled) { Mockito.doReturn(HUNDRED_CONTINUE) .when(urlConnection) @@ -137,6 +206,10 @@ private AbfsRestOperation getRestOperation() throws Exception{ switch (errorType) { case OUTPUTSTREAM: + // If the getOutputStream() throws IOException and Expect Header is + // enabled, it returns back to processResponse and hence we have + // mocked the response code and the response message to check different + // behaviour based on response code. Mockito.doReturn(responseCode).when(urlConnection).getResponseCode(); Mockito.doReturn(responseMessage) .when(urlConnection) @@ -146,6 +219,8 @@ private AbfsRestOperation getRestOperation() throws Exception{ .getOutputStream(); break; case WRITE: + // If write() throws IOException and Expect Header is + // enabled or not, it should throw back the exception. OutputStream outputStream = Mockito.spy(new OutputStream() { @Override public void write(final int i) throws IOException { @@ -160,46 +235,59 @@ public void write(final int i) throws IOException { default: break; } + + // Sets the urlConnection for the httpOperation. abfsHttpOperation.setConnection(urlConnection); + + // Sets the httpOperation for the rest operation. Mockito.doReturn(abfsHttpOperation) .when(op) .getHttpOperation(Mockito.any(), Mockito.any(), Mockito.any()); return op; } + /** + * Test the functionalities based on whether getOutputStream() or write() + * throws exception and what is the corresponding response code. + * @throws Exception + */ @Test public void testExpectHundredContinue() throws Exception { + // Gets the AbfsRestOperation. AbfsRestOperation op = getRestOperation(); TracingContext tracingContext = Mockito.spy(new TracingContext("abcd", "abcde", FSOperationType.APPEND, TracingHeaderFormat.ALL_ID_FORMAT, null)); switch (errorType) { case WRITE: + // If write() throws IOException and Expect Header is + // enabled or not, it should throw back the exception + // which is caught and exponential retry logic comes into place. intercept(IOException.class, () -> op.execute(tracingContext)); + + // Assert that the request is retried based on reduced retry count configured. Assertions.assertThat(tracingContext.getRetryCount()) .describedAs("The retry count is incorrect") .isEqualTo(2); break; case OUTPUTSTREAM: switch (responseCode) { - case 503: - intercept(IOException.class, - () -> op.execute(tracingContext)); - Assertions.assertThat(tracingContext.getRetryCount()) - .describedAs("The retry count is incorrect") - .isEqualTo(2); - break; - case 404: - intercept(AzureBlobFileSystemException.class, + case HTTP_UNAVAILABLE: + // In the case of 503 i.e. throttled case, we should retry. + intercept(IOException.class, () -> op.execute(tracingContext)); + // Assert that the request is retried based on reduced retry count configured. Assertions.assertThat(tracingContext.getRetryCount()) .describedAs("The retry count is incorrect") - .isEqualTo(0); + .isEqualTo(2); break; + case 404: case 417: + // In the case of 4xx error. i.e. user error, retry should not happen. intercept(AzureBlobFileSystemException.class, () -> op.execute(tracingContext)); + // Assert that the request is not retried. Assertions.assertThat(tracingContext.getRetryCount()) .describedAs("The retry count is incorrect") .isEqualTo(0); From 61138e9aa4f504309a8cf12ae0d72060d293e340 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Mon, 19 Dec 2022 22:19:56 -0800 Subject: [PATCH 37/53] Update metrics fix --- .../AbfsClientThrottlingIntercept.java | 21 ++++++++++++++ .../azurebfs/services/AbfsHttpOperation.java | 28 ++++++++++++------- .../services/TestAbfsRestOperation.java | 22 ++++++++++----- 3 files changed, 54 insertions(+), 17 deletions(-) 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 52a46bc746934..6f4a5b018ae9f 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 @@ -28,6 +28,8 @@ 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 @@ -114,6 +116,18 @@ static AbfsClientThrottlingIntercept initializeSingleton(AbfsConfiguration abfsC return singleton; } + /** + * Updates the metrics for the case when getOutputStream() caught an IOException + * and response code signifies throttling. + * @param isThrottledOperation returns true if status code is HTTP_UNAVAILABLE + * @param abfsHttpOperation Used for status code and data transferred. + * @return + */ + private boolean updateBytesTransferred(boolean isThrottledOperation, + AbfsHttpOperation abfsHttpOperation) { + return isThrottledOperation && abfsHttpOperation.getExpectedBytesSent() > 0; + } + /** * Updates the metrics for successful and failed read and write operations. * @param operationType Only applicable for read and write operations. @@ -134,9 +148,16 @@ public void updateMetrics(AbfsRestOperationType operationType, boolean isFailedOperation = (status < HttpURLConnection.HTTP_OK || status >= HttpURLConnection.HTTP_INTERNAL_ERROR); + boolean isThrottledOperation = (status == HTTP_UNAVAILABLE); + switch (operationType) { case Append: contentLength = abfsHttpOperation.getBytesSent(); + if (contentLength == 0) { + if (updateBytesTransferred(isThrottledOperation, abfsHttpOperation)) { + contentLength = abfsHttpOperation.getExpectedBytesSent(); + } + } if (contentLength > 0) { writeThrottler.addBytesTransferred(contentLength, isFailedOperation); 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 b134de911b2f0..75ea8eb0805a7 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 @@ -28,7 +28,6 @@ import javax.net.ssl.HttpsURLConnection; import javax.net.ssl.SSLSocketFactory; -import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.fs.azurebfs.utils.UriUtils; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; @@ -77,6 +76,7 @@ public class AbfsHttpOperation implements AbfsPerfLoggable { // metrics private int bytesSent; + private int expectedBytesSent; private long bytesReceived; // optional trace enabled metrics @@ -163,6 +163,10 @@ public int getBytesSent() { return bytesSent; } + public int getExpectedBytesSent() { + return expectedBytesSent; + } + public long getBytesReceived() { return bytesReceived; } @@ -327,10 +331,11 @@ public void sendRequest(byte[] buffer, int offset, int length) throws IOExceptio try { outputStream = this.connection.getOutputStream(); } catch (IOException e) { - // If getOutputStream fails with an exception and 100-continue - // is enabled, we return back without throwing an exception - // because processResponse will give the correct status code - // based on which the retry logic will come into place. + // If getOutputStream fails with an exception and expect header + // is enabled, we return back without throwing an exception to + // the caller. The caller is responsible for setting the correct status code. + // If expect header is not enabled, we throw back the exception. + this.expectedBytesSent = length; String expectHeader = this.connection.getRequestProperty(EXPECT); if (expectHeader != null && expectHeader.equals(HUNDRED_CONTINUE)) { return; @@ -338,16 +343,19 @@ public void sendRequest(byte[] buffer, int offset, int length) throws IOExceptio throw e; } } - // This will normally throw an IOException. + // update bytes sent for successful as well as failed attempts via the + // accompanying statusCode. + this.bytesSent = length; + + // If this fails with or without expect header enabled, + // it throws an IOException. outputStream.write(buffer, offset, length); } finally { // Closing the opened output stream if (outputStream != null) { outputStream.close(); } - // update bytes sent for successful as well as failed attempts via the - // accompanying statusCode. - this.bytesSent = length; + if (this.isTraceEnabled) { this.sendRequestTimeMs = elapsedTimeMs(startTime); } @@ -459,7 +467,7 @@ public void setRequestProperty(String key, String value) { * * @throws IOException if an error occurs. */ - HttpURLConnection openConnection() throws IOException { + private HttpURLConnection openConnection() throws IOException { if (!isTraceEnabled) { return (HttpURLConnection) url.openConnection(); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java index 5c4cc2898caf7..9ca46d3e23540 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java @@ -35,6 +35,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.Abfs; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; @@ -255,6 +256,7 @@ public void write(final int i) throws IOException { public void testExpectHundredContinue() throws Exception { // Gets the AbfsRestOperation. AbfsRestOperation op = getRestOperation(); + AbfsHttpOperation httpOperation = op.getHttpOperation(Mockito.any(), Mockito.any(), Mockito.any()); TracingContext tracingContext = Mockito.spy(new TracingContext("abcd", "abcde", FSOperationType.APPEND, TracingHeaderFormat.ALL_ID_FORMAT, null)); @@ -265,11 +267,14 @@ public void testExpectHundredContinue() throws Exception { // which is caught and exponential retry logic comes into place. intercept(IOException.class, () -> op.execute(tracingContext)); - // Assert that the request is retried based on reduced retry count configured. Assertions.assertThat(tracingContext.getRetryCount()) .describedAs("The retry count is incorrect") - .isEqualTo(2); + .isEqualTo(REDUCED_RETRY_COUNT); + Assertions.assertThat(httpOperation.getBytesSent()) + .isEqualTo(5); + Assertions.assertThat(httpOperation.getExpectedBytesSent()) + .isEqualTo(0); break; case OUTPUTSTREAM: switch (responseCode) { @@ -280,10 +285,15 @@ public void testExpectHundredContinue() throws Exception { // Assert that the request is retried based on reduced retry count configured. Assertions.assertThat(tracingContext.getRetryCount()) .describedAs("The retry count is incorrect") - .isEqualTo(2); + .isEqualTo(REDUCED_RETRY_COUNT); + // Assert that metrics will be updated correctly + Assertions.assertThat(httpOperation.getBytesSent()) + .isEqualTo(0); + Assertions.assertThat(httpOperation.getExpectedBytesSent()) + .isEqualTo(5); break; - case 404: - case 417: + case HTTP_NOT_FOUND: + case HTTP_EXPECTATION_FAILED: // In the case of 4xx error. i.e. user error, retry should not happen. intercept(AzureBlobFileSystemException.class, () -> op.execute(tracingContext)); @@ -301,5 +311,3 @@ public void testExpectHundredContinue() throws Exception { } } } - - From ac3e973bb1f7bb89192abef486d766c5effe8533 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Tue, 20 Dec 2022 01:54:14 -0800 Subject: [PATCH 38/53] Metric update changes --- .../AbfsClientThrottlingIntercept.java | 10 +++++-- .../azurebfs/services/AbfsHttpOperation.java | 30 +++++++++++++------ .../azurebfs/services/AbfsRestOperation.java | 11 +++---- .../services/TestAbfsRestOperation.java | 20 ++++++++----- 4 files changed, 47 insertions(+), 24 deletions(-) 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 6f4a5b018ae9f..a0907bc9c7d9f 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 @@ -125,7 +125,7 @@ static AbfsClientThrottlingIntercept initializeSingleton(AbfsConfiguration abfsC */ private boolean updateBytesTransferred(boolean isThrottledOperation, AbfsHttpOperation abfsHttpOperation) { - return isThrottledOperation && abfsHttpOperation.getExpectedBytesSent() > 0; + return isThrottledOperation && abfsHttpOperation.getExpectedBytesToBeSent() > 0; } /** @@ -148,14 +148,20 @@ public void updateMetrics(AbfsRestOperationType operationType, boolean isFailedOperation = (status < HttpURLConnection.HTTP_OK || status >= HttpURLConnection.HTTP_INTERNAL_ERROR); + // If status code is 503, it considered a throttled operation. boolean isThrottledOperation = (status == HTTP_UNAVAILABLE); switch (operationType) { case Append: 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)) { - contentLength = abfsHttpOperation.getExpectedBytesSent(); + LOG.debug("Updating metrics due to throttling"); + contentLength = abfsHttpOperation.getExpectedBytesToBeSent(); } } if (contentLength > 0) { 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 75ea8eb0805a7..71f8c475966b4 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,11 +23,13 @@ import java.io.OutputStream; import java.net.HttpURLConnection; import java.net.URL; +import java.util.Arrays; import java.util.List; import javax.net.ssl.HttpsURLConnection; import javax.net.ssl.SSLSocketFactory; +import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.hadoop.fs.azurebfs.utils.UriUtils; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; @@ -76,7 +78,7 @@ public class AbfsHttpOperation implements AbfsPerfLoggable { // metrics private int bytesSent; - private int expectedBytesSent; + private int expectedBytesToBeSent; private long bytesReceived; // optional trace enabled metrics @@ -163,8 +165,8 @@ public int getBytesSent() { return bytesSent; } - public int getExpectedBytesSent() { - return expectedBytesSent; + public int getExpectedBytesToBeSent() { + return expectedBytesToBeSent; } public long getBytesReceived() { @@ -327,19 +329,30 @@ public void sendRequest(byte[] buffer, int offset, int length) throws IOExceptio startTime = System.nanoTime(); } OutputStream outputStream = null; + // Updates the expected bytes to be sent based on length. + this.expectedBytesToBeSent = length; try { try { + /* Without expect header enabled, if getOutputStream() throws + an exception, it gets caught by the restOperation. But with + expect header enabled we return back without throwing an exception + for the correct response code processing. + */ outputStream = this.connection.getOutputStream(); } catch (IOException e) { - // If getOutputStream fails with an exception and expect header - // is enabled, we return back without throwing an exception to - // the caller. The caller is responsible for setting the correct status code. - // If expect header is not enabled, we throw back the exception. - this.expectedBytesSent = length; + /* If getOutputStream fails with an exception and expect header + is enabled, we return back without throwing an exception to + the caller. The caller is responsible for setting the correct status code. + If expect header is not enabled, we throw back the exception. + */ String expectHeader = this.connection.getRequestProperty(EXPECT); if (expectHeader != null && expectHeader.equals(HUNDRED_CONTINUE)) { + LOG.debug("Getting output stream failed with expect header enabled, returning back " + + ExceptionUtils.getStackTrace(e)); return; } else { + LOG.debug("Getting output stream failed without expect header enabled, throwing exception " + + ExceptionUtils.getStackTrace(e)); throw e; } } @@ -355,7 +368,6 @@ public void sendRequest(byte[] buffer, int offset, int length) throws IOExceptio if (outputStream != null) { outputStream.close(); } - if (this.isTraceEnabled) { this.sendRequestTimeMs = elapsedTimeMs(startTime); } 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 19610f8082745..dc284d60a3aff 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 @@ -326,11 +326,12 @@ private boolean executeHttpOperation(final int retryCount, return false; } finally { int status = httpOperation.getStatusCode(); - // If httpStatusCode lies in the range of 400 to 500, it signifies user error. - boolean userError = (status >= HttpURLConnection.HTTP_BAD_REQUEST - && status < HttpURLConnection.HTTP_INTERNAL_ERROR); - // We should not update the metrics for user error cases. - if (!userError) { + // If the socket is terminated prior to receiving a response, the HTTP + // status may be 0 or -1. A status less than 300 (2xx range) or greater than or equal + // to 500 (5xx range) should contribute to metrics updation. + boolean updateMetricsResponseCode = (status < HttpURLConnection.HTTP_MULT_CHOICE + || status >= HttpURLConnection.HTTP_INTERNAL_ERROR); + if (updateMetricsResponseCode) { intercept.updateMetrics(operationType, httpOperation); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java index 9ca46d3e23540..56ca039caa09e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java @@ -125,7 +125,6 @@ private byte[] getRandomBytesArray(int length) { /** * Gives the AbfsRestOperation. * @return abfsRestOperation. - * @throws Exception */ private AbfsRestOperation getRestOperation() throws Exception { // Get the filesystem. @@ -250,16 +249,17 @@ public void write(final int i) throws IOException { /** * Test the functionalities based on whether getOutputStream() or write() * throws exception and what is the corresponding response code. - * @throws Exception */ @Test public void testExpectHundredContinue() throws Exception { // Gets the AbfsRestOperation. AbfsRestOperation op = getRestOperation(); AbfsHttpOperation httpOperation = op.getHttpOperation(Mockito.any(), Mockito.any(), Mockito.any()); + TracingContext tracingContext = Mockito.spy(new TracingContext("abcd", "abcde", FSOperationType.APPEND, TracingHeaderFormat.ALL_ID_FORMAT, null)); + switch (errorType) { case WRITE: // If write() throws IOException and Expect Header is @@ -267,14 +267,15 @@ public void testExpectHundredContinue() throws Exception { // which is caught and exponential retry logic comes into place. intercept(IOException.class, () -> op.execute(tracingContext)); + // Assert that the request is retried based on reduced retry count configured. Assertions.assertThat(tracingContext.getRetryCount()) .describedAs("The retry count is incorrect") .isEqualTo(REDUCED_RETRY_COUNT); + + // Assert that metrics will be updated correctly. Assertions.assertThat(httpOperation.getBytesSent()) - .isEqualTo(5); - Assertions.assertThat(httpOperation.getExpectedBytesSent()) - .isEqualTo(0); + .isEqualTo(BUFFER_LENGTH); break; case OUTPUTSTREAM: switch (responseCode) { @@ -282,21 +283,24 @@ public void testExpectHundredContinue() throws Exception { // In the case of 503 i.e. throttled case, we should retry. intercept(IOException.class, () -> op.execute(tracingContext)); + // Assert that the request is retried based on reduced retry count configured. Assertions.assertThat(tracingContext.getRetryCount()) .describedAs("The retry count is incorrect") .isEqualTo(REDUCED_RETRY_COUNT); - // Assert that metrics will be updated correctly + + // Assert that metrics will be updated correctly. Assertions.assertThat(httpOperation.getBytesSent()) .isEqualTo(0); - Assertions.assertThat(httpOperation.getExpectedBytesSent()) - .isEqualTo(5); + Assertions.assertThat(httpOperation.getExpectedBytesToBeSent()) + .isEqualTo(BUFFER_LENGTH); break; case HTTP_NOT_FOUND: case HTTP_EXPECTATION_FAILED: // In the case of 4xx error. i.e. user error, retry should not happen. intercept(AzureBlobFileSystemException.class, () -> op.execute(tracingContext)); + // Assert that the request is not retried. Assertions.assertThat(tracingContext.getRetryCount()) .describedAs("The retry count is incorrect") From 36ec26094831fd3d32876e71781449c2485f333b Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Tue, 20 Dec 2022 05:27:47 -0800 Subject: [PATCH 39/53] Tests for metric updation verification --- .../AbfsClientThrottlingIntercept.java | 2 +- .../azurebfs/services/AbfsRestOperation.java | 3 +- .../services/ExponentialRetryPolicy.java | 8 ++- .../services/TestAbfsRestOperation.java | 60 ++++++++++++++++--- 4 files changed, 61 insertions(+), 12 deletions(-) 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 a0907bc9c7d9f..ee1d9442462e7 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 @@ -62,7 +62,7 @@ public AbfsClientThrottlingIntercept(String accountName, AbfsConfiguration abfsC // Hide default constructor private AbfsClientThrottlingIntercept(AbfsConfiguration abfsConfiguration) { - //Account name is kept as empty as same instance is shared across all accounts + //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); 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 dc284d60a3aff..3a09c8ca88a8b 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 @@ -229,7 +229,8 @@ private void completeExecute(TracingContext tracingContext) } } - if (result.getStatusCode() >= HttpURLConnection.HTTP_BAD_REQUEST) { + int status = result.getStatusCode(); + if (status < HttpURLConnection.HTTP_OK || status >= HttpURLConnection.HTTP_BAD_REQUEST) { throw new AbfsRestOperationException(result.getStatusCode(), result.getStorageErrorCode(), result.getStorageErrorMessage(), null, result); } 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 bcf94651acad9..21e50bad2be5d 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 @@ -56,6 +56,12 @@ public class ExponentialRetryPolicy { */ private static final double MAX_RANDOM_RATIO = 1.2; + /** + * All status codes less than http 100 signify error + * and should qualify for retry. + */ + private static final int HTTP_CONTINUE = 100; + /** * Holds the random number generator used to calculate randomized backoff intervals */ @@ -126,7 +132,7 @@ 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_INTERNAL_ERROR && statusCode != HttpURLConnection.HTTP_NOT_IMPLEMENTED diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java index 56ca039caa09e..c8533ddd77db5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java @@ -45,6 +45,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; +import org.apache.http.client.methods.HttpGet; import static java.net.HttpURLConnection.HTTP_NOT_FOUND; import static java.net.HttpURLConnection.HTTP_OK; @@ -61,6 +62,8 @@ import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; @RunWith(Parameterized.class) public class TestAbfsRestOperation extends AbstractAbfsIntegrationTest { @@ -68,11 +71,12 @@ public class TestAbfsRestOperation extends AbstractAbfsIntegrationTest { // Specifies whether getOutputStream() or write() throws IOException. public enum error {OUTPUTSTREAM, WRITE}; - public static final int HTTP_EXPECTATION_FAILED = 417; - public static final int REDUCED_RETRY_COUNT = 2; - public static final int REDUCED_BACKOFF_INTERVAL = 100; - public static final int BUFFER_LENGTH = 5; - public static final int BUFFER_OFFSET = 0; + private static final int HTTP_EXPECTATION_FAILED = 417; + private static final int HTTP_ERROR = 0; + private static final int REDUCED_RETRY_COUNT = 2; + private static final int REDUCED_BACKOFF_INTERVAL = 100; + private static final int BUFFER_LENGTH = 5; + private static final int BUFFER_OFFSET = 0; // Specifies whether the expect header is enabled or not. @Parameterized.Parameter @@ -90,11 +94,15 @@ public enum error {OUTPUTSTREAM, WRITE}; @Parameterized.Parameter(3) public error errorType; + // The intercept. + AbfsThrottlingIntercept intercept; + /* HTTP_OK = 200, HTTP_UNAVAILABLE = 503, HTTP_NOT_FOUND = 404, - HTTP_EXPECTATION_FAILED = 417. + HTTP_EXPECTATION_FAILED = 417, + HTTP_ERROR = 0. */ @Parameterized.Parameters(name = "expect={0}-code={1}-error={3}") public static Iterable params() { @@ -103,7 +111,8 @@ public static Iterable params() { {false, HTTP_OK, "OK", error.WRITE}, {true, HTTP_UNAVAILABLE, "ServerBusy", error.OUTPUTSTREAM}, {true, HTTP_NOT_FOUND, "Resource Not Found", error.OUTPUTSTREAM}, - {true, HTTP_EXPECTATION_FAILED, "Expectation Failed", error.OUTPUTSTREAM} + {true, HTTP_EXPECTATION_FAILED, "Expectation Failed", error.OUTPUTSTREAM}, + {true, HTTP_ERROR, "Error", error.OUTPUTSTREAM} }); } @@ -143,10 +152,15 @@ private AbfsRestOperation getRestOperation() throws Exception { abfsConfiguration, REDUCED_RETRY_COUNT, REDUCED_BACKOFF_INTERVAL); + intercept = Mockito.mock(AbfsThrottlingIntercept.class); + Mockito.doNothing().when(intercept).updateMetrics(Mockito.any(), Mockito.any()); + // Gets the client. - AbfsClient testClient = TestAbfsClient.createTestClientFromCurrentContext( + AbfsClient testClient = Mockito.spy(TestAbfsClient.createTestClientFromCurrentContext( abfsClient, - abfsConfig); + abfsConfig)); + + Mockito.doReturn(intercept).when(testClient).getIntercept(); // Expect header is enabled or not based on the parameter. AppendRequestParameters appendRequestParameters @@ -294,6 +308,30 @@ public void testExpectHundredContinue() throws Exception { .isEqualTo(0); Assertions.assertThat(httpOperation.getExpectedBytesToBeSent()) .isEqualTo(BUFFER_LENGTH); + + // Verifies that update Metrics call is made for throttle case and for the first without retry + + // for the retried cases as well. + Mockito.verify(intercept, times(REDUCED_RETRY_COUNT + 1)) + .updateMetrics(Mockito.any(), Mockito.any()); + break; + case HTTP_ERROR: + // In the case of 0 i.e. error case, we should retry. + intercept(IOException.class, + () -> op.execute(tracingContext)); + + // Assert that the request is retried based on reduced retry count configured. + Assertions.assertThat(tracingContext.getRetryCount()) + .describedAs("The retry count is incorrect") + .isEqualTo(REDUCED_RETRY_COUNT); + + // Assert that metrics will be updated correctly. + Assertions.assertThat(httpOperation.getBytesSent()) + .isEqualTo(0); + + // Verifies that update Metrics call is made for error case and for the first without retry + + // for the retried cases as well. + Mockito.verify(intercept, times(REDUCED_RETRY_COUNT + 1)) + .updateMetrics(Mockito.any(), Mockito.any()); break; case HTTP_NOT_FOUND: case HTTP_EXPECTATION_FAILED: @@ -305,6 +343,10 @@ public void testExpectHundredContinue() throws Exception { Assertions.assertThat(tracingContext.getRetryCount()) .describedAs("The retry count is incorrect") .isEqualTo(0); + + // Verifies that update Metrics call is not made for user error case. + Mockito.verify(intercept, never()) + .updateMetrics(Mockito.any(), Mockito.any()); break; default: break; From 8283ef2b79bc7b244d8d61a93df4080cb06d2900 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Tue, 20 Dec 2022 05:32:02 -0800 Subject: [PATCH 40/53] Update md file --- hadoop-tools/hadoop-azure/src/site/markdown/abfs.md | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md index 52a5ac99a3976..aff1e32b83f2d 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md @@ -771,7 +771,13 @@ performance issues. `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. +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 From 9611999a54334d4dd5cc3dcdbd74b28d625b6e18 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Tue, 20 Dec 2022 05:58:43 -0800 Subject: [PATCH 41/53] Remove unused imports --- .../fs/azurebfs/services/AbfsHttpOperation.java | 1 - .../hadoop/fs/azurebfs/utils/TracingContext.java | 1 - .../fs/azurebfs/services/TestAbfsHttpOperation.java | 4 +--- .../fs/azurebfs/services/TestAbfsRestOperation.java | 11 +++++------ .../hadoop-azure/src/test/resources/log4j.properties | 1 - 5 files changed, 6 insertions(+), 12 deletions(-) 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 71f8c475966b4..4cdefcea75137 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,7 +23,6 @@ import java.io.OutputStream; import java.net.HttpURLConnection; import java.net.URL; -import java.util.Arrays; import java.util.List; import javax.net.ssl.HttpsURLConnection; 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 c56ab673819c1..c3af485f285a7 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 @@ -23,7 +23,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsHttpOperation.java index cc9661092fe01..36914a4e4f365 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsHttpOperation.java @@ -18,16 +18,13 @@ package org.apache.hadoop.fs.azurebfs.services; -import java.io.IOException; import java.io.UnsupportedEncodingException; -import java.net.HttpURLConnection; import java.net.MalformedURLException; import java.net.URL; import java.net.URLEncoder; import org.assertj.core.api.Assertions; import org.junit.Test; -import org.mockito.Mockito; import org.apache.hadoop.fs.azurebfs.utils.UriUtils; @@ -115,4 +112,5 @@ private void testIfMaskAndEncodeSuccessful(final String scenario, + "be: " + expectedMaskedEncodedUrl) .isEqualTo(expectedMaskedEncodedUrl); } + } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java index c8533ddd77db5..427bac29270fc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java @@ -35,7 +35,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.azurebfs.Abfs; import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; @@ -45,7 +44,6 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat; -import org.apache.http.client.methods.HttpGet; import static java.net.HttpURLConnection.HTTP_NOT_FOUND; import static java.net.HttpURLConnection.HTTP_OK; @@ -73,6 +71,7 @@ public enum error {OUTPUTSTREAM, WRITE}; private static final int HTTP_EXPECTATION_FAILED = 417; private static final int HTTP_ERROR = 0; + private static final int ZERO = 0; private static final int REDUCED_RETRY_COUNT = 2; private static final int REDUCED_BACKOFF_INTERVAL = 100; private static final int BUFFER_LENGTH = 5; @@ -305,7 +304,7 @@ public void testExpectHundredContinue() throws Exception { // Assert that metrics will be updated correctly. Assertions.assertThat(httpOperation.getBytesSent()) - .isEqualTo(0); + .isEqualTo(ZERO); Assertions.assertThat(httpOperation.getExpectedBytesToBeSent()) .isEqualTo(BUFFER_LENGTH); @@ -315,7 +314,7 @@ public void testExpectHundredContinue() throws Exception { .updateMetrics(Mockito.any(), Mockito.any()); break; case HTTP_ERROR: - // In the case of 0 i.e. error case, we should retry. + // In the case of http status code 0 i.e. error case, we should retry. intercept(IOException.class, () -> op.execute(tracingContext)); @@ -326,7 +325,7 @@ public void testExpectHundredContinue() throws Exception { // Assert that metrics will be updated correctly. Assertions.assertThat(httpOperation.getBytesSent()) - .isEqualTo(0); + .isEqualTo(ZERO); // Verifies that update Metrics call is made for error case and for the first without retry + // for the retried cases as well. @@ -342,7 +341,7 @@ public void testExpectHundredContinue() throws Exception { // Assert that the request is not retried. Assertions.assertThat(tracingContext.getRetryCount()) .describedAs("The retry count is incorrect") - .isEqualTo(0); + .isEqualTo(ZERO); // Verifies that update Metrics call is not made for user error case. Mockito.verify(intercept, never()) diff --git a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties index daca5870dd5b2..9f72d03653306 100644 --- a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties @@ -26,7 +26,6 @@ log4j.logger.org.apache.hadoop.fs.azure.AzureFileSystemThreadPoolExecutor=DEBUG log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG log4j.logger.org.apache.hadoop.fs.azurebfs.contracts.services.TracingService=TRACE log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsClient=DEBUG -log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsIoUtils=DEBUG # after here: turn off log messages from other parts of the system # which only clutter test reports. From 7899d1c28796a28dcdd9823d7afdaf33e1aada6c Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Tue, 20 Dec 2022 11:21:51 -0800 Subject: [PATCH 42/53] Checkstyle fixes --- .../InvalidAbfsRestOperationException.java | 6 ++-- .../fs/azurebfs/services/TestAbfsClient.java | 19 +++++------- .../services/TestAbfsRestOperation.java | 30 +++++++++---------- 3 files changed, 26 insertions(+), 29 deletions(-) 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 9cc7b2389f0f8..2b8bb2d26445b 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 @@ -31,7 +31,7 @@ @InterfaceStability.Evolving public class InvalidAbfsRestOperationException extends AbfsRestOperationException { - private final static String errorMessage = "InvalidAbfsRestOperationException"; + private static final String ERROR_MESSAGE = "InvalidAbfsRestOperationException"; public InvalidAbfsRestOperationException( final Exception innerException) { @@ -40,7 +40,7 @@ public InvalidAbfsRestOperationException( AzureServiceErrorCode.UNKNOWN.getErrorCode(), innerException != null ? innerException.toString() - : errorMessage, + : ERROR_MESSAGE, innerException); } @@ -56,7 +56,7 @@ public InvalidAbfsRestOperationException( AzureServiceErrorCode.UNKNOWN.getErrorCode(), innerException != null ? innerException.toString() - : errorMessage + " RetryCount: " + retryCount, + : ERROR_MESSAGE + " RetryCount: " + retryCount, innerException); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index 2bbc61df16ae0..2f60d83e07945 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -86,6 +86,7 @@ public final class TestAbfsClient extends AbstractAbfsIntegrationTest { private static final String ACCOUNT_NAME = "bogusAccountName.dfs.core.windows.net"; private static final String FS_AZURE_USER_AGENT_PREFIX = "Partner Service"; + private static final String TEST_PATH = "/testfile"; public static final int REDUCED_RETRY_COUNT = 2; public static final int REDUCED_BACKOFF_INTERVAL = 100; public static final int BUFFER_LENGTH = 5; @@ -481,7 +482,6 @@ public void testExpectHundredContinue() throws Exception { byte[] buffer = getRandomBytesArray(BUFFER_LENGTH); // Create a test container to upload the data. - final String TEST_PATH = "/testfile"; Path testPath = path(TEST_PATH); fs.create(testPath); String finalTestPath = testPath.toString() @@ -538,7 +538,7 @@ public void testExpectHundredContinue() throws Exception { .when(urlConnection) .getResponseMessage(); - // Make the getOuputStream throw IOException to see it returns from the sendRequest correctly. + // Make the getOutputStream throw IOException to see it returns from the sendRequest correctly. Mockito.doThrow(new ProtocolException("Server rejected Operation")) .when(urlConnection) .getOutputStream(); @@ -550,15 +550,12 @@ public void testExpectHundredContinue() throws Exception { .getHttpOperation(Mockito.any(), Mockito.any(), Mockito.any()); // Mock the restOperation for the client. - Mockito.doReturn(op).when(testClient).getAbfsRestOperationForAppend( - Mockito.nullable(AbfsRestOperationType.class), - Mockito.nullable(String.class), - Mockito.nullable(URL.class), - Mockito.nullable(List.class), - Mockito.nullable(byte[].class), - Mockito.nullable(int.class), - Mockito.nullable(int.class), - Mockito.nullable(String.class)); + Mockito.doReturn(op) + .when(testClient) + .getAbfsRestOperationForAppend(Mockito.any(), + Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.nullable(int.class), Mockito.nullable(int.class), + Mockito.any()); TracingContext tracingContext = Mockito.spy(new TracingContext("abcd", "abcde", FSOperationType.APPEND, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java index 427bac29270fc..f0c310d1c668e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java @@ -67,7 +67,7 @@ public class TestAbfsRestOperation extends AbstractAbfsIntegrationTest { // Specifies whether getOutputStream() or write() throws IOException. - public enum error {OUTPUTSTREAM, WRITE}; + public enum ErrorType {OUTPUTSTREAM, WRITE}; private static final int HTTP_EXPECTATION_FAILED = 417; private static final int HTTP_ERROR = 0; @@ -76,6 +76,7 @@ public enum error {OUTPUTSTREAM, WRITE}; private static final int REDUCED_BACKOFF_INTERVAL = 100; private static final int BUFFER_LENGTH = 5; private static final int BUFFER_OFFSET = 0; + private final String TEST_PATH = "/testfile"; // Specifies whether the expect header is enabled or not. @Parameterized.Parameter @@ -91,10 +92,10 @@ public enum error {OUTPUTSTREAM, WRITE}; // Gives the errorType based on the enum. @Parameterized.Parameter(3) - public error errorType; + public ErrorType errorType; // The intercept. - AbfsThrottlingIntercept intercept; + private AbfsThrottlingIntercept intercept; /* HTTP_OK = 200, @@ -103,15 +104,15 @@ public enum error {OUTPUTSTREAM, WRITE}; HTTP_EXPECTATION_FAILED = 417, HTTP_ERROR = 0. */ - @Parameterized.Parameters(name = "expect={0}-code={1}-error={3}") + @Parameterized.Parameters(name = "expect={0}-code={1}-ErrorType={3}") public static Iterable params() { return Arrays.asList(new Object[][]{ - {true, HTTP_OK, "OK", error.WRITE}, - {false, HTTP_OK, "OK", error.WRITE}, - {true, HTTP_UNAVAILABLE, "ServerBusy", error.OUTPUTSTREAM}, - {true, HTTP_NOT_FOUND, "Resource Not Found", error.OUTPUTSTREAM}, - {true, HTTP_EXPECTATION_FAILED, "Expectation Failed", error.OUTPUTSTREAM}, - {true, HTTP_ERROR, "Error", error.OUTPUTSTREAM} + {true, HTTP_OK, "OK", ErrorType.WRITE}, + {false, HTTP_OK, "OK", ErrorType.WRITE}, + {true, HTTP_UNAVAILABLE, "ServerBusy", ErrorType.OUTPUTSTREAM}, + {true, HTTP_NOT_FOUND, "Resource Not Found", ErrorType.OUTPUTSTREAM}, + {true, HTTP_EXPECTATION_FAILED, "Expectation Failed", ErrorType.OUTPUTSTREAM}, + {true, HTTP_ERROR, "Error", ErrorType.OUTPUTSTREAM} }); } @@ -171,7 +172,6 @@ private AbfsRestOperation getRestOperation() throws Exception { byte[] buffer = getRandomBytesArray(5); // Create a test container to upload the data. - final String TEST_PATH = "/testfile"; Path testPath = path(TEST_PATH); fs.create(testPath); String finalTestPath = testPath.toString().substring(testPath.toString().lastIndexOf("/")); @@ -314,7 +314,7 @@ public void testExpectHundredContinue() throws Exception { .updateMetrics(Mockito.any(), Mockito.any()); break; case HTTP_ERROR: - // In the case of http status code 0 i.e. error case, we should retry. + // In the case of http status code 0 i.e. ErrorType case, we should retry. intercept(IOException.class, () -> op.execute(tracingContext)); @@ -327,14 +327,14 @@ public void testExpectHundredContinue() throws Exception { Assertions.assertThat(httpOperation.getBytesSent()) .isEqualTo(ZERO); - // Verifies that update Metrics call is made for error case and for the first without retry + + // Verifies that update Metrics call is made for ErrorType case and for the first without retry + // for the retried cases as well. Mockito.verify(intercept, times(REDUCED_RETRY_COUNT + 1)) .updateMetrics(Mockito.any(), Mockito.any()); break; case HTTP_NOT_FOUND: case HTTP_EXPECTATION_FAILED: - // In the case of 4xx error. i.e. user error, retry should not happen. + // In the case of 4xx ErrorType. i.e. user ErrorType, retry should not happen. intercept(AzureBlobFileSystemException.class, () -> op.execute(tracingContext)); @@ -343,7 +343,7 @@ public void testExpectHundredContinue() throws Exception { .describedAs("The retry count is incorrect") .isEqualTo(ZERO); - // Verifies that update Metrics call is not made for user error case. + // Verifies that update Metrics call is not made for user ErrorType case. Mockito.verify(intercept, never()) .updateMetrics(Mockito.any(), Mockito.any()); break; From 5398f2aae8c0fa83bb4997d3bd794de23dfcbac3 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Wed, 21 Dec 2022 01:28:26 -0800 Subject: [PATCH 43/53] Checkstyle fixes --- .../hadoop-azure/src/config/checkstyle-suppressions.xml | 2 ++ .../exceptions/InvalidAbfsRestOperationException.java | 2 +- .../hadoop/fs/azurebfs/services/AbfsRestOperation.java | 7 +++++-- .../hadoop/fs/azurebfs/services/TestAbfsRestOperation.java | 2 +- 4 files changed, 9 insertions(+), 4 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml index e9ca716400e4e..231a5b5ea149d 100644 --- a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml +++ b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml @@ -48,6 +48,8 @@ files="org[\\/]apache[\\/]hadoop[\\/]fs[\\/]azurebfs[\\/]utils[\\/]Base64.java"/> + 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 2b8bb2d26445b..285297024c710 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 @@ -45,7 +45,7 @@ public InvalidAbfsRestOperationException( } /** - * Adds the retry count along with the exception + * 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. */ 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 3a09c8ca88a8b..f5d058b6761d6 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 @@ -230,11 +230,14 @@ private void completeExecute(TracingContext tracingContext) } int status = result.getStatusCode(); - if (status < HttpURLConnection.HTTP_OK || status >= HttpURLConnection.HTTP_BAD_REQUEST) { + if (status < HttpURLConnection.HTTP_OK) { + 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); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java index f0c310d1c668e..77eb533a6c58c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java @@ -76,7 +76,7 @@ public enum ErrorType {OUTPUTSTREAM, WRITE}; private static final int REDUCED_BACKOFF_INTERVAL = 100; private static final int BUFFER_LENGTH = 5; private static final int BUFFER_OFFSET = 0; - private final String TEST_PATH = "/testfile"; + private static final String TEST_PATH = "/testfile"; // Specifies whether the expect header is enabled or not. @Parameterized.Parameter From 796b77438fb444a78506869a7bd53942d4d3dfb7 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Thu, 22 Dec 2022 04:10:59 -0800 Subject: [PATCH 44/53] PR comments addressing --- .../azurebfs/services/AbfsHttpOperation.java | 6 ++--- .../azurebfs/services/AbfsRestOperation.java | 22 +++++++++++++++---- .../services/ExponentialRetryPolicy.java | 4 +++- 3 files changed, 23 insertions(+), 9 deletions(-) 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 4cdefcea75137..692b7f42a93d9 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 @@ -346,12 +346,10 @@ public void sendRequest(byte[] buffer, int offset, int length) throws IOExceptio */ String expectHeader = this.connection.getRequestProperty(EXPECT); if (expectHeader != null && expectHeader.equals(HUNDRED_CONTINUE)) { - LOG.debug("Getting output stream failed with expect header enabled, returning back " - + ExceptionUtils.getStackTrace(e)); + LOG.debug("Getting output stream failed with expect header enabled, returning back ", e); return; } else { - LOG.debug("Getting output stream failed without expect header enabled, throwing exception " - + ExceptionUtils.getStackTrace(e)); + LOG.debug("Getting output stream failed without expect header enabled, throwing exception ", e); throw e; } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java index f5d058b6761d6..b28d1dce8d882 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 @@ -61,6 +61,9 @@ public class AbfsRestOperation { // Used only by AbfsInputStream/AbfsOutputStream to reuse SAS tokens. private final String sasToken; + // All status codes less than http 100 signify error. + private static final int HTTP_CONTINUE = 100; + private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); // For uploads, this is the request entity body. For downloads, @@ -230,7 +233,13 @@ private void completeExecute(TracingContext tracingContext) } int status = result.getStatusCode(); - if (status < HttpURLConnection.HTTP_OK) { + /* + 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); } @@ -330,9 +339,14 @@ private boolean executeHttpOperation(final int retryCount, return false; } finally { int status = httpOperation.getStatusCode(); - // If the socket is terminated prior to receiving a response, the HTTP - // status may be 0 or -1. A status less than 300 (2xx range) or greater than or equal - // to 500 (5xx range) should contribute to metrics updation. + /* + A status less than 300 (2xx range) or greater than or equal + to 500 (5xx range) should contribute to throttling metrics updation. + 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) { 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 21e50bad2be5d..effa5cefc604c 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 @@ -124,7 +124,9 @@ 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. * * @param retryCount The current retry attempt count. * @param statusCode The status code of the response, or -1 for socket error. From 115b0b630c4077504169fcdf85bae01f928e3fef Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Thu, 22 Dec 2022 05:53:50 -0800 Subject: [PATCH 45/53] PR comments --- .../apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java | 1 - 1 file changed, 1 deletion(-) 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 692b7f42a93d9..7e3efa686ed7c 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 @@ -28,7 +28,6 @@ import javax.net.ssl.HttpsURLConnection; import javax.net.ssl.SSLSocketFactory; -import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.hadoop.fs.azurebfs.utils.UriUtils; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; From db89c786440ea48f040d07c817f135f5d294a343 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Fri, 23 Dec 2022 01:51:34 -0800 Subject: [PATCH 46/53] remove stter for connection --- .../azurebfs/services/AbfsHttpOperation.java | 66 ++++++++++++++++--- .../azurebfs/services/AbfsRestOperation.java | 2 +- .../fs/azurebfs/services/TestAbfsClient.java | 32 +++++---- .../services/TestAbfsRestOperation.java | 34 +++++----- 4 files changed, 94 insertions(+), 40 deletions(-) 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 7e3efa686ed7c..67ac0c31665d6 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 @@ -114,10 +114,6 @@ protected HttpURLConnection getConnection() { return connection; } - void setConnection(final HttpURLConnection connection) { - this.connection = connection; - } - public String getMethod() { return method; } @@ -294,7 +290,7 @@ public AbfsHttpOperation(final URL url, final String method, final List Date: Mon, 26 Dec 2022 15:10:49 +0530 Subject: [PATCH 47/53] Update AbfsClient.java --- .../hadoop/fs/azurebfs/services/AbfsClient.java | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) 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 2178acee8e7b6..c29ed47c7941e 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 @@ -691,13 +691,15 @@ public AbfsRestOperation append(final String path, final byte[] buffer, 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 disabled. - // When "100-continue" header is enabled but a non Http 100 response comes, - // JDK fails to provide all response headers. - // 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. + /* + If the http response code indicates a user error we retry + the same append request with expect header 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"); From f3e2e14e915c9a70c16529531236b5aa25c979c1 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Thu, 16 Mar 2023 06:48:35 -0700 Subject: [PATCH 48/53] Addressing PR comments --- .../azurebfs/constants/AbfsHttpConstants.java | 5 ++ .../azurebfs/constants/ConfigurationKeys.java | 4 ++ .../fs/azurebfs/services/AbfsClient.java | 26 +++++--- .../AbfsClientThrottlingIntercept.java | 12 ++-- .../azurebfs/services/AbfsRestOperation.java | 15 +++-- .../services/ExponentialRetryPolicy.java | 8 +-- .../azurebfs/AbstractAbfsIntegrationTest.java | 4 +- .../ITestAzureBlobFileSystemCreate.java | 4 +- .../ITestAzureBlobFileSystemDelete.java | 14 ++-- ...stAbfsClient.java => ITestAbfsClient.java} | 26 ++++---- ...ation.java => ITestAbfsRestOperation.java} | 66 +++++++++---------- .../services/TestAbfsRenameRetryRecovery.java | 2 +- .../services/TestExponentialRetryPolicy.java | 4 +- 13 files changed, 103 insertions(+), 87 deletions(-) rename hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/{TestAbfsClient.java => ITestAbfsClient.java} (96%) rename hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/{TestAbfsRestOperation.java => ITestAbfsRestOperation.java} (87%) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java index 3d8d5f8e78d38..241907657af72 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java @@ -64,6 +64,11 @@ public final class AbfsHttpConstants { public static final String HTTP_METHOD_PATCH = "PATCH"; public static final String HTTP_METHOD_POST = "POST"; public static final String HTTP_METHOD_PUT = "PUT"; + /** + * All status codes less than http 100 signify error + * and should qualify for retry. + */ + public static final int HTTP_CONTINUE = 100; // Abfs generic constants public static final String SINGLE_WHITE_SPACE = " "; 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 6488a7f29e07d..e3052cd7bbc45 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,6 +35,10 @@ 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\\.(.*)"; 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 c29ed47c7941e..8648cc3e187a6 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 @@ -686,8 +686,13 @@ public AbfsRestOperation append(final String path, final byte[] buffer, final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString()); final AbfsRestOperation op = getAbfsRestOperationForAppend(AbfsRestOperationType.Append, - HTTP_METHOD_PUT, url, requestHeaders, buffer, reqParams.getoffset(), - reqParams.getLength(), sasTokenForReuse); + HTTP_METHOD_PUT, + url, + requestHeaders, + buffer, + reqParams.getoffset(), + reqParams.getLength(), + sasTokenForReuse); try { op.execute(tracingContext); } catch (AzureBlobFileSystemException e) { @@ -702,7 +707,7 @@ public AbfsRestOperation append(final String path, final byte[] buffer, */ int responseStatusCode = ((AbfsRestOperationException) e).getStatusCode(); if (checkUserError(responseStatusCode) && reqParams.isExpectHeaderEnabled()) { - LOG.debug("User error, retrying without 100 continue enabled"); + LOG.debug("User error, retrying without 100 continue enabled for the given path " + path); reqParams.setExpectHeaderEnabled(false); return this.append(path, buffer, reqParams, cachedSasToken, tracingContext); @@ -715,9 +720,14 @@ public AbfsRestOperation append(final String path, final byte[] buffer, && appendSuccessCheckOp(op, path, (reqParams.getPosition() + reqParams.getLength()), tracingContext)) { final AbfsRestOperation successOp = getAbfsRestOperationForAppend( - AbfsRestOperationType.Append, HTTP_METHOD_PUT, url, requestHeaders, - buffer, reqParams.getoffset(), reqParams.getLength(), - sasTokenForReuse); + AbfsRestOperationType.Append, + HTTP_METHOD_PUT, + url, + requestHeaders, + buffer, + reqParams.getoffset(), + reqParams.getLength(), + sasTokenForReuse); successOp.hardSetResult(HttpURLConnection.HTTP_OK); return successOp; } @@ -728,7 +738,7 @@ && appendSuccessCheckOp(op, path, } /** - * Returns the rest operation for append + * Returns the rest operation for append. * @param operationType The AbfsRestOperationType. * @param httpMethod specifies the httpMethod. * @param url specifies the url. @@ -737,7 +747,7 @@ && appendSuccessCheckOp(op, path, * @param bufferOffset The buffer offset. * @param bufferLength The buffer Length. * @param sasTokenForReuse The sasToken. - * @return AbfsRestOperation op + * @return AbfsRestOperation op. */ @VisibleForTesting AbfsRestOperation getAbfsRestOperationForAppend(final AbfsRestOperationType operationType, 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 ee1d9442462e7..b66c8f0893066 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 @@ -62,7 +62,7 @@ public AbfsClientThrottlingIntercept(String accountName, AbfsConfiguration abfsC // Hide default constructor private AbfsClientThrottlingIntercept(AbfsConfiguration abfsConfiguration) { - //Account name is kept as empty as same instance is shared across all accounts. + // 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); @@ -117,11 +117,11 @@ static AbfsClientThrottlingIntercept initializeSingleton(AbfsConfiguration abfsC } /** - * Updates the metrics for the case when getOutputStream() caught an IOException - * and response code signifies throttling. + * 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 + * @return true if the operation is throttled and has some bytes to transfer. */ private boolean updateBytesTransferred(boolean isThrottledOperation, AbfsHttpOperation abfsHttpOperation) { @@ -148,7 +148,7 @@ public void updateMetrics(AbfsRestOperationType operationType, boolean isFailedOperation = (status < HttpURLConnection.HTTP_OK || status >= HttpURLConnection.HTTP_INTERNAL_ERROR); - // If status code is 503, it considered a throttled operation. + // If status code is 503, it is considered as a throttled operation. boolean isThrottledOperation = (status == HTTP_UNAVAILABLE); switch (operationType) { @@ -160,7 +160,7 @@ public void updateMetrics(AbfsRestOperationType operationType, throttling but there were some expectedBytesToBeSent. */ if (updateBytesTransferred(isThrottledOperation, abfsHttpOperation)) { - LOG.debug("Updating metrics due to throttling"); + LOG.debug("Updating metrics due to throttling for path " + abfsHttpOperation.getConnUrl().getPath()); contentLength = abfsHttpOperation.getExpectedBytesToBeSent(); } } 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 badf8cacd9f83..872f8fa529352 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 @@ -37,6 +37,8 @@ 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.HTTP_CONTINUE; + /** * The AbfsRestOperation for Rest AbfsClient. */ @@ -61,9 +63,6 @@ public class AbfsRestOperation { // Used only by AbfsInputStream/AbfsOutputStream to reuse SAS tokens. private final String sasToken; - // All status codes less than http 100 signify error. - private static final int HTTP_CONTINUE = 100; - private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class); // For uploads, this is the request entity body. For downloads, @@ -250,6 +249,14 @@ private void completeExecute(TracingContext tracingContext) LOG.trace("{} REST operation complete", operationType); } + /** + * Returns a new object of AbfsHttpOperation. + * @param url The url for the operation. + * @param method The http method. + * @param requestHeaders The request headers for the operation. + * @return AbfsHttpOperation object. + * @throws IOException + */ AbfsHttpOperation getHttpOperation(final URL url, final String method, final List requestHeaders) throws IOException { return new AbfsHttpOperation(url, method, requestHeaders); @@ -341,7 +348,7 @@ private boolean executeHttpOperation(final int retryCount, 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 updation. + 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 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 effa5cefc604c..227bdc5fc1c9b 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.classification.VisibleForTesting; +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_CONTINUE; + /** * Retry policy used by AbfsClient. * */ @@ -56,12 +58,6 @@ public class ExponentialRetryPolicy { */ private static final double MAX_RANDOM_RATIO = 1.2; - /** - * All status codes less than http 100 signify error - * and should qualify for retry. - */ - private static final int HTTP_CONTINUE = 100; - /** * Holds the random number generator used to calculate randomized backoff intervals */ 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 c5bf85a4f8190..74655fd573620 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 @@ -42,7 +42,7 @@ 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.TestAbfsClient; +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; @@ -254,7 +254,7 @@ public Hashtable call() throws Exception { } public AccessTokenProvider getAccessTokenProvider(final AzureBlobFileSystem fs) { - return TestAbfsClient.getAccessTokenProvider(fs.getAbfsStore().getClient()); + return ITestAbfsClient.getAccessTokenProvider(fs.getAbfsStore().getClient()); } public void loadConfiguredFileSystem() throws Exception { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 2f23ac5c5c708..d9a3cea089f63 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -43,7 +43,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; -import org.apache.hadoop.fs.azurebfs.services.TestAbfsClient; +import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; @@ -362,7 +362,7 @@ public void testNegativeScenariosForCreateOverwriteDisabled() // Get mock AbfsClient with current config AbfsClient mockClient - = TestAbfsClient.getMockAbfsClient( + = ITestAbfsClient.getMockAbfsClient( fs.getAbfsStore().getClient(), fs.getAbfsStore().getAbfsConfiguration()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java index db181fb5dd660..1f0ff667522da 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java @@ -35,7 +35,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; -import org.apache.hadoop.fs.azurebfs.services.TestAbfsClient; +import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient; import org.apache.hadoop.fs.azurebfs.services.TestAbfsPerfTracker; import org.apache.hadoop.fs.azurebfs.utils.TestMockHelpers; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; @@ -176,7 +176,7 @@ public void testDeleteIdempotency() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); AbfsClient abfsClient = fs.getAbfsStore().getClient(); - AbfsClient testClient = TestAbfsClient.createTestClientFromCurrentContext( + AbfsClient testClient = ITestAbfsClient.createTestClientFromCurrentContext( abfsClient, abfsConfig); @@ -223,7 +223,7 @@ public void testDeleteIdempotency() throws Exception { public void testDeleteIdempotencyTriggerHttp404() throws Exception { final AzureBlobFileSystem fs = getFileSystem(); - AbfsClient client = TestAbfsClient.createTestClientFromCurrentContext( + AbfsClient client = ITestAbfsClient.createTestClientFromCurrentContext( fs.getAbfsStore().getClient(), this.getConfiguration()); @@ -242,7 +242,7 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { getTestTracingContext(fs, true))); // mock idempotency check to mimic retried case - AbfsClient mockClient = TestAbfsClient.getMockAbfsClient( + AbfsClient mockClient = ITestAbfsClient.getMockAbfsClient( fs.getAbfsStore().getClient(), this.getConfiguration()); AzureBlobFileSystemStore mockStore = mock(AzureBlobFileSystemStore.class); @@ -257,10 +257,10 @@ public void testDeleteIdempotencyTriggerHttp404() throws Exception { // Case 2: Mimic retried case // Idempotency check on Delete always returns success - AbfsRestOperation idempotencyRetOp = TestAbfsClient.getRestOp( + AbfsRestOperation idempotencyRetOp = ITestAbfsClient.getRestOp( DeletePath, mockClient, HTTP_METHOD_DELETE, - TestAbfsClient.getTestUrl(mockClient, "/NonExistingPath"), - TestAbfsClient.getTestRequestHeaders(mockClient)); + ITestAbfsClient.getTestUrl(mockClient, "/NonExistingPath"), + ITestAbfsClient.getTestRequestHeaders(mockClient)); idempotencyRetOp.hardSetResult(HTTP_OK); doReturn(idempotencyRetOp).when(mockClient).deleteIdempotencyCheckOp(any()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java similarity index 96% rename from hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java rename to hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java index 2273cf5b77f69..bcad51ec233ae 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java @@ -82,7 +82,7 @@ * Test useragent of abfs client. * */ -public final class TestAbfsClient extends AbstractAbfsIntegrationTest { +public final class ITestAbfsClient extends AbstractAbfsIntegrationTest { private static final String ACCOUNT_NAME = "bogusAccountName.dfs.core.windows.net"; private static final String FS_AZURE_USER_AGENT_PREFIX = "Partner Service"; @@ -94,7 +94,7 @@ public final class TestAbfsClient extends AbstractAbfsIntegrationTest { private final Pattern userAgentStringPattern; - public TestAbfsClient() throws Exception { + public ITestAbfsClient() throws Exception { StringBuilder regEx = new StringBuilder(); regEx.append("^"); regEx.append(APN_VERSION); @@ -340,22 +340,22 @@ public static AbfsClient getMockAbfsClient(AbfsClient baseAbfsClientInstance, abfsConfig.getAccountName().substring(0, abfsConfig.getAccountName().indexOf(DOT)), abfsConfig)); // override baseurl - client = TestAbfsClient.setAbfsClientField(client, "abfsConfiguration", + client = ITestAbfsClient.setAbfsClientField(client, "abfsConfiguration", abfsConfig); // override baseurl - client = TestAbfsClient.setAbfsClientField(client, "baseUrl", + client = ITestAbfsClient.setAbfsClientField(client, "baseUrl", baseAbfsClientInstance.getBaseUrl()); // override auth provider if (currentAuthType == AuthType.SharedKey) { - client = TestAbfsClient.setAbfsClientField(client, "sharedKeyCredentials", + client = ITestAbfsClient.setAbfsClientField(client, "sharedKeyCredentials", new SharedKeyCredentials( abfsConfig.getAccountName().substring(0, abfsConfig.getAccountName().indexOf(DOT)), abfsConfig.getStorageAccountKey())); } else { - client = TestAbfsClient.setAbfsClientField(client, "tokenProvider", + client = ITestAbfsClient.setAbfsClientField(client, "tokenProvider", abfsConfig.getTokenProvider()); } @@ -363,7 +363,7 @@ public static AbfsClient getMockAbfsClient(AbfsClient baseAbfsClientInstance, String userAgent = "APN/1.0 Azure Blob FS/3.4.0-SNAPSHOT (PrivateBuild " + "JavaJRE 1.8.0_252; Linux 5.3.0-59-generic/amd64; openssl-1.0; " + "UNKNOWN/UNKNOWN) MSFT"; - client = TestAbfsClient.setAbfsClientField(client, "userAgent", userAgent); + client = ITestAbfsClient.setAbfsClientField(client, "userAgent", userAgent); return client; } @@ -469,7 +469,7 @@ public void testExpectHundredContinue() throws Exception { // Gets the client. AbfsClient testClient = Mockito.spy( - TestAbfsClient.createTestClientFromCurrentContext( + ITestAbfsClient.createTestClientFromCurrentContext( abfsClient, abfsConfig)); @@ -489,7 +489,7 @@ public void testExpectHundredContinue() throws Exception { // Creates a list of request headers. final List requestHeaders - = TestAbfsClient.getTestRequestHeaders(testClient); + = ITestAbfsClient.getTestRequestHeaders(testClient); requestHeaders.add( new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); if (appendRequestParameters.isExpectHeaderEnabled()) { @@ -566,9 +566,7 @@ public void testExpectHundredContinue() throws Exception { TracingHeaderFormat.ALL_ID_FORMAT, null)); // Check that expect header is enabled before the append call. - Assertions.assertThat(appendRequestParameters.isExpectHeaderEnabled()) - .describedAs("The expect header is not true before the append call") - .isEqualTo(true); + assertTrue(appendRequestParameters.isExpectHeaderEnabled()); intercept(AzureBlobFileSystemException.class, () -> testClient.append(finalTestPath, buffer, appendRequestParameters, null, tracingContext)); @@ -579,8 +577,6 @@ public void testExpectHundredContinue() throws Exception { .isEqualTo(0); // Verify that the same request was retried with expect header disabled. - Assertions.assertThat(appendRequestParameters.isExpectHeaderEnabled()) - .describedAs("The expect header is not false") - .isEqualTo(false); + assertFalse(appendRequestParameters.isExpectHeaderEnabled()); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java similarity index 87% rename from hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java rename to hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java index aea98e480b09a..95cde41579ce7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java @@ -64,7 +64,7 @@ import static org.mockito.Mockito.times; @RunWith(Parameterized.class) -public class TestAbfsRestOperation extends AbstractAbfsIntegrationTest { +public class ITestAbfsRestOperation extends AbstractAbfsIntegrationTest { // Specifies whether getOutputStream() or write() throws IOException. public enum ErrorType {OUTPUTSTREAM, WRITE}; @@ -116,7 +116,7 @@ public static Iterable params() { }); } - public TestAbfsRestOperation() throws Exception { + public ITestAbfsRestOperation() throws Exception { super(); } @@ -156,7 +156,7 @@ private AbfsRestOperation getRestOperation() throws Exception { Mockito.doNothing().when(intercept).updateMetrics(Mockito.any(), Mockito.any()); // Gets the client. - AbfsClient testClient = Mockito.spy(TestAbfsClient.createTestClientFromCurrentContext( + AbfsClient testClient = Mockito.spy(ITestAbfsClient.createTestClientFromCurrentContext( abfsClient, abfsConfig)); @@ -177,7 +177,7 @@ private AbfsRestOperation getRestOperation() throws Exception { String finalTestPath = testPath.toString().substring(testPath.toString().lastIndexOf("/")); // Creates a list of request headers. - final List requestHeaders = TestAbfsClient.getTestRequestHeaders(testClient); + final List requestHeaders = ITestAbfsClient.getTestRequestHeaders(testClient); requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, HTTP_METHOD_PATCH)); if (appendRequestParameters.isExpectHeaderEnabled()) { requestHeaders.add(new AbfsHttpHeader(EXPECT, HUNDRED_CONTINUE)); @@ -261,6 +261,22 @@ public void write(final int i) throws IOException { return op; } + void assertTraceContextState(int retryCount, int assertRetryCount, int bytesSent, int assertBytesSent, + int expectedBytesSent, int assertExpectedBytesSent) { + // Assert that the request is retried or not. + Assertions.assertThat(retryCount) + .describedAs("The retry count is incorrect") + .isEqualTo(assertRetryCount); + + // Assert that metrics will be updated correctly. + Assertions.assertThat(bytesSent) + .describedAs("The bytes sent is incorrect") + .isEqualTo(assertBytesSent); + Assertions.assertThat(expectedBytesSent) + .describedAs("The expected bytes sent is incorrect") + .isEqualTo(assertExpectedBytesSent); + } + /** * Test the functionalities based on whether getOutputStream() or write() * throws exception and what is the corresponding response code. @@ -283,14 +299,9 @@ public void testExpectHundredContinue() throws Exception { intercept(IOException.class, () -> op.execute(tracingContext)); - // Assert that the request is retried based on reduced retry count configured. - Assertions.assertThat(tracingContext.getRetryCount()) - .describedAs("The retry count is incorrect") - .isEqualTo(REDUCED_RETRY_COUNT); - - // Assert that metrics will be updated correctly. - Assertions.assertThat(httpOperation.getBytesSent()) - .isEqualTo(BUFFER_LENGTH); + // Asserting update of metrics and retries. + assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, httpOperation.getBytesSent(), BUFFER_LENGTH, + 0, 0); break; case OUTPUTSTREAM: switch (responseCode) { @@ -299,16 +310,9 @@ public void testExpectHundredContinue() throws Exception { intercept(IOException.class, () -> op.execute(tracingContext)); - // Assert that the request is retried based on reduced retry count configured. - Assertions.assertThat(tracingContext.getRetryCount()) - .describedAs("The retry count is incorrect") - .isEqualTo(REDUCED_RETRY_COUNT); - - // Assert that metrics will be updated correctly. - Assertions.assertThat(httpOperation.getBytesSent()) - .isEqualTo(ZERO); - Assertions.assertThat(httpOperation.getExpectedBytesToBeSent()) - .isEqualTo(BUFFER_LENGTH); + // Asserting update of metrics and retries. + assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, httpOperation.getBytesSent(), ZERO, + httpOperation.getExpectedBytesToBeSent(), BUFFER_LENGTH); // Verifies that update Metrics call is made for throttle case and for the first without retry + // for the retried cases as well. @@ -320,14 +324,9 @@ public void testExpectHundredContinue() throws Exception { intercept(IOException.class, () -> op.execute(tracingContext)); - // Assert that the request is retried based on reduced retry count configured. - Assertions.assertThat(tracingContext.getRetryCount()) - .describedAs("The retry count is incorrect") - .isEqualTo(REDUCED_RETRY_COUNT); - - // Assert that metrics will be updated correctly. - Assertions.assertThat(httpOperation.getBytesSent()) - .isEqualTo(ZERO); + // Asserting update of metrics and retries. + assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, httpOperation.getBytesSent(), + ZERO, 0, 0); // Verifies that update Metrics call is made for ErrorType case and for the first without retry + // for the retried cases as well. @@ -340,10 +339,9 @@ public void testExpectHundredContinue() throws Exception { intercept(AzureBlobFileSystemException.class, () -> op.execute(tracingContext)); - // Assert that the request is not retried. - Assertions.assertThat(tracingContext.getRetryCount()) - .describedAs("The retry count is incorrect") - .isEqualTo(ZERO); + // Asserting update of metrics and retries. + assertTraceContextState(tracingContext.getRetryCount(), ZERO, 0, + 0, 0, 0); // Verifies that update Metrics call is not made for user ErrorType case. Mockito.verify(intercept, never()) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java index 65ea79b36bd0e..f5cbceaddd8b4 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java @@ -58,7 +58,7 @@ public void testRenameFailuresDueToIncompleteMetadata() throws Exception { String destNoParentPath = "/NoParent/Dest"; AzureBlobFileSystem fs = getFileSystem(); - AbfsClient mockClient = TestAbfsClient.getMockAbfsClient( + AbfsClient mockClient = ITestAbfsClient.getMockAbfsClient( fs.getAbfsStore().getClient(), fs.getAbfsStore().getAbfsConfiguration()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java index a1fc4e138d6cd..12ab4e9ead688 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestExponentialRetryPolicy.java @@ -103,7 +103,7 @@ public void testThrottlingIntercept() throws Exception { AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration, "dummy.dfs.core.windows.net"); AbfsThrottlingIntercept intercept; - AbfsClient abfsClient = TestAbfsClient.createTestClientFromCurrentContext(fs.getAbfsStore().getClient(), abfsConfiguration); + AbfsClient abfsClient = ITestAbfsClient.createTestClientFromCurrentContext(fs.getAbfsStore().getClient(), abfsConfiguration); intercept = abfsClient.getIntercept(); Assertions.assertThat(intercept) .describedAs("AbfsNoOpThrottlingIntercept instance expected") @@ -114,7 +114,7 @@ public void testThrottlingIntercept() throws Exception { // On disabling throttling AbfsClientThrottlingIntercept object is returned AbfsConfiguration abfsConfiguration1 = new AbfsConfiguration(configuration, "dummy1.dfs.core.windows.net"); - AbfsClient abfsClient1 = TestAbfsClient.createTestClientFromCurrentContext(fs.getAbfsStore().getClient(), abfsConfiguration1); + AbfsClient abfsClient1 = ITestAbfsClient.createTestClientFromCurrentContext(fs.getAbfsStore().getClient(), abfsConfiguration1); intercept = abfsClient1.getIntercept(); Assertions.assertThat(intercept) .describedAs("AbfsClientThrottlingIntercept instance expected") From 51bdeceebda701df58ca30e7da7303bcab763514 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Thu, 16 Mar 2023 08:47:08 -0700 Subject: [PATCH 49/53] String fix --- .../fs/azurebfs/services/AbfsClient.java | 2 +- .../AbfsClientThrottlingIntercept.java | 2 +- .../azurebfs/services/AbfsRestOperation.java | 25 ++++++++----------- .../fs/azurebfs/services/ITestAbfsClient.java | 11 +++++--- .../services/ITestAbfsRestOperation.java | 4 +-- 5 files changed, 23 insertions(+), 21 deletions(-) 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 8648cc3e187a6..36db7e4ea100a 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 @@ -707,7 +707,7 @@ public AbfsRestOperation append(final String path, final byte[] buffer, */ int responseStatusCode = ((AbfsRestOperationException) e).getStatusCode(); if (checkUserError(responseStatusCode) && reqParams.isExpectHeaderEnabled()) { - LOG.debug("User error, retrying without 100 continue enabled for the given path " + path); + LOG.debug("User error, retrying without 100 continue enabled for the given path {} " , path); reqParams.setExpectHeaderEnabled(false); return this.append(path, buffer, reqParams, cachedSasToken, tracingContext); 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 b66c8f0893066..2522e1107e8be 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 @@ -160,7 +160,7 @@ public void updateMetrics(AbfsRestOperationType operationType, throttling but there were some expectedBytesToBeSent. */ if (updateBytesTransferred(isThrottledOperation, abfsHttpOperation)) { - LOG.debug("Updating metrics due to throttling for path " + abfsHttpOperation.getConnUrl().getPath()); + LOG.debug("Updating metrics due to throttling for path {} " , abfsHttpOperation.getConnUrl().getPath()); contentLength = abfsHttpOperation.getExpectedBytesToBeSent(); } } 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 872f8fa529352..085512993d273 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; @@ -249,19 +250,6 @@ private void completeExecute(TracingContext tracingContext) LOG.trace("{} REST operation complete", operationType); } - /** - * Returns a new object of AbfsHttpOperation. - * @param url The url for the operation. - * @param method The http method. - * @param requestHeaders The request headers for the operation. - * @return AbfsHttpOperation object. - * @throws IOException - */ - AbfsHttpOperation getHttpOperation(final URL url, final String method, - final List requestHeaders) throws IOException { - return new AbfsHttpOperation(url, method, requestHeaders); - } - /** * Executes a single HTTP operation to complete the REST operation. If it * fails, there may be a retry. The retryCount is incremented with each @@ -273,7 +261,7 @@ private boolean executeHttpOperation(final int retryCount, try { // initialize the HTTP request and open the connection - httpOperation = getHttpOperation(url, method, requestHeaders); + httpOperation = createHttpOperation(); incrementCounter(AbfsStatistic.CONNECTIONS_MADE, 1); tracingContext.constructHeader(httpOperation); @@ -372,6 +360,15 @@ private boolean executeHttpOperation(final int retryCount, return true; } + /** + * 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); + } + /** * Incrementing Abfs counters with a long value. * diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java index bcad51ec233ae..1910d75cf8b67 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java @@ -27,6 +27,7 @@ import java.util.Random; import java.util.regex.Pattern; +import org.assertj.core.api.AbstractBooleanAssert; import org.assertj.core.api.Assertions; import org.junit.Test; import org.mockito.Mockito; @@ -551,7 +552,7 @@ public void testExpectHundredContinue() throws Exception { // Sets the httpOperation for the rest operation. Mockito.doReturn(abfsHttpOperation) .when(op) - .getHttpOperation(Mockito.any(), Mockito.any(), Mockito.any()); + .createHttpOperation(); // Mock the restOperation for the client. Mockito.doReturn(op) @@ -566,7 +567,9 @@ public void testExpectHundredContinue() throws Exception { TracingHeaderFormat.ALL_ID_FORMAT, null)); // Check that expect header is enabled before the append call. - assertTrue(appendRequestParameters.isExpectHeaderEnabled()); + Assertions.assertThat(appendRequestParameters.isExpectHeaderEnabled()) + .describedAs("The expect header is not true before the append call") + .isTrue(); intercept(AzureBlobFileSystemException.class, () -> testClient.append(finalTestPath, buffer, appendRequestParameters, null, tracingContext)); @@ -577,6 +580,8 @@ public void testExpectHundredContinue() throws Exception { .isEqualTo(0); // Verify that the same request was retried with expect header disabled. - assertFalse(appendRequestParameters.isExpectHeaderEnabled()); + Assertions.assertThat(appendRequestParameters.isExpectHeaderEnabled()) + .describedAs("The expect header is not false") + .isFalse(); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java index 95cde41579ce7..6ffe2e2773bbf 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java @@ -257,7 +257,7 @@ public void write(final int i) throws IOException { // Sets the httpOperation for the rest operation. Mockito.doReturn(abfsHttpOperation) .when(op) - .getHttpOperation(Mockito.any(), Mockito.any(), Mockito.any()); + .createHttpOperation(); return op; } @@ -285,7 +285,7 @@ void assertTraceContextState(int retryCount, int assertRetryCount, int bytesSent public void testExpectHundredContinue() throws Exception { // Gets the AbfsRestOperation. AbfsRestOperation op = getRestOperation(); - AbfsHttpOperation httpOperation = op.getHttpOperation(Mockito.any(), Mockito.any(), Mockito.any()); + AbfsHttpOperation httpOperation = op.createHttpOperation(); TracingContext tracingContext = Mockito.spy(new TracingContext("abcd", "abcde", FSOperationType.APPEND, From c3268dc17919213e019686b82ab7727a80bc22aa Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Thu, 16 Mar 2023 21:23:31 +0530 Subject: [PATCH 50/53] Remove unused imports --- .../org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java index 1910d75cf8b67..c031e5daa6c44 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java @@ -27,7 +27,6 @@ import java.util.Random; import java.util.regex.Pattern; -import org.assertj.core.api.AbstractBooleanAssert; import org.assertj.core.api.Assertions; import org.junit.Test; import org.mockito.Mockito; From 675687c772d357e658832fc081f782e42c85d5d1 Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Thu, 16 Mar 2023 23:03:35 +0530 Subject: [PATCH 51/53] Import fix --- .../apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java | 1 - 1 file changed, 1 deletion(-) 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 362e0d1213fd5..a9a72635422bb 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,7 +25,6 @@ import java.net.UnknownHostException; import java.util.List; -import org.apache.hadoop.classification.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From 99a93773990cff08c97f1dfd86b1f41a8697cccc Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Thu, 16 Mar 2023 10:45:44 -0700 Subject: [PATCH 52/53] Checkstyle fixes --- .../hadoop-azure/src/config/checkstyle-suppressions.xml | 2 +- .../java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java | 2 +- .../fs/azurebfs/services/AbfsClientThrottlingIntercept.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml index 231a5b5ea149d..2065746b76611 100644 --- a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml +++ b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml @@ -49,7 +49,7 @@ + files="org[\\/]apache[\\/]hadoop[\\/]fs[\\/]azurebfs[\\/]services[\\/]ITestAbfsRestOperation.java"/> 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 36db7e4ea100a..bafeb2407eee4 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 @@ -707,7 +707,7 @@ public AbfsRestOperation append(final String path, final byte[] buffer, */ int responseStatusCode = ((AbfsRestOperationException) e).getStatusCode(); if (checkUserError(responseStatusCode) && reqParams.isExpectHeaderEnabled()) { - LOG.debug("User error, retrying without 100 continue enabled for the given path {} " , path); + LOG.debug("User error, retrying without 100 continue enabled for the given path {}", path); reqParams.setExpectHeaderEnabled(false); return this.append(path, buffer, reqParams, cachedSasToken, tracingContext); 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 2522e1107e8be..3bb225d4be862 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 @@ -160,7 +160,7 @@ public void updateMetrics(AbfsRestOperationType operationType, throttling but there were some expectedBytesToBeSent. */ if (updateBytesTransferred(isThrottledOperation, abfsHttpOperation)) { - LOG.debug("Updating metrics due to throttling for path {} " , abfsHttpOperation.getConnUrl().getPath()); + LOG.debug("Updating metrics due to throttling for path {}", abfsHttpOperation.getConnUrl().getPath()); contentLength = abfsHttpOperation.getExpectedBytesToBeSent(); } } From e210b043c28446b11e23fc7285ff62f9f56a34ed Mon Sep 17 00:00:00 2001 From: Anmol Asrani Date: Thu, 16 Mar 2023 20:47:50 -0700 Subject: [PATCH 53/53] Build fixed --- .../java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 bafeb2407eee4..2c367333300b6 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 @@ -698,7 +698,7 @@ public AbfsRestOperation append(final String path, final byte[] buffer, } catch (AzureBlobFileSystemException e) { /* If the http response code indicates a user error we retry - the same append request with expect header disabled. + 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