Browse Source

HADOOP-19120. ApacheHttpClient adaptation in ABFS. (#6633)

Apache httpclient 4.5.x is the new default implementation of http connections;
this supports a large configurable pool of connections along with
the ability to limit their lifespan.

The networking library can be chosen using the configuration
option fs.azure.networking.library

The supported values are
- APACHE_HTTP_CLIENT : Use Apache HttpClient [Default]
- JDK_HTTP_URL_CONNECTION : Use JDK networking library

Important: unless the networking library is switched back to
the JDK, the apache httpcore and httpclient must be on the classpath

Contributed by Pranav Saxena
Pranav Saxena 9 months ago
parent
commit
071440c924
49 changed files with 3912 additions and 420 deletions
  1. 39 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ClosedIOException.java
  2. 44 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java
  3. 2 1
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
  4. 3 3
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
  5. 11 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
  6. 12 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
  7. 12 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
  8. 24 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpOperationType.java
  9. 34 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsApacheHttpExpect100Exception.java
  10. 40 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/HttpResponseException.java
  11. 394 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsAHCHttpOperation.java
  12. 144 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsApacheHttpClient.java
  13. 40 4
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
  14. 2 1
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java
  15. 207 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConnectionManager.java
  16. 45 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientConnectionFactory.java
  17. 324 212
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
  18. 300 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsJdkHttpOperation.java
  19. 240 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsManagedApacheHttpConnection.java
  20. 76 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsManagedHttpClientContext.java
  21. 109 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsManagedHttpRequestExecutor.java
  22. 7 1
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsNoOpThrottlingIntercept.java
  23. 81 21
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java
  24. 3 2
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingIntercept.java
  25. 306 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/KeepAliveCache.java
  26. 8 9
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java
  27. 1 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java
  28. 39 0
      hadoop-tools/hadoop-azure/src/site/markdown/abfs.md
  29. 1 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java
  30. 57 7
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java
  31. 5 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java
  32. 1 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java
  33. 2 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java
  34. 10 7
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java
  35. 23 2
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java
  36. 11 3
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java
  37. 5 4
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java
  38. 4 4
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java
  39. 87 43
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java
  40. 406 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpClientRequestExecutor.java
  41. 38 8
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java
  42. 90 48
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java
  43. 63 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestApacheClientConnectionPool.java
  44. 18 18
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsPerfTracker.java
  45. 16 11
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java
  46. 11 4
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperationMockFailures.java
  47. 290 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestApacheClientConnectionPool.java
  48. 226 0
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestApacheHttpClientFallback.java
  49. 1 1
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java

+ 39 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ClosedIOException.java

@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Exception to denote if the underlying stream, cache or other closable resource
+ * is closed.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ClosedIOException extends PathIOException {
+
+  /**
+   * Appends the custom error-message to the default error message.
+   * @param path path that encountered the closed resource.
+   * @param message custom error message.
+   */
+  public ClosedIOException(String path, String message) {
+    super(path, message);
+  }
+}

+ 44 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java

@@ -23,6 +23,7 @@ import java.lang.reflect.Field;
 
 import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.fs.azurebfs.services.FixedSASTokenProvider;
+import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
 import org.apache.hadoop.util.Preconditions;
 
 import org.apache.commons.lang3.StringUtils;
@@ -369,6 +370,20 @@ public class AbfsConfiguration{
       FS_AZURE_ENABLE_PAGINATED_DELETE, DefaultValue = DEFAULT_ENABLE_PAGINATED_DELETE)
   private boolean isPaginatedDeleteEnabled;
 
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey =
+      FS_AZURE_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES, DefaultValue = DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES)
+  private int maxApacheHttpClientIoExceptionsRetries;
+
+  /**
+   * Max idle TTL configuration for connection given in
+   * {@value org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys#FS_AZURE_APACHE_HTTP_CLIENT_IDLE_CONNECTION_TTL}
+   * with default of
+   * {@value org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations#DEFAULT_HTTP_CLIENT_CONN_MAX_IDLE_TIME}
+   */
+  @LongConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_APACHE_HTTP_CLIENT_IDLE_CONNECTION_TTL,
+      DefaultValue = DEFAULT_HTTP_CLIENT_CONN_MAX_IDLE_TIME)
+  private long maxApacheHttpClientConnectionIdleTime;
+
   private String clientProvidedEncryptionKey;
   private String clientProvidedEncryptionKeySHA;
 
@@ -470,6 +485,17 @@ public class AbfsConfiguration{
     return rawConfig.getLong(accountConf(key), rawConfig.getLong(key, defaultValue));
   }
 
+  /**
+   * Returns the account-specific value if it exists, then looks for an
+   * account-agnostic value, and finally tries the default value.
+   * @param key Account-agnostic configuration key
+   * @param defaultValue Value returned if none is configured
+   * @return value if one exists, else the default value
+   */
+  public int getInt(String key, int defaultValue) {
+    return rawConfig.getInt(accountConf(key), rawConfig.getInt(key, defaultValue));
+  }
+
   /**
    * Returns the account-specific password in string form if it exists, then
    * looks for an account-agnostic value.
@@ -848,6 +874,24 @@ public class AbfsConfiguration{
     return getEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, DEFAULT_FS_AZURE_SSL_CHANNEL_MODE);
   }
 
+  /**
+   * @return Config to select netlib for server communication.
+   */
+  public HttpOperationType getPreferredHttpOperationType() {
+    return getEnum(FS_AZURE_NETWORKING_LIBRARY, DEFAULT_NETWORKING_LIBRARY);
+  }
+
+  public int getMaxApacheHttpClientIoExceptionsRetries() {
+    return maxApacheHttpClientIoExceptionsRetries;
+  }
+
+  /**
+   * @return {@link #maxApacheHttpClientConnectionIdleTime}.
+   */
+  public long getMaxApacheHttpClientConnectionIdleTime() {
+    return maxApacheHttpClientConnectionIdleTime;
+  }
+
   /**
    * Enum config to allow user to pick format of x-ms-client-request-id header
    * @return tracingContextFormat config if valid, else default ALL_ID_FORMAT

+ 2 - 1
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java

@@ -740,7 +740,8 @@ public class AzureBlobFileSystem extends FileSystem
               IOSTATISTICS_LOGGING_LEVEL_DEFAULT);
       logIOStatisticsAtLevel(LOG, iostatisticsLoggingLevel, getIOStatistics());
     }
-    IOUtils.cleanupWithLogger(LOG, abfsStore, delegationTokenManager);
+    IOUtils.cleanupWithLogger(LOG, abfsStore, delegationTokenManager,
+        getAbfsClient());
     this.isClosed = true;
     if (LOG.isDebugEnabled()) {
       LOG.debug("Closing Abfs: {}", toString());

+ 3 - 3
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java

@@ -55,6 +55,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
 import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider;
 import org.apache.hadoop.fs.azurebfs.security.ContextProviderEncryptionAdapter;
 import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
@@ -106,7 +107,6 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsClientContext;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClientContextBuilder;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClientRenameResult;
 import org.apache.hadoop.fs.azurebfs.services.AbfsCounters;
-import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
 import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
 import org.apache.hadoop.fs.azurebfs.services.AbfsInputStreamContext;
 import org.apache.hadoop.fs.azurebfs.services.AbfsInputStreamStatisticsImpl;
@@ -694,7 +694,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
           populateAbfsOutputStreamContext(
               isAppendBlob,
               lease,
-              client,
+              getClient(),
               statistics,
               relativePath,
               0,
@@ -933,7 +933,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport {
       perfInfo.registerSuccess(true);
 
       // Add statistics for InputStream
-      return new AbfsInputStream(client, statistics, relativePath,
+      return new AbfsInputStream(getClient(), statistics, relativePath,
           contentLength, populateAbfsInputStreamContext(
           parameters.map(OpenFileParameters::getOptions),
           contextEncryptionAdapter),

+ 11 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java

@@ -199,5 +199,16 @@ public final class AbfsHttpConstants {
           + "non-hierarchical-namespace account:"
           + CPK_CONFIG_LIST;
 
+  /**
+   * System property that define maximum number of cached-connection per fileSystem for
+   * ApacheHttpClient. JDK network library uses the same property to define maximum
+   * number of cached-connections at JVM level.
+   */
+  public static final String HTTP_MAX_CONN_SYS_PROP = "http.maxConnections";
+  public static final String JDK_IMPL = "JDK";
+  public static final String APACHE_IMPL = "Apache";
+  public static final String JDK_FALLBACK = "JDK_fallback";
+  public static final String KEEP_ALIVE_CACHE_CLOSED = "KeepAliveCache is closed";
+
   private AbfsHttpConstants() {}
 }

+ 12 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java

@@ -314,5 +314,17 @@ public final class ConfigurationKeys {
    * @see FileSystem#openFile(org.apache.hadoop.fs.Path)
    */
   public static final String FS_AZURE_BUFFERED_PREAD_DISABLE = "fs.azure.buffered.pread.disable";
+  /**Defines what network library to use for server IO calls: {@value}*/
+  public static final String FS_AZURE_NETWORKING_LIBRARY = "fs.azure.networking.library";
+  /**
+   * Maximum number of IOExceptions retries for a single server call on ApacheHttpClient.
+   * Breach of this count would turn off future uses of the ApacheHttpClient library
+   * in the JVM lifecycle: {@value}
+   */
+  public static final String FS_AZURE_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES = "fs.azure.apache.http.client.max.io.exception.retries";
+  /**Maximum ApacheHttpClient-connection cache size at filesystem level: {@value}*/
+  public static final String FS_AZURE_APACHE_HTTP_CLIENT_MAX_CACHE_CONNECTION_SIZE = "fs.azure.apache.http.client.max.cache.connection.size";
+  /**Maximum idle time for a ApacheHttpClient-connection: {@value}*/
+  public static final String FS_AZURE_APACHE_HTTP_CLIENT_IDLE_CONNECTION_TTL = "fs.azure.apache.http.client.idle.connection.ttl";
   private ConfigurationKeys() {}
 }

+ 12 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java

@@ -161,5 +161,17 @@ public final class FileSystemConfigurations {
    */
   public static final int RATE_LIMIT_DEFAULT = 1_000;
 
+  public static final int ZERO = 0;
+  public static final int HUNDRED = 100;
+  public static final long THOUSAND = 1000L;
+
+  public static final HttpOperationType DEFAULT_NETWORKING_LIBRARY
+      = HttpOperationType.APACHE_HTTP_CLIENT;
+
+  public static final int DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES = 3;
+
+  public static final long DEFAULT_HTTP_CLIENT_CONN_MAX_IDLE_TIME = 5_000L;
+
+  public static final int DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS = 5;
   private FileSystemConfigurations() {}
 }

+ 24 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpOperationType.java

@@ -0,0 +1,24 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.constants;
+
+public enum HttpOperationType {
+  JDK_HTTP_URL_CONNECTION,
+  APACHE_HTTP_CLIENT;
+}

+ 34 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsApacheHttpExpect100Exception.java

@@ -0,0 +1,34 @@
+/**
+ * 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.contracts.exceptions;
+
+import org.apache.http.HttpResponse;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EXPECT_100_JDK_ERROR;
+
+/**
+ * Exception that marks expect100 handshake error. This exception is thrown when
+ * the expect100 handshake fails with ADLS server sending 4xx or 5xx status code.
+ */
+public class AbfsApacheHttpExpect100Exception extends HttpResponseException {
+
+  public AbfsApacheHttpExpect100Exception(final HttpResponse httpResponse) {
+    super(EXPECT_100_JDK_ERROR, httpResponse);
+  }
+}

+ 40 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/HttpResponseException.java

@@ -0,0 +1,40 @@
+/**
+ * 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.contracts.exceptions;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import org.apache.http.HttpResponse;
+
+/**
+ * Encapsulates an exception thrown from ApacheHttpClient response parsing.
+ */
+public class HttpResponseException extends IOException {
+  private final HttpResponse httpResponse;
+  public HttpResponseException(final String s, final HttpResponse httpResponse) {
+    super(s);
+    Objects.requireNonNull(httpResponse, "httpResponse should be non-null");
+    this.httpResponse = httpResponse;
+  }
+
+  public HttpResponse getHttpResponse() {
+    return httpResponse;
+  }
+}

+ 394 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsAHCHttpOperation.java

@@ -0,0 +1,394 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsApacheHttpExpect100Exception;
+import org.apache.http.Header;
+import org.apache.http.HttpEntity;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpDelete;
+import org.apache.http.client.methods.HttpEntityEnclosingRequestBase;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpHead;
+import org.apache.http.client.methods.HttpPatch;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.methods.HttpPut;
+import org.apache.http.client.methods.HttpRequestBase;
+import org.apache.http.entity.ByteArrayEntity;
+import org.apache.http.util.EntityUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APACHE_IMPL;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_DELETE;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_GET;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_HEAD;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PATCH;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_POST;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT;
+import static org.apache.http.entity.ContentType.TEXT_PLAIN;
+
+/**
+ * Implementation of {@link AbfsHttpOperation} for orchestrating server calls using
+ * Apache Http Client.
+ */
+public class AbfsAHCHttpOperation extends AbfsHttpOperation {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbfsAHCHttpOperation.class);
+
+  /**
+   * Request object for network call over ApacheHttpClient.
+   */
+  private final HttpRequestBase httpRequestBase;
+
+  /**
+   * Response object received from a server call over ApacheHttpClient.
+   */
+  private HttpResponse httpResponse;
+
+  /**
+   * Flag to indicate if the request is a payload request. HTTP methods PUT, POST,
+   * PATCH qualify for payload requests.
+   */
+  private final boolean isPayloadRequest;
+
+  /**
+   * ApacheHttpClient to make network calls.
+   */
+  private final AbfsApacheHttpClient abfsApacheHttpClient;
+
+  public AbfsAHCHttpOperation(final URL url,
+      final String method,
+      final List<AbfsHttpHeader> requestHeaders,
+      final Duration connectionTimeout,
+      final Duration readTimeout,
+      final AbfsApacheHttpClient abfsApacheHttpClient) throws IOException {
+    super(LOG, url, method, requestHeaders, connectionTimeout, readTimeout);
+    this.isPayloadRequest = HTTP_METHOD_PUT.equals(method)
+        || HTTP_METHOD_PATCH.equals(method)
+        || HTTP_METHOD_POST.equals(method);
+    this.abfsApacheHttpClient = abfsApacheHttpClient;
+    LOG.debug("Creating AbfsAHCHttpOperation for URL: {}, method: {}",
+        url, method);
+
+    final URI requestUri;
+    try {
+      requestUri = url.toURI();
+    } catch (URISyntaxException e) {
+      throw new IOException(e);
+    }
+    switch (getMethod()) {
+    case HTTP_METHOD_PUT:
+      httpRequestBase = new HttpPut(requestUri);
+      break;
+    case HTTP_METHOD_PATCH:
+      httpRequestBase = new HttpPatch(requestUri);
+      break;
+    case HTTP_METHOD_POST:
+      httpRequestBase = new HttpPost(requestUri);
+      break;
+    case HTTP_METHOD_GET:
+      httpRequestBase = new HttpGet(requestUri);
+      break;
+    case HTTP_METHOD_DELETE:
+      httpRequestBase = new HttpDelete(requestUri);
+      break;
+    case HTTP_METHOD_HEAD:
+      httpRequestBase = new HttpHead(requestUri);
+      break;
+    default:
+      /*
+       * This would not happen as the AbfsClient would always be sending valid
+       * method.
+       */
+      throw new PathIOException(getUrl().toString(),
+          "Unsupported HTTP method: " + getMethod());
+    }
+  }
+
+  /**
+   * @return AbfsManagedHttpClientContext instance that captures latencies at
+   * different phases of network call.
+   */
+  @VisibleForTesting
+  AbfsManagedHttpClientContext getHttpClientContext() {
+    return new AbfsManagedHttpClientContext();
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  protected InputStream getErrorStream() throws IOException {
+    HttpEntity entity = httpResponse.getEntity();
+    if (entity == null) {
+      return null;
+    }
+    return entity.getContent();
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  String getConnProperty(final String key) {
+    for (AbfsHttpHeader header : getRequestHeaders()) {
+      if (header.getName().equals(key)) {
+        return header.getValue();
+      }
+    }
+    return null;
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  URL getConnUrl() {
+    return getUrl();
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  Integer getConnResponseCode() throws IOException {
+    return getStatusCode();
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  String getConnResponseMessage() throws IOException {
+    return getStatusDescription();
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void processResponse(final byte[] buffer,
+      final int offset,
+      final int length) throws IOException {
+    try {
+      if (!isPayloadRequest) {
+        prepareRequest();
+        LOG.debug("Sending request: {}", httpRequestBase);
+        httpResponse = executeRequest();
+        LOG.debug("Request sent: {}; response {}", httpRequestBase,
+            httpResponse);
+      }
+      parseResponseHeaderAndBody(buffer, offset, length);
+    } finally {
+      if (httpResponse != null) {
+        try {
+          EntityUtils.consume(httpResponse.getEntity());
+        } finally {
+          if (httpResponse instanceof CloseableHttpResponse) {
+            ((CloseableHttpResponse) httpResponse).close();
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Parse response stream for headers and body.
+   *
+   * @param buffer byte array to store response body.
+   * @param offset offset in the buffer to start storing the response body.
+   * @param length length of the response body.
+   *
+   * @throws IOException network error while read response stream
+   */
+  @VisibleForTesting
+  void parseResponseHeaderAndBody(final byte[] buffer,
+      final int offset,
+      final int length) throws IOException {
+    setStatusCode(parseStatusCode(httpResponse));
+
+    setStatusDescription(httpResponse.getStatusLine().getReasonPhrase());
+    setRequestId();
+
+    // dump the headers
+    if (LOG.isDebugEnabled()) {
+      AbfsIoUtils.dumpHeadersToDebugLog("Request Headers",
+          getRequestProperties());
+    }
+    parseResponse(buffer, offset, length);
+  }
+
+  /**
+   * Parse status code from response
+   *
+   * @param httpResponse response object
+   * @return status code
+   */
+  @VisibleForTesting
+  int parseStatusCode(HttpResponse httpResponse) {
+    return httpResponse.getStatusLine().getStatusCode();
+  }
+
+  /**
+   * Execute network call for the request
+   *
+   * @return response object
+   * @throws IOException network error while executing the request
+   */
+  @VisibleForTesting
+  HttpResponse executeRequest() throws IOException {
+    AbfsManagedHttpClientContext abfsHttpClientContext
+        = getHttpClientContext();
+    try {
+      LOG.debug("Executing request: {}", httpRequestBase);
+      HttpResponse response = abfsApacheHttpClient.execute(httpRequestBase,
+          abfsHttpClientContext, getConnectionTimeout(), getReadTimeout());
+      setConnectionTimeMs(abfsHttpClientContext.getConnectTime());
+      setSendRequestTimeMs(abfsHttpClientContext.getSendTime());
+      setRecvResponseTimeMs(abfsHttpClientContext.getReadTime());
+      return response;
+    } catch (IOException e) {
+      LOG.debug("Failed to execute request: {}", httpRequestBase, e);
+      throw e;
+    }
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void setRequestProperty(final String key, final String value) {
+    List<AbfsHttpHeader> headers = getRequestHeaders();
+    if (headers != null) {
+      headers.add(new AbfsHttpHeader(key, value));
+    }
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  Map<String, List<String>> getRequestProperties() {
+    Map<String, List<String>> map = new HashMap<>();
+    for (AbfsHttpHeader header : getRequestHeaders()) {
+      map.put(header.getName(),
+          new ArrayList<String>() {{
+            add(header.getValue());
+          }});
+    }
+    return map;
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public String getResponseHeader(final String headerName) {
+    if (httpResponse == null) {
+      return null;
+    }
+    Header header = httpResponse.getFirstHeader(headerName);
+    if (header != null) {
+      return header.getValue();
+    }
+    return null;
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  protected InputStream getContentInputStream()
+      throws IOException {
+    if (httpResponse == null || httpResponse.getEntity() == null) {
+      return null;
+    }
+    return httpResponse.getEntity().getContent();
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void sendPayload(final byte[] buffer,
+      final int offset,
+      final int length)
+      throws IOException {
+    if (!isPayloadRequest) {
+      return;
+    }
+
+    setExpectedBytesToBeSent(length);
+    if (buffer != null) {
+      HttpEntity httpEntity = new ByteArrayEntity(buffer, offset, length,
+          TEXT_PLAIN);
+      ((HttpEntityEnclosingRequestBase) httpRequestBase).setEntity(
+          httpEntity);
+    }
+
+    prepareRequest();
+    try {
+      LOG.debug("Sending request: {}", httpRequestBase);
+      httpResponse = executeRequest();
+    } catch (AbfsApacheHttpExpect100Exception ex) {
+      LOG.debug(
+          "Getting output stream failed with expect header enabled, returning back."
+              + "Expect 100 assertion failed for uri {} with status code: {}",
+          getMaskedUrl(), parseStatusCode(ex.getHttpResponse()),
+          ex);
+      setConnectionDisconnectedOnError();
+      httpResponse = ex.getHttpResponse();
+    } catch (IOException ex) {
+      LOG.debug("Getting output stream failed for uri {}, exception: {}",
+          getMaskedUrl(), ex);
+      throw ex;
+    } finally {
+      if (httpResponse != null) {
+        LOG.debug("Request sent: {}; response {}", httpRequestBase,
+            httpResponse);
+      }
+      if (!isConnectionDisconnectedOnError()
+          && httpRequestBase instanceof HttpEntityEnclosingRequestBase) {
+        setBytesSent(length);
+      }
+    }
+  }
+
+  /**
+   * Sets the header on the request.
+   */
+  private void prepareRequest() {
+    for (AbfsHttpHeader header : getRequestHeaders()) {
+      httpRequestBase.setHeader(header.getName(), header.getValue());
+    }
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public String getRequestProperty(String name) {
+    for (AbfsHttpHeader header : getRequestHeaders()) {
+      if (header.getName().equals(name)) {
+        return header.getValue();
+      }
+    }
+    return EMPTY_STRING;
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public String getTracingContextSuffix() {
+    return APACHE_IMPL;
+  }
+}

+ 144 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsApacheHttpClient.java

@@ -0,0 +1,144 @@
+/**
+ * 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.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.config.RequestConfig;
+import org.apache.http.client.methods.HttpRequestBase;
+import org.apache.http.config.Registry;
+import org.apache.http.config.RegistryBuilder;
+import org.apache.http.conn.socket.ConnectionSocketFactory;
+import org.apache.http.conn.socket.PlainConnectionSocketFactory;
+import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClientBuilder;
+import org.apache.http.impl.client.HttpClients;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTP_SCHEME;
+import static org.apache.http.conn.ssl.SSLConnectionSocketFactory.getDefaultHostnameVerifier;
+
+/**
+ * Client for AzureBlobFileSystem to execute HTTP requests over ApacheHttpClient.
+ */
+final class AbfsApacheHttpClient implements Closeable {
+
+  /**
+   * ApacheHttpClient instance that executes HTTP request.
+   */
+  private final CloseableHttpClient httpClient;
+
+  /**
+   * Flag to indicate if the client is usable. This is a JVM level flag, state of
+   * this flag is shared across all instances of fileSystems. Once switched off,
+   * the ApacheHttpClient would not be used for whole JVM lifecycle.
+   */
+  private static boolean usable = true;
+
+  /**
+   * Registers the switch off of ApacheHttpClient for all future use in the JVM.
+   */
+  static void registerFallback() {
+    usable = false;
+  }
+
+  /**
+   * @return if ApacheHttpClient is usable.
+   */
+  static boolean usable() {
+    return usable;
+  }
+
+  AbfsApacheHttpClient(DelegatingSSLSocketFactory delegatingSSLSocketFactory,
+      final int readTimeout, final KeepAliveCache keepAliveCache) {
+    final AbfsConnectionManager connMgr = new AbfsConnectionManager(
+        createSocketFactoryRegistry(
+            new SSLConnectionSocketFactory(delegatingSSLSocketFactory,
+                getDefaultHostnameVerifier())),
+        new AbfsHttpClientConnectionFactory(), keepAliveCache);
+    final HttpClientBuilder builder = HttpClients.custom();
+    builder.setConnectionManager(connMgr)
+        .setRequestExecutor(new AbfsManagedHttpRequestExecutor(readTimeout))
+        .disableContentCompression()
+        .disableRedirectHandling()
+        .disableAutomaticRetries()
+        /*
+         * To prevent the read of system property http.agent. The agent is set
+         * in request headers by AbfsClient. System property read is an
+         * overhead.
+         */
+        .setUserAgent(EMPTY_STRING);
+    httpClient = builder.build();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (httpClient != null) {
+      httpClient.close();
+    }
+  }
+
+  /**
+   * Executes the HTTP request.
+   *
+   * @param httpRequest HTTP request to execute.
+   * @param abfsHttpClientContext HttpClient context.
+   * @param connectTimeout Connection timeout.
+   * @param readTimeout Read timeout.
+   *
+   * @return HTTP response.
+   * @throws IOException network error.
+   */
+  public HttpResponse execute(HttpRequestBase httpRequest,
+      final AbfsManagedHttpClientContext abfsHttpClientContext,
+      final int connectTimeout,
+      final int readTimeout) throws IOException {
+    RequestConfig.Builder requestConfigBuilder = RequestConfig
+        .custom()
+        .setConnectTimeout(connectTimeout)
+        .setSocketTimeout(readTimeout);
+    httpRequest.setConfig(requestConfigBuilder.build());
+    return httpClient.execute(httpRequest, abfsHttpClientContext);
+  }
+
+  /**
+   * Creates the socket factory registry for HTTP and HTTPS.
+   *
+   * @param sslSocketFactory SSL socket factory.
+   * @return Socket factory registry.
+   */
+  private Registry<ConnectionSocketFactory> createSocketFactoryRegistry(
+      ConnectionSocketFactory sslSocketFactory) {
+    if (sslSocketFactory == null) {
+      return RegistryBuilder.<ConnectionSocketFactory>create()
+          .register(HTTP_SCHEME,
+              PlainConnectionSocketFactory.getSocketFactory())
+          .build();
+    }
+    return RegistryBuilder.<ConnectionSocketFactory>create()
+        .register(HTTP_SCHEME, PlainConnectionSocketFactory.getSocketFactory())
+        .register(HTTPS_SCHEME, sslSocketFactory)
+        .build();
+  }
+}

+ 40 - 4
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java

@@ -37,6 +37,7 @@ import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsInvalidChecksumException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsDriverException;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
@@ -122,6 +123,10 @@ public class AbfsClient implements Closeable {
 
   private boolean renameResilience;
 
+  private KeepAliveCache keepAliveCache;
+
+  private AbfsApacheHttpClient abfsApacheHttpClient;
+
   /**
    * logging the rename failure if metadata is in an incomplete state.
    */
@@ -170,6 +175,15 @@ public class AbfsClient implements Closeable {
             + "{}", e.getMessage());
       }
     }
+    if (abfsConfiguration.getPreferredHttpOperationType()
+        == HttpOperationType.APACHE_HTTP_CLIENT) {
+      keepAliveCache = new KeepAliveCache(abfsConfiguration);
+
+      abfsApacheHttpClient = new AbfsApacheHttpClient(
+          DelegatingSSLSocketFactory.getDefaultFactory(),
+          abfsConfiguration.getHttpReadTimeout(),
+          keepAliveCache);
+    }
 
     this.userAgent = initializeUserAgent(abfsConfiguration, sslProviderName);
     this.abfsPerfTracker = abfsClientContext.getAbfsPerfTracker();
@@ -205,6 +219,12 @@ public class AbfsClient implements Closeable {
 
   @Override
   public void close() throws IOException {
+    if (keepAliveCache != null) {
+      keepAliveCache.close();
+    }
+    if (abfsApacheHttpClient != null) {
+      abfsApacheHttpClient.close();
+    }
     if (tokenProvider instanceof Closeable) {
       IOUtils.cleanupWithLogger(LOG,
           (Closeable) tokenProvider);
@@ -1161,7 +1181,8 @@ public class AbfsClient implements Closeable {
             this,
             HTTP_METHOD_DELETE,
             url,
-            requestHeaders);
+            requestHeaders,
+            abfsConfiguration);
     try {
     op.execute(tracingContext);
     } catch (AzureBlobFileSystemException e) {
@@ -1539,6 +1560,9 @@ public class AbfsClient implements Closeable {
       sb.append(HUNDRED_CONTINUE);
       sb.append(SEMICOLON);
     }
+    sb.append(SINGLE_WHITE_SPACE)
+        .append(abfsConfiguration.getPreferredHttpOperationType())
+        .append(SEMICOLON);
 
     sb.append(SINGLE_WHITE_SPACE);
     sb.append(abfsConfiguration.getClusterName());
@@ -1745,7 +1769,8 @@ public class AbfsClient implements Closeable {
         buffer,
         bufferOffset,
         bufferLength,
-        sasTokenForReuse);
+        sasTokenForReuse,
+        abfsConfiguration);
   }
 
   /**
@@ -1766,7 +1791,8 @@ public class AbfsClient implements Closeable {
         this,
         httpMethod,
         url,
-        requestHeaders
+        requestHeaders,
+        abfsConfiguration
     );
   }
 
@@ -1790,6 +1816,16 @@ public class AbfsClient implements Closeable {
         this,
         httpMethod,
         url,
-        requestHeaders, sasTokenForReuse);
+        requestHeaders, sasTokenForReuse, abfsConfiguration);
+  }
+
+  @VisibleForTesting
+  AbfsApacheHttpClient getAbfsApacheHttpClient() {
+    return abfsApacheHttpClient;
+  }
+
+  @VisibleForTesting
+  KeepAliveCache getKeepAliveCache() {
+    return keepAliveCache;
   }
 }

+ 2 - 1
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java

@@ -130,6 +130,7 @@ public final class AbfsClientThrottlingIntercept implements AbfsThrottlingInterc
 
   /**
    * Updates the metrics for successful and failed read and write operations.
+   *
    * @param operationType Only applicable for read and write operations.
    * @param abfsHttpOperation Used for status code and data transferred.
    */
@@ -170,7 +171,7 @@ public final class AbfsClientThrottlingIntercept implements AbfsThrottlingInterc
         }
         break;
       case ReadFile:
-        String range = abfsHttpOperation.getConnection().getRequestProperty(HttpHeaderConfigurations.RANGE);
+        String range = abfsHttpOperation.getRequestProperty(HttpHeaderConfigurations.RANGE);
         contentLength = getContentLengthIfKnown(range);
         if (contentLength > 0) {
           readThrottler.addBytesTransferred(contentLength,

+ 207 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsConnectionManager.java

@@ -0,0 +1,207 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.http.HttpClientConnection;
+import org.apache.http.config.Registry;
+import org.apache.http.config.SocketConfig;
+import org.apache.http.conn.ConnectionPoolTimeoutException;
+import org.apache.http.conn.ConnectionRequest;
+import org.apache.http.conn.HttpClientConnectionManager;
+import org.apache.http.conn.HttpClientConnectionOperator;
+import org.apache.http.conn.ManagedHttpClientConnection;
+import org.apache.http.conn.routing.HttpRoute;
+import org.apache.http.conn.socket.ConnectionSocketFactory;
+import org.apache.http.impl.conn.DefaultHttpClientConnectionOperator;
+import org.apache.http.impl.conn.ManagedHttpClientConnectionFactory;
+import org.apache.http.protocol.HttpContext;
+
+/**
+ * AbfsConnectionManager is a custom implementation of {@code HttpClientConnectionManager}.
+ * This implementation manages connection-pooling heuristics and custom implementation
+ * of {@link ManagedHttpClientConnectionFactory}.
+ */
+class AbfsConnectionManager implements HttpClientConnectionManager {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbfsConnectionManager.class);
+
+  /**
+   * Connection pool for the ABFS managed connections.
+   */
+  private final KeepAliveCache kac;
+
+  /**
+   * Factory to create new connections.
+   */
+  private final AbfsHttpClientConnectionFactory httpConnectionFactory;
+
+  /**
+   * Operator to manage the network connection state of ABFS managed connections.
+   */
+  private final HttpClientConnectionOperator connectionOperator;
+
+  AbfsConnectionManager(Registry<ConnectionSocketFactory> socketFactoryRegistry,
+      AbfsHttpClientConnectionFactory connectionFactory, KeepAliveCache kac) {
+    this.httpConnectionFactory = connectionFactory;
+    this.kac = kac;
+    this.connectionOperator = new DefaultHttpClientConnectionOperator(
+        socketFactoryRegistry, null, null);
+  }
+
+  /**
+   * Returns a custom implementation of connection request for the given route.
+   * The implementation would return a connection from the {@link KeepAliveCache} if available,
+   * else it would create a new non-connected {@link AbfsManagedApacheHttpConnection}.
+   */
+  @Override
+  public ConnectionRequest requestConnection(final HttpRoute route,
+      final Object state) {
+    return new ConnectionRequest() {
+
+      /**
+       * Synchronously gets a connection from the {@link KeepAliveCache} or
+       * creates a new un-connected instance of {@link AbfsManagedApacheHttpConnection}.
+       */
+      @Override
+      public HttpClientConnection get(final long timeout,
+          final TimeUnit timeUnit)
+          throws InterruptedException, ExecutionException,
+          ConnectionPoolTimeoutException {
+        String requestId = UUID.randomUUID().toString();
+        logDebug("Connection requested for request {}", requestId);
+        try {
+          HttpClientConnection clientConn = kac.get();
+          if (clientConn != null) {
+            logDebug("Connection retrieved from KAC: {} for requestId: {}",
+                clientConn, requestId);
+            return clientConn;
+          }
+          logDebug("Creating new connection for requestId: {}", requestId);
+          ManagedHttpClientConnection conn = httpConnectionFactory.create(route,
+              null);
+          logDebug("Connection created: {} for requestId: {}", conn, requestId);
+          return conn;
+        } catch (IOException ex) {
+          throw new ExecutionException(ex);
+        }
+      }
+
+      @Override
+      public boolean cancel() {
+        return false;
+      }
+    };
+  }
+
+  /**
+   * Releases a connection for reuse. It can be reused only if validDuration is greater than 0.
+   * This method is called by {@link org.apache.http.impl.execchain} internal class `ConnectionHolder`.
+   * If it wants to reuse the connection, it will send a non-zero validDuration, else it will send 0.
+   * @param conn the connection to release
+   * @param newState the new state of the connection
+   * @param validDuration the duration for which the connection is valid
+   * @param timeUnit the time unit for the validDuration
+   */
+  @Override
+  public void releaseConnection(final HttpClientConnection conn,
+      final Object newState,
+      final long validDuration,
+      final TimeUnit timeUnit) {
+    if (validDuration == 0) {
+      return;
+    }
+    if (conn.isOpen() && conn instanceof AbfsManagedApacheHttpConnection) {
+      boolean connAddedInKac = kac.put(conn);
+      if (connAddedInKac) {
+        logDebug("Connection cached: {}", conn);
+      } else {
+        logDebug("Connection not cached, and is released: {}", conn);
+      }
+    }
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void connect(final HttpClientConnection conn,
+      final HttpRoute route,
+      final int connectTimeout,
+      final HttpContext context) throws IOException {
+    long start = System.currentTimeMillis();
+    logDebug("Connecting {} to {}", conn, route.getTargetHost());
+    connectionOperator.connect((AbfsManagedApacheHttpConnection) conn,
+        route.getTargetHost(), route.getLocalSocketAddress(),
+        connectTimeout, SocketConfig.DEFAULT, context);
+    logDebug("Connection established: {}", conn);
+    if (context instanceof AbfsManagedHttpClientContext) {
+      ((AbfsManagedHttpClientContext) context).setConnectTime(
+          System.currentTimeMillis() - start);
+    }
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void upgrade(final HttpClientConnection conn,
+      final HttpRoute route,
+      final HttpContext context) throws IOException {
+    connectionOperator.upgrade((AbfsManagedApacheHttpConnection) conn,
+        route.getTargetHost(), context);
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void routeComplete(final HttpClientConnection conn,
+      final HttpRoute route,
+      final HttpContext context) throws IOException {
+
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void closeIdleConnections(final long idletime,
+      final TimeUnit timeUnit) {
+    kac.evictIdleConnection();
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void closeExpiredConnections() {
+    kac.evictIdleConnection();
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void shutdown() {
+    kac.close();
+  }
+
+  private void logDebug(String message, Object... args) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(message, args);
+    }
+  }
+}

+ 45 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpClientConnectionFactory.java

@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.http.config.ConnectionConfig;
+import org.apache.http.conn.ManagedHttpClientConnection;
+import org.apache.http.conn.routing.HttpRoute;
+import org.apache.http.impl.conn.ManagedHttpClientConnectionFactory;
+
+/**
+ * Custom implementation of {@link ManagedHttpClientConnectionFactory} and overrides
+ * {@link ManagedHttpClientConnectionFactory#create(HttpRoute, ConnectionConfig)} to return
+ * {@link AbfsManagedApacheHttpConnection}.
+ */
+public class AbfsHttpClientConnectionFactory extends ManagedHttpClientConnectionFactory {
+
+  /**
+   * Creates a new {@link AbfsManagedApacheHttpConnection} instance which has to
+   * be connected.
+   * @param route route for which connection is required.
+   * @param config connection configuration.
+   * @return new {@link AbfsManagedApacheHttpConnection} instance.
+   */
+  @Override
+  public ManagedHttpClientConnection create(final HttpRoute route,
+      final ConnectionConfig config) {
+    return new AbfsManagedApacheHttpConnection(super.create(route, config), route);
+  }
+}

+ 324 - 212
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java

@@ -20,18 +20,12 @@ package org.apache.hadoop.fs.azurebfs.services;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.OutputStream;
 import java.net.HttpURLConnection;
-import java.net.ProtocolException;
 import java.net.URL;
+import java.time.Duration;
+import java.util.ArrayList;
 import java.util.List;
-
-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;
+import java.util.Map;
 
 import com.fasterxml.jackson.core.JsonFactory;
 import com.fasterxml.jackson.core.JsonParser;
@@ -40,37 +34,39 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
 import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
 import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsPerfLoggable;
 import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
-
-import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EXPECT_100_JDK_ERROR;
-import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE;
-import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT;
+import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
 
 /**
- * Represents an HTTP operation.
+ * Base Http operation class for orchestrating server IO calls. Child classes would
+ * define the certain orchestration implementation on the basis of network library used.
+ * <p>
+ * For JDK netlib usage, the child class would be {@link AbfsJdkHttpOperation}. <br>
+ * For ApacheHttpClient netlib usage, the child class would be {@link AbfsAHCHttpOperation}.
  */
-public class AbfsHttpOperation implements AbfsPerfLoggable {
-  private static final Logger LOG = LoggerFactory.getLogger(AbfsHttpOperation.class);
+public abstract class AbfsHttpOperation implements AbfsPerfLoggable {
+
+  private final Logger log;
 
   private static final int CLEAN_UP_BUFFER_SIZE = 64 * 1024;
 
   private static final int ONE_THOUSAND = 1000;
+
   private static final int ONE_MILLION = ONE_THOUSAND * ONE_THOUSAND;
 
   private final String method;
   private final URL url;
   private String maskedUrl;
   private String maskedEncodedUrl;
-
-  private HttpURLConnection connection;
   private int statusCode;
   private String statusDescription;
   private String storageErrorCode = "";
-  private String storageErrorMessage  = "";
-  private String requestId  = "";
+  private String storageErrorMessage = "";
+  private String requestId = "";
   private String expectedAppendPos = "";
   private ListResultSchema listResultSchema = null;
 
@@ -85,6 +81,23 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
   private boolean shouldMask = false;
   private boolean connectionDisconnectedOnError = false;
 
+  /**Request headers to be sent in the request.*/
+  private final List<AbfsHttpHeader> requestHeaders;
+
+  /**
+   * Timeout that defines maximum allowed connection establishment time for a request.
+   * Timeout is in milliseconds. Not all requests need to establish a new connection,
+   * it depends on the connection pooling-heuristic of the networking library.
+   */
+  private final int connectionTimeout;
+
+  /**
+   * Timeout in milliseconds that defines maximum allowed time to read the response.
+   * This timeout starts once request is sent. It includes server reponse time,
+   * network latency, and time to read the response.
+   */
+  private final int readTimeout;
+
   public static AbfsHttpOperation getAbfsHttpOperationWithFixedResult(
       final URL url,
       final String method,
@@ -94,6 +107,21 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
     return httpOp;
   }
 
+  public AbfsHttpOperation(
+      final Logger log,
+      final URL url,
+      final String method,
+      final List<AbfsHttpHeader> requestHeaders,
+      final Duration connectionTimeout,
+      final Duration readTimeout) {
+    this.log = log;
+    this.url = url;
+    this.method = method;
+    this.requestHeaders = requestHeaders;
+    this.connectionTimeout = (int) connectionTimeout.toMillis();
+    this.readTimeout = (int) readTimeout.toMillis();
+  }
+
   /**
    * Constructor for FixedResult instance, avoiding connection init.
    * @param url request url
@@ -103,13 +131,25 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
   protected AbfsHttpOperation(final URL url,
       final String method,
       final int httpStatus) {
+    this.log = LoggerFactory.getLogger(AbfsHttpOperation.class);
     this.url = url;
     this.method = method;
     this.statusCode = httpStatus;
+    this.requestHeaders = new ArrayList<>();
+    this.connectionTimeout = 0;
+    this.readTimeout = 0;
   }
 
-  protected  HttpURLConnection getConnection() {
-    return connection;
+  int getConnectionTimeout() {
+    return connectionTimeout;
+  }
+
+  int getReadTimeout() {
+    return readTimeout;
+  }
+
+  List<AbfsHttpHeader> getRequestHeaders() {
+    return requestHeaders;
   }
 
   public String getMethod() {
@@ -137,8 +177,7 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
   }
 
   public String getClientRequestId() {
-    return this.connection
-        .getRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID);
+    return getRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID);
   }
 
   public String getExpectedAppendPos() {
@@ -165,13 +204,21 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
     return bytesReceived;
   }
 
+  public URL getUrl() {
+    return url;
+  }
+
   public ListResultSchema getListResultSchema() {
     return listResultSchema;
   }
 
-  public String getResponseHeader(String httpHeader) {
-    return connection.getHeaderField(httpHeader);
-  }
+  /**
+   * Get response header value for the given headerKey.
+   *
+   * @param httpHeader header key.
+   * @return header value.
+   */
+  public abstract String getResponseHeader(String httpHeader);
 
   // Returns a trace message for the request
   @Override
@@ -235,6 +282,7 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
     return sb.toString();
   }
 
+  @VisibleForTesting
   public String getMaskedUrl() {
     if (!shouldMask) {
       return url.toString();
@@ -246,7 +294,7 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
     return maskedUrl;
   }
 
-  public String getMaskedEncodedUrl() {
+  public final String getMaskedEncodedUrl() {
     if (maskedEncodedUrl != null) {
       return maskedEncodedUrl;
     }
@@ -255,40 +303,6 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
   }
 
   /**
-   * Initializes a new HTTP request and opens the connection.
-   *
-   * @param url The full URL including query string parameters.
-   * @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE).
-   * @param requestHeaders The HTTP request headers.READ_TIMEOUT
-   * @param connectionTimeout The Connection Timeout value to be used while establishing http connection
-   * @param readTimeout The Read Timeout value to be used with http connection while making a request
-   * @throws IOException if an error occurs.
-   */
-  public AbfsHttpOperation(final URL url, final String method, final List<AbfsHttpHeader> requestHeaders,
-                           final int connectionTimeout, final int readTimeout)
-      throws IOException {
-    this.url = url;
-    this.method = method;
-
-    this.connection = openConnection();
-    if (this.connection instanceof HttpsURLConnection) {
-      HttpsURLConnection secureConn = (HttpsURLConnection) this.connection;
-      SSLSocketFactory sslSocketFactory = DelegatingSSLSocketFactory.getDefaultFactory();
-      if (sslSocketFactory != null) {
-        secureConn.setSSLSocketFactory(sslSocketFactory);
-      }
-    }
-
-    this.connection.setConnectTimeout(connectionTimeout);
-    this.connection.setReadTimeout(readTimeout);
-    this.connection.setRequestMethod(method);
-
-    for (AbfsHttpHeader header : requestHeaders) {
-      setRequestProperty(header.getName(), header.getValue());
-    }
-  }
-
-   /**
    * Sends the HTTP request.  Note that HttpUrlConnection requires that an
    * empty buffer be sent in order to set the "Content-Length: 0" header, which
    * is required by our endpoint.
@@ -299,74 +313,9 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
    *
    * @throws IOException if an error occurs.
    */
-  public void sendRequest(byte[] buffer, int offset, int length) throws IOException {
-    this.connection.setDoOutput(true);
-    this.connection.setFixedLengthStreamingMode(length);
-    if (buffer == null) {
-      // An empty buffer is sent to set the "Content-Length: 0" header, which
-      // is required by our endpoint.
-      buffer = new byte[]{};
-      offset = 0;
-      length = 0;
-    }
-
-    // send the request body
 
-    long startTime = 0;
-    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 = getConnOutputStream();
-      } catch (IOException e) {
-        connectionDisconnectedOnError = true;
-        /* If getOutputStream fails with an expect-100 exception , we return back
-           without throwing an exception to the caller. Else, we throw back the exception.
-         */
-        String expectHeader = getConnProperty(EXPECT);
-        if (expectHeader != null && expectHeader.equals(HUNDRED_CONTINUE)
-            && e instanceof ProtocolException
-            && EXPECT_100_JDK_ERROR.equals(e.getMessage())) {
-          LOG.debug("Getting output stream failed with expect header enabled, returning back ", e);
-          /*
-           * In case expect-100 assertion has failed, headers and inputStream should not
-           * be parsed. Reason being, conn.getHeaderField(), conn.getHeaderFields(),
-           * conn.getInputStream() will lead to repeated server call.
-           * ref: https://bugs.openjdk.org/browse/JDK-8314978.
-           * Reading conn.responseCode() and conn.getResponseMessage() is safe in
-           * case of Expect-100 error. Reason being, in JDK, it stores the responseCode
-           * in the HttpUrlConnection object before throwing exception to the caller.
-           */
-          this.statusCode = getConnResponseCode();
-          this.statusDescription = getConnResponseMessage();
-          return;
-        } else {
-          LOG.debug("Getting output stream failed without expect header enabled, throwing exception ", e);
-          throw e;
-        }
-      }
-      // 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();
-      }
-      this.sendRequestTimeMs = elapsedTimeMs(startTime);
-    }
-  }
+  public abstract void sendPayload(byte[] buffer, int offset, int length) throws
+      IOException;
 
   /**
    * Gets and processes the HTTP response.
@@ -377,35 +326,31 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
    *
    * @throws IOException if an error occurs.
    */
-  public void processResponse(final byte[] buffer, final int offset, final int length) throws IOException {
-    if (connectionDisconnectedOnError) {
-      LOG.debug("This connection was not successful or has been disconnected, "
-          + "hence not parsing headers and inputStream");
-      return;
-    }
-    processConnHeadersAndInputStreams(buffer, offset, length);
-  }
+  public abstract void processResponse(byte[] buffer,
+      int offset,
+      int length) throws IOException;
+
+  /**
+   * Set request header.
+   *
+   * @param key header key.
+   * @param value header value.
+   */
+  public abstract void setRequestProperty(String key, String value);
 
-  void processConnHeadersAndInputStreams(final byte[] buffer,
+  /**
+   * Parse response body from the connection.
+   *
+   * @param buffer byte array to store the response body.
+   * @param offset offset in the buffer.
+   * @param length length of the response body.
+   *
+   * @throws IOException if network error occurs while reading the response.
+   */
+  final void parseResponse(final byte[] buffer,
       final int offset,
       final int length) throws IOException {
-    // get the response
-    long startTime = 0;
-    startTime = System.nanoTime();
-
-    this.statusCode = getConnResponseCode();
-    this.recvResponseTimeMs = elapsedTimeMs(startTime);
-
-    this.statusDescription = getConnResponseMessage();
-
-    this.requestId = this.connection.getHeaderField(HttpHeaderConfigurations.X_MS_REQUEST_ID);
-    if (this.requestId == null) {
-      this.requestId = AbfsHttpConstants.EMPTY_STRING;
-    }
-    // dump the headers
-    AbfsIoUtils.dumpHeadersToDebugLog("Response Headers",
-        connection.getHeaderFields());
-
+    long startTime;
     if (AbfsHttpConstants.HTTP_METHOD_HEAD.equals(this.method)) {
       // If it is HEAD, and it is ERROR
       return;
@@ -416,12 +361,19 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
     if (statusCode >= HttpURLConnection.HTTP_BAD_REQUEST) {
       processStorageErrorResponse();
       this.recvResponseTimeMs += elapsedTimeMs(startTime);
-      this.bytesReceived = this.connection.getHeaderFieldLong(HttpHeaderConfigurations.CONTENT_LENGTH, 0);
+      String contentLength = getResponseHeader(
+          HttpHeaderConfigurations.CONTENT_LENGTH);
+      if (contentLength != null) {
+        this.bytesReceived = Long.parseLong(contentLength);
+      } else {
+        this.bytesReceived = 0L;
+      }
+
     } else {
       // consume the input stream to release resources
       int totalBytesRead = 0;
 
-      try (InputStream stream = this.connection.getInputStream()) {
+      try (InputStream stream = getContentInputStream()) {
         if (isNullInputStream(stream)) {
           return;
         }
@@ -429,12 +381,15 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
 
         // this is a list operation and need to retrieve the data
         // need a better solution
-        if (AbfsHttpConstants.HTTP_METHOD_GET.equals(this.method) && buffer == null) {
+        if (AbfsHttpConstants.HTTP_METHOD_GET.equals(this.method)
+            && buffer == null) {
           parseListFilesResponse(stream);
         } else {
           if (buffer != null) {
             while (totalBytesRead < length) {
-              int bytesRead = stream.read(buffer, offset + totalBytesRead, length - totalBytesRead);
+              int bytesRead = stream.read(buffer, offset + totalBytesRead,
+                  length
+                      - totalBytesRead);
               if (bytesRead == -1) {
                 endOfStream = true;
                 break;
@@ -452,9 +407,9 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
           }
         }
       } catch (IOException ex) {
-        LOG.warn("IO/Network error: {} {}: {}",
+        log.warn("IO/Network error: {} {}: {}",
             method, getMaskedUrl(), ex.getMessage());
-        LOG.debug("IO Error: ", ex);
+        log.debug("IO Error: ", ex);
         throw ex;
       } finally {
         this.recvResponseTimeMs += elapsedTimeMs(startTime);
@@ -463,23 +418,12 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
     }
   }
 
-  public void setRequestProperty(String key, String value) {
-    this.connection.setRequestProperty(key, value);
-  }
-
   /**
-   * Open the HTTP connection.
-   *
-   * @throws IOException if an error occurs.
+   * Get the response stream from the connection.
+   * @return InputStream: response stream from the connection after network call.
+   * @throws IOException if the response stream could not be created from the connection.
    */
-  private HttpURLConnection openConnection() throws IOException {
-    long start = System.nanoTime();
-    try {
-      return (HttpURLConnection) url.openConnection();
-    } finally {
-      connectionTimeMs = elapsedTimeMs(start);
-    }
-  }
+  protected abstract InputStream getContentInputStream() throws IOException;
 
   /**
    * When the request fails, this function is used to parse the responseAbfsHttpClient.LOG.debug("ExpectedError: ", ex);
@@ -499,7 +443,7 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
    *
    */
   private void processStorageErrorResponse() {
-    try (InputStream stream = connection.getErrorStream()) {
+    try (InputStream stream = getErrorStream()) {
       if (stream == null) {
         return;
       }
@@ -536,24 +480,25 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
       // Ignore errors that occur while attempting to parse the storage
       // error, since the response may have been handled by the HTTP driver
       // or for other reasons have an unexpected
-      LOG.debug("ExpectedError: ", ex);
+      log.debug("ExpectedError: ", ex);
     }
   }
 
   /**
-   * Returns the elapsed time in milliseconds.
+   * Get the error stream from the connection.
+   * @return InputStream
+   * @throws IOException if the error stream could not be created from the response stream.
    */
-  private long elapsedTimeMs(final long startTime) {
-    return (System.nanoTime() - startTime) / ONE_MILLION;
-  }
+  protected abstract InputStream getErrorStream() throws IOException;
 
   /**
    * Parse the list file response
    *
    * @param stream InputStream contains the list results.
-   * @throws IOException
+   * @throws IOException if the response cannot be deserialized.
    */
-  private void parseListFilesResponse(final InputStream stream) throws IOException {
+  private void parseListFilesResponse(final InputStream stream)
+      throws IOException {
     if (stream == null) {
       return;
     }
@@ -565,13 +510,21 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
 
     try {
       final ObjectMapper objectMapper = new ObjectMapper();
-      this.listResultSchema = objectMapper.readValue(stream, ListResultSchema.class);
+      this.listResultSchema = objectMapper.readValue(stream,
+          ListResultSchema.class);
     } catch (IOException ex) {
-      LOG.error("Unable to deserialize list results", ex);
+      log.error("Unable to deserialize list results", ex);
       throw ex;
     }
   }
 
+  /**
+   * Returns the elapsed time in milliseconds.
+   */
+  final long elapsedTimeMs(final long startTime) {
+    return (System.nanoTime() - startTime) / ONE_MILLION;
+  }
+
   /**
    * Check null stream, this is to pass findbugs's redundant check for NULL
    * @param stream InputStream
@@ -585,55 +538,148 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
    * @param key The request property key.
    * @return request peoperty value.
    */
-  String getConnProperty(String key) {
-    return connection.getRequestProperty(key);
-  }
+  abstract String getConnProperty(String key);
 
   /**
    * Gets the connection url.
    * @return url.
    */
-  URL getConnUrl() {
-    return connection.getURL();
+  abstract URL getConnUrl();
+
+  /**
+   * Gets the connection response code.
+   * @return response code.
+   * @throws IOException
+   */
+  abstract Integer getConnResponseCode() throws IOException;
+
+
+  /**
+   * Gets the connection response message.
+   * @return response message.
+   * @throws IOException
+   */
+  abstract String getConnResponseMessage() throws IOException;
+
+  /**
+   * Get request headers.
+   *
+   * @return request headers.
+   */
+  abstract Map<String, List<String>> getRequestProperties();
+
+  /**
+   * Get request header value for a header name.
+   *
+   * @param headerName header name.
+   * @return header value.
+   */
+  abstract String getRequestProperty(String headerName);
+
+  boolean getConnectionDisconnectedOnError() {
+    return connectionDisconnectedOnError;
   }
 
   /**
-   * Gets the connection request method.
-   * @return request method.
+   * Get the suffix to add to the tracing context that defines what http-client is
+   * used to make the network call
+   * @return the suffix to distinguish http client
    */
-  String getConnRequestMethod() {
-    return connection.getRequestMethod();
+  public abstract String getTracingContextSuffix();
+
+  public final long getSendLatency() {
+    return sendRequestTimeMs;
+  }
+
+  public final long getRecvLatency() {
+    return recvResponseTimeMs;
   }
 
   /**
-   * Gets the connection response code.
-   * @return response code.
-   * @throws IOException
+   * Set response status code for the server call.
+   *
+   * @param statusCode status code.
    */
-  Integer getConnResponseCode() throws IOException {
-    return connection.getResponseCode();
+  protected void setStatusCode(final int statusCode) {
+    this.statusCode = statusCode;
   }
 
   /**
-   * Gets the connection output stream.
-   * @return output stream.
-   * @throws IOException
+   * Sets response status description for the server call.
+   *
+   * @param statusDescription status description.
    */
-  OutputStream getConnOutputStream() throws IOException {
-    return connection.getOutputStream();
+  protected void setStatusDescription(final String statusDescription) {
+    this.statusDescription = statusDescription;
   }
 
   /**
-   * Gets the connection response message.
-   * @return response message.
-   * @throws IOException
+   * Set x-ms-request-id value from the server call response header.
    */
-  String getConnResponseMessage() throws IOException {
-    return connection.getResponseMessage();
+  protected void setRequestId() {
+    requestId = getResponseHeader(
+        HttpHeaderConfigurations.X_MS_REQUEST_ID);
+    if (requestId == null) {
+      requestId = AbfsHttpConstants.EMPTY_STRING;
+    }
   }
 
-  @VisibleForTesting
-  Boolean getConnectionDisconnectedOnError() {
+  /**
+   * Sets byteSent metric.
+   *
+   * @param bytesSent bytes sent.
+   */
+  protected void setBytesSent(final int bytesSent) {
+    this.bytesSent = bytesSent;
+  }
+
+  /**
+   * Sets expected bytes to be sent.
+   *
+   * @param expectedBytesToBeSent expected bytes to be sent.
+   */
+  protected void setExpectedBytesToBeSent(final int expectedBytesToBeSent) {
+    this.expectedBytesToBeSent = expectedBytesToBeSent;
+  }
+
+  /**
+   * Sets connection time in milliseconds taken to establish the connection.
+   *
+   * @param connectionTimeMs connection time in milliseconds.
+   */
+  protected void setConnectionTimeMs(final long connectionTimeMs) {
+    this.connectionTimeMs = connectionTimeMs;
+  }
+
+  /**
+   * Sets send request time in milliseconds.
+   *
+   * @param sendRequestTimeMs send request time in milliseconds.
+   */
+  protected void setSendRequestTimeMs(final long sendRequestTimeMs) {
+    this.sendRequestTimeMs = sendRequestTimeMs;
+  }
+
+  /**
+   * Sets receive response time in milliseconds.
+   *
+   * @param recvResponseTimeMs receive response time in milliseconds.
+   */
+  protected void setRecvResponseTimeMs(final long recvResponseTimeMs) {
+    this.recvResponseTimeMs = recvResponseTimeMs;
+  }
+
+  /**
+   * Marks network error and expect100 failures for send-payload phase.
+   */
+  protected void setConnectionDisconnectedOnError() {
+    this.connectionDisconnectedOnError = true;
+  }
+
+  /**
+   * @return value of {@link #connectionDisconnectedOnError}
+   */
+  protected boolean isConnectionDisconnectedOnError() {
     return connectionDisconnectedOnError;
   }
 
@@ -652,9 +698,75 @@ public class AbfsHttpOperation implements AbfsPerfLoggable {
       super(url, method, httpStatus);
     }
 
+    @Override
+    public void processResponse(final byte[] buffer,
+        final int offset,
+        final int length)
+        throws IOException {
+
+    }
+
+    @Override
+    public void setRequestProperty(final String key, final String value) {
+
+    }
+
+    @Override
+    protected InputStream getContentInputStream() throws IOException {
+      return null;
+    }
+
+    @Override
+    protected InputStream getErrorStream() throws IOException {
+      return null;
+    }
+
+    @Override
+    String getConnProperty(final String key) {
+      return null;
+    }
+
+    @Override
+    URL getConnUrl() {
+      return null;
+    }
+
+    @Override
+    Integer getConnResponseCode() throws IOException {
+      return null;
+    }
+
+    @Override
+    String getConnResponseMessage() throws IOException {
+      return null;
+    }
+
+    @Override
+    Map<String, List<String>> getRequestProperties() {
+      return null;
+    }
+
+    @Override
+    String getRequestProperty(final String headerName) {
+      return null;
+    }
+
+    @Override
+    public String getTracingContextSuffix() {
+      return null;
+    }
+
     @Override
     public String getResponseHeader(final String httpHeader) {
       return "";
     }
+
+    @Override
+    public void sendPayload(final byte[] buffer,
+        final int offset,
+        final int length)
+        throws IOException {
+
+    }
   }
 }

+ 300 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsJdkHttpOperation.java

@@ -0,0 +1,300 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.net.ProtocolException;
+import java.net.URL;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+
+import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EXPECT_100_JDK_ERROR;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HUNDRED_CONTINUE;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.JDK_FALLBACK;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.JDK_IMPL;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT;
+
+/**
+ * Implementation of {@link AbfsHttpOperation} for orchestrating calls using JDK's HttpURLConnection.
+ */
+public class AbfsJdkHttpOperation extends AbfsHttpOperation {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbfsJdkHttpOperation.class);
+
+  private final HttpURLConnection connection;
+
+  /**
+   * Initializes a new HTTP request and opens the connection.
+   *
+   * @param url The full URL including query string parameters.
+   * @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE).
+   * @param requestHeaders The HTTP request headers.READ_TIMEOUT
+   * @param connectionTimeout The Connection Timeout value to be used while establishing http connection
+   * @param readTimeout The Read Timeout value to be used with http connection while making a request
+   * @throws IOException if an error occurs.
+   */
+  public AbfsJdkHttpOperation(final URL url,
+      final String method,
+      final List<AbfsHttpHeader> requestHeaders,
+      final Duration connectionTimeout,
+      final Duration readTimeout)
+      throws IOException {
+    super(LOG, url, method, requestHeaders, connectionTimeout, readTimeout);
+
+    this.connection = openConnection();
+    if (this.connection instanceof HttpsURLConnection) {
+      HttpsURLConnection secureConn = (HttpsURLConnection) this.connection;
+      SSLSocketFactory sslSocketFactory
+          = DelegatingSSLSocketFactory.getDefaultFactory();
+      if (sslSocketFactory != null) {
+        secureConn.setSSLSocketFactory(sslSocketFactory);
+      }
+    }
+
+    this.connection.setConnectTimeout(getConnectionTimeout());
+    this.connection.setReadTimeout(getReadTimeout());
+    this.connection.setRequestMethod(method);
+
+    for (AbfsHttpHeader header : requestHeaders) {
+      setRequestProperty(header.getName(), header.getValue());
+    }
+  }
+
+  /**{@inheritDoc}*/
+  public String getResponseHeader(String httpHeader) {
+    return connection.getHeaderField(httpHeader);
+  }
+
+  /**{@inheritDoc}*/
+  public void sendPayload(byte[] buffer, int offset, int length)
+      throws IOException {
+    this.connection.setDoOutput(true);
+    this.connection.setFixedLengthStreamingMode(length);
+    if (buffer == null) {
+      // An empty buffer is sent to set the "Content-Length: 0" header, which
+      // is required by our endpoint.
+      buffer = new byte[]{};
+      offset = 0;
+      length = 0;
+    }
+
+    // send the request body
+
+    long startTime = 0;
+    startTime = System.nanoTime();
+    OutputStream outputStream = null;
+    // Updates the expected bytes to be sent based on length.
+    setExpectedBytesToBeSent(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 = getConnOutputStream();
+      } catch (IOException e) {
+        setConnectionDisconnectedOnError();
+        /* If getOutputStream fails with an expect-100 exception , we return back
+           without throwing an exception to the caller. Else, we throw back the exception.
+         */
+        String expectHeader = getConnProperty(EXPECT);
+        if (expectHeader != null && expectHeader.equals(HUNDRED_CONTINUE)
+            && e instanceof ProtocolException
+            && EXPECT_100_JDK_ERROR.equals(e.getMessage())) {
+          LOG.debug(
+              "Getting output stream failed with expect header enabled, returning back ",
+              e);
+          /*
+           * In case expect-100 assertion has failed, headers and inputStream should not
+           * be parsed. Reason being, conn.getHeaderField(), conn.getHeaderFields(),
+           * conn.getInputStream() will lead to repeated server call.
+           * ref: https://bugs.openjdk.org/browse/JDK-8314978.
+           * Reading conn.responseCode() and conn.getResponseMessage() is safe in
+           * case of Expect-100 error. Reason being, in JDK, it stores the responseCode
+           * in the HttpUrlConnection object before throwing exception to the caller.
+           */
+          setStatusCode(getConnResponseCode());
+          setStatusDescription(getConnResponseMessage());
+          return;
+        } else {
+          LOG.debug(
+              "Getting output stream failed without expect header enabled, throwing exception ",
+              e);
+          throw e;
+        }
+      }
+      // update bytes sent for successful as well as failed attempts via the
+      // accompanying statusCode.
+      setBytesSent(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();
+      }
+      setSendRequestTimeMs(elapsedTimeMs(startTime));
+    }
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  String getRequestProperty(final String headerName) {
+    return connection.getRequestProperty(headerName);
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  Map<String, List<String>> getRequestProperties() {
+    return connection.getRequestProperties();
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  protected InputStream getContentInputStream() throws IOException {
+    return connection.getInputStream();
+  }
+
+  /**{@inheritDoc}*/
+  public void processResponse(final byte[] buffer,
+      final int offset,
+      final int length) throws IOException {
+    if (isConnectionDisconnectedOnError()) {
+      LOG.debug("This connection was not successful or has been disconnected, "
+          + "hence not parsing headers and inputStream");
+      return;
+    }
+    processConnHeadersAndInputStreams(buffer, offset, length);
+  }
+
+  /**
+   * Parses headers and body of the response. Execute server call if {@link #sendPayload(byte[], int, int)}
+   * is not called.
+   *
+   * @param buffer buffer to store the response body.
+   * @param offset offset in the buffer.
+   * @param length length of the response body.
+   *
+   * @throws IOException network error or parsing error.
+   */
+  void processConnHeadersAndInputStreams(final byte[] buffer,
+      final int offset,
+      final int length) throws IOException {
+    // get the response
+    long startTime = 0;
+    startTime = System.nanoTime();
+
+    setStatusCode(getConnResponseCode());
+    setRecvResponseTimeMs(elapsedTimeMs(startTime));
+
+    setStatusDescription(getConnResponseMessage());
+    setRequestId();
+
+    // dump the headers
+    AbfsIoUtils.dumpHeadersToDebugLog("Response Headers",
+        connection.getHeaderFields());
+
+    if (AbfsHttpConstants.HTTP_METHOD_HEAD.equals(getMethod())) {
+      // If it is HEAD, and it is ERROR
+      return;
+    }
+
+    parseResponse(buffer, offset, length);
+  }
+
+  /**{@inheritDoc}*/
+  public void setRequestProperty(String key, String value) {
+    this.connection.setRequestProperty(key, value);
+  }
+
+  /**
+   * Creates a new {@link HttpURLConnection} instance. This instance is not connected.
+   * Any API call on the instance would make it reuse an existing connection or
+   * establish a new connection.
+   *
+   * @throws IOException if an error occurs.
+   */
+  private HttpURLConnection openConnection() throws IOException {
+    long start = System.nanoTime();
+    try {
+      return (HttpURLConnection) getUrl().openConnection();
+    } finally {
+      setConnectionTimeMs(elapsedTimeMs(start));
+    }
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  protected InputStream getErrorStream() {
+    return connection.getErrorStream();
+  }
+
+  /**{@inheritDoc}*/
+  String getConnProperty(String key) {
+    return connection.getRequestProperty(key);
+  }
+
+  /**{@inheritDoc}*/
+  URL getConnUrl() {
+    return connection.getURL();
+  }
+
+  /**{@inheritDoc}*/
+  Integer getConnResponseCode() throws IOException {
+    return connection.getResponseCode();
+  }
+
+  /**
+   * Gets the connection output stream.
+   * @return output stream.
+   * @throws IOException if creating outputStream on connection failed
+   */
+  OutputStream getConnOutputStream() throws IOException {
+    return connection.getOutputStream();
+  }
+
+  /**{@inheritDoc}*/
+  String getConnResponseMessage() throws IOException {
+    return connection.getResponseMessage();
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public String getTracingContextSuffix() {
+    return AbfsApacheHttpClient.usable() ? JDK_IMPL : JDK_FALLBACK;
+  }
+}

+ 240 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsManagedApacheHttpConnection.java

@@ -0,0 +1,240 @@
+/**
+ * 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 javax.net.ssl.SSLSession;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.util.UUID;
+
+import org.apache.http.HttpConnectionMetrics;
+import org.apache.http.HttpEntityEnclosingRequest;
+import org.apache.http.HttpException;
+import org.apache.http.HttpRequest;
+import org.apache.http.HttpResponse;
+import org.apache.http.conn.ManagedHttpClientConnection;
+import org.apache.http.conn.routing.HttpRoute;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.COLON;
+
+/**
+ * This class wraps the {@link ManagedHttpClientConnection} and provides
+ * insights onto the connection level activity.
+ */
+class AbfsManagedApacheHttpConnection
+    implements ManagedHttpClientConnection {
+
+  /**
+   * Underlying ApacheHttpClient connection that actually does the work over network.
+   */
+  private final ManagedHttpClientConnection httpClientConnection;
+
+  /**
+   * Managed HTTP context to track the connection level activity.
+   */
+  private AbfsManagedHttpClientContext managedHttpContext;
+
+  private final int hashCode;
+
+  AbfsManagedApacheHttpConnection(ManagedHttpClientConnection conn,
+      final HttpRoute route) {
+    this.httpClientConnection = conn;
+    this.hashCode = (UUID.randomUUID().toString()
+        + httpClientConnection.getId()).hashCode();
+  }
+
+  /**
+   * Sets the managed HTTP context to track the connection level activity.
+   */
+  void setManagedHttpContext(AbfsManagedHttpClientContext managedHttpContext) {
+    this.managedHttpContext = managedHttpContext;
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void close() throws IOException {
+    httpClientConnection.close();
+  }
+
+  /**{@inheritDoc}*/
+
+  @Override
+  public boolean isOpen() {
+    return httpClientConnection.isOpen();
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public boolean isStale() {
+    return httpClientConnection.isStale();
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void setSocketTimeout(final int timeout) {
+    httpClientConnection.setSocketTimeout(timeout);
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public int getSocketTimeout() {
+    return httpClientConnection.getSocketTimeout();
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void shutdown() throws IOException {
+    httpClientConnection.shutdown();
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public HttpConnectionMetrics getMetrics() {
+    return httpClientConnection.getMetrics();
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public boolean isResponseAvailable(final int timeout) throws IOException {
+    long start = System.currentTimeMillis();
+    boolean val = httpClientConnection.isResponseAvailable(timeout);
+    managedHttpContext.addReadTime(System.currentTimeMillis() - start);
+    return val;
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void sendRequestHeader(final HttpRequest request)
+      throws HttpException, IOException {
+    long start = System.currentTimeMillis();
+    httpClientConnection.sendRequestHeader(request);
+    managedHttpContext.addSendTime(System.currentTimeMillis() - start);
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void sendRequestEntity(final HttpEntityEnclosingRequest request)
+      throws HttpException, IOException {
+    long start = System.currentTimeMillis();
+    httpClientConnection.sendRequestEntity(request);
+    managedHttpContext.addSendTime(System.currentTimeMillis() - start);
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public HttpResponse receiveResponseHeader()
+      throws HttpException, IOException {
+    long start = System.currentTimeMillis();
+    HttpResponse response = httpClientConnection.receiveResponseHeader();
+    managedHttpContext.addReadTime(System.currentTimeMillis() - start);
+    return response;
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void receiveResponseEntity(final HttpResponse response)
+      throws HttpException, IOException {
+    long start = System.currentTimeMillis();
+    httpClientConnection.receiveResponseEntity(response);
+    managedHttpContext.addReadTime(System.currentTimeMillis() - start);
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void flush() throws IOException {
+    long start = System.currentTimeMillis();
+    httpClientConnection.flush();
+    managedHttpContext.addSendTime(System.currentTimeMillis() - start);
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public String getId() {
+    return httpClientConnection.getId();
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public void bind(final Socket socket) throws IOException {
+    httpClientConnection.bind(socket);
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public Socket getSocket() {
+    return httpClientConnection.getSocket();
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public SSLSession getSSLSession() {
+    return httpClientConnection.getSSLSession();
+  }
+
+  /**Gets the local address to which the socket is bound.*/
+  @Override
+  public InetAddress getLocalAddress() {
+    return httpClientConnection.getLocalAddress();
+  }
+
+  /**Gets the local port to which the socket is bound.*/
+  @Override
+  public int getLocalPort() {
+    return httpClientConnection.getLocalPort();
+  }
+
+  /**Returns the address to which the socket is connected.*/
+  @Override
+  public InetAddress getRemoteAddress() {
+    return httpClientConnection.getRemoteAddress();
+  }
+
+  /**Returns the remote port number to which this socket is connected.*/
+  @Override
+  public int getRemotePort() {
+    return httpClientConnection.getRemotePort();
+  }
+
+  @Override
+  public boolean equals(final Object o) {
+    if (o instanceof AbfsManagedApacheHttpConnection) {
+      return httpClientConnection.getId().equals(
+          ((AbfsManagedApacheHttpConnection) o).httpClientConnection.getId());
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return hashCode;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder stringBuilder = new StringBuilder();
+    stringBuilder.append(
+            httpClientConnection.getRemoteAddress().getHostName())
+        .append(COLON)
+        .append(httpClientConnection.getRemotePort())
+        .append(COLON)
+        .append(hashCode());
+    return stringBuilder.toString();
+  }
+}

+ 76 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsManagedHttpClientContext.java

@@ -0,0 +1,76 @@
+/**
+ * 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.http.HttpClientConnection;
+import org.apache.http.client.protocol.HttpClientContext;
+
+/**
+ * Registers the latency of different phases of a network call.
+ */
+public class AbfsManagedHttpClientContext extends HttpClientContext {
+
+  /**Connection establishment time*/
+  private long connectTime = 0L;
+
+  /**Time taken to receive and read response*/
+  private long readTime = 0L;
+
+ /***Time taken to send request*/
+  private long sendTime = 0L;
+
+  public AbfsManagedHttpClientContext() {
+  }
+
+  /**
+   * This to be used only in tests to get connection level activity.
+   * @param httpClientConnection HttpClientConnection which has to be intercepted
+   * by test-implementation.
+   * @return HttpClientConnection which is intercepted by test-implementation. For production
+   * implementation, it will return the same httpClientConnection.
+   */
+  protected HttpClientConnection interceptConnectionActivity(
+      HttpClientConnection httpClientConnection) {
+    return httpClientConnection;
+  }
+
+  public long getConnectTime() {
+    return connectTime;
+  }
+
+  public void setConnectTime(long connectTime) {
+    this.connectTime = connectTime;
+  }
+
+  public long getReadTime() {
+    return readTime;
+  }
+
+  public long getSendTime() {
+    return sendTime;
+  }
+
+  public void addSendTime(long sendTime) {
+    this.sendTime += sendTime;
+  }
+
+  public void addReadTime(long readTime) {
+    this.readTime += readTime;
+  }
+}

+ 109 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsManagedHttpRequestExecutor.java

@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsApacheHttpExpect100Exception;
+import org.apache.http.HttpClientConnection;
+import org.apache.http.HttpException;
+import org.apache.http.HttpRequest;
+import org.apache.http.HttpResponse;
+import org.apache.http.protocol.HttpContext;
+import org.apache.http.protocol.HttpRequestExecutor;
+
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.EXPECT;
+
+/**
+ * This class extends {@link HttpRequestExecutor} to intercept the connection
+ * activity and register the latency of different phases of a network call. It
+ * also overrides the HttpRequestExecutor's expect100 failure handling as the ADLS
+ * can send any failure statusCode in expect100 hand-shake failure and non
+ * necessarily 1XX code.
+ */
+public class AbfsManagedHttpRequestExecutor extends HttpRequestExecutor {
+
+  public AbfsManagedHttpRequestExecutor(final int expect100WaitTimeout) {
+    super(expect100WaitTimeout);
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  public HttpResponse execute(final HttpRequest request,
+      final HttpClientConnection conn,
+      final HttpContext context) throws IOException, HttpException {
+    if (context instanceof AbfsManagedHttpClientContext
+        && conn instanceof AbfsManagedApacheHttpConnection) {
+      ((AbfsManagedApacheHttpConnection) conn).setManagedHttpContext(
+          (AbfsManagedHttpClientContext) context);
+    }
+    return super.execute(request, conn, context);
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  protected HttpResponse doSendRequest(final HttpRequest request,
+      final HttpClientConnection conn,
+      final HttpContext context) throws IOException, HttpException {
+    final HttpClientConnection inteceptedConnection;
+    if (context instanceof AbfsManagedHttpClientContext) {
+      inteceptedConnection
+          = ((AbfsManagedHttpClientContext) context).interceptConnectionActivity(
+          conn);
+    } else {
+      inteceptedConnection = conn;
+    }
+    final HttpResponse res = super.doSendRequest(request, inteceptedConnection,
+        context);
+
+    /*
+     * ApacheHttpClient implementation does not raise an exception if the status
+     * of expect100 hand-shake is not less than 200. Although it sends payload only
+     * if the statusCode of the expect100 hand-shake is 100.
+     *
+     * ADLS can send any failure statusCode in exect100 handshake. So, an exception
+     * needs to be explicitly raised if expect100 assertion is failure but the
+     * ApacheHttpClient has not raised an exception.
+     *
+     * Response is only returned by this method if there is no expect100 request header
+     * or the expect100 assertion is failed.
+     */
+    if (request != null && request.containsHeader(EXPECT) && res != null) {
+      throw new AbfsApacheHttpExpect100Exception(res);
+    }
+    return res;
+  }
+
+  /**{@inheritDoc}*/
+  @Override
+  protected HttpResponse doReceiveResponse(final HttpRequest request,
+      final HttpClientConnection conn,
+      final HttpContext context) throws HttpException, IOException {
+    final HttpClientConnection interceptedConnection;
+    if (context instanceof AbfsManagedHttpClientContext) {
+      interceptedConnection
+          = ((AbfsManagedHttpClientContext) context).interceptConnectionActivity(
+          conn);
+    } else {
+      interceptedConnection = conn;
+    }
+    return super.doReceiveResponse(request,
+        interceptedConnection, context);
+  }
+}

+ 7 - 1
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsNoOpThrottlingIntercept.java

@@ -18,6 +18,10 @@
 
 package org.apache.hadoop.fs.azurebfs.services;
 
+/**
+ * Implementation of {@link AbfsThrottlingIntercept} that does not throttle
+ * the ABFS process.
+ */
 final class AbfsNoOpThrottlingIntercept implements AbfsThrottlingIntercept {
 
   public static final AbfsNoOpThrottlingIntercept INSTANCE = new AbfsNoOpThrottlingIntercept();
@@ -25,11 +29,13 @@ final class AbfsNoOpThrottlingIntercept implements AbfsThrottlingIntercept {
   private AbfsNoOpThrottlingIntercept() {
   }
 
+  /**{@inheritDoc}*/
   @Override
   public void updateMetrics(final AbfsRestOperationType operationType,
-      final AbfsHttpOperation abfsHttpOperation) {
+      final AbfsHttpOperation httpOperation) {
   }
 
+  /**{@inheritDoc}*/
   @Override
   public void sendingRequest(final AbfsRestOperationType operationType,
       final AbfsCounters abfsCounters) {

+ 81 - 21
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java

@@ -23,20 +23,26 @@ import java.io.UncheckedIOException;
 import java.net.HttpURLConnection;
 import java.net.URL;
 import java.net.UnknownHostException;
+import java.time.Duration;
 import java.util.List;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.ClosedIOException;
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
 import org.apache.hadoop.fs.azurebfs.AbfsStatistic;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsDriverException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
+import org.apache.http.impl.execchain.RequestAbortedException;
 
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_CONTINUE;
 import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.EGRESS_LIMIT_BREACH_ABBREVIATION;
@@ -86,11 +92,18 @@ public class AbfsRestOperation {
   private String failureReason;
   private AbfsRetryPolicy retryPolicy;
 
+  private final AbfsConfiguration abfsConfiguration;
+
   /**
    * This variable stores the tracing context used for last Rest Operation.
    */
   private TracingContext lastUsedTracingContext;
 
+  /**
+   * Number of retries due to IOException.
+   */
+  private int apacheHttpClientIoExceptions = 0;
+
   /**
    * Checks if there is non-null HTTP response.
    * @return true if there is a non-null HTTP response from the ABFS call.
@@ -136,8 +149,10 @@ public class AbfsRestOperation {
                     final AbfsClient client,
                     final String method,
                     final URL url,
-                    final List<AbfsHttpHeader> requestHeaders) {
-    this(operationType, client, method, url, requestHeaders, null);
+                    final List<AbfsHttpHeader> requestHeaders,
+                    final AbfsConfiguration abfsConfiguration) {
+    this(operationType, client, method, url, requestHeaders, null, abfsConfiguration
+    );
   }
 
   /**
@@ -154,7 +169,8 @@ public class AbfsRestOperation {
                     final String method,
                     final URL url,
                     final List<AbfsHttpHeader> requestHeaders,
-                    final String sasToken) {
+                    final String sasToken,
+                    final AbfsConfiguration abfsConfiguration) {
     this.operationType = operationType;
     this.client = client;
     this.method = method;
@@ -166,6 +182,7 @@ public class AbfsRestOperation {
     this.sasToken = sasToken;
     this.abfsCounters = client.getAbfsCounters();
     this.intercept = client.getIntercept();
+    this.abfsConfiguration = abfsConfiguration;
     this.retryPolicy = client.getExponentialRetryPolicy();
   }
 
@@ -178,7 +195,7 @@ public class AbfsRestOperation {
    * @param url The full URL including query string parameters.
    * @param requestHeaders The HTTP request headers.
    * @param buffer For uploads, this is the request entity body.  For downloads,
-   *               this will hold the response entity body.
+   * this will hold the response entity body.
    * @param bufferOffset An offset into the buffer where the data beings.
    * @param bufferLength The length of the data in the buffer.
    * @param sasToken A sasToken for optional re-use by AbfsInputStream/AbfsOutputStream.
@@ -191,8 +208,10 @@ public class AbfsRestOperation {
                     byte[] buffer,
                     int bufferOffset,
                     int bufferLength,
-                    String sasToken) {
-    this(operationType, client, method, url, requestHeaders, sasToken);
+                    String sasToken,
+                    final AbfsConfiguration abfsConfiguration) {
+    this(operationType, client, method, url, requestHeaders, sasToken, abfsConfiguration
+    );
     this.buffer = buffer;
     this.bufferOffset = bufferOffset;
     this.bufferLength = bufferLength;
@@ -284,7 +303,7 @@ public class AbfsRestOperation {
    */
   private boolean executeHttpOperation(final int retryCount,
     TracingContext tracingContext) throws AzureBlobFileSystemException {
-    AbfsHttpOperation httpOperation;
+    final AbfsHttpOperation httpOperation;
     // Used to avoid CST Metric Update in Case of UnknownHost/IO Exception.
     boolean wasKnownExceptionThrown = false;
 
@@ -305,15 +324,13 @@ public class AbfsRestOperation {
     try {
       // dump the headers
       AbfsIoUtils.dumpHeadersToDebugLog("Request Headers",
-          httpOperation.getConnection().getRequestProperties());
+          httpOperation.getRequestProperties());
       intercept.sendingRequest(operationType, abfsCounters);
       if (hasRequestBody) {
-        // HttpUrlConnection requires
-        httpOperation.sendRequest(buffer, bufferOffset, bufferLength);
+        httpOperation.sendPayload(buffer, bufferOffset, bufferLength);
         incrementCounter(AbfsStatistic.SEND_REQUESTS, 1);
         incrementCounter(AbfsStatistic.BYTES_SENT, bufferLength);
       }
-
       httpOperation.processResponse(buffer, bufferOffset, bufferLength);
       incrementCounter(AbfsStatistic.GET_RESPONSES, 1);
       //Only increment bytesReceived counter when the status code is 2XX.
@@ -351,6 +368,9 @@ public class AbfsRestOperation {
       retryPolicy = client.getRetryPolicy(failureReason);
       LOG.warn("Unknown host name: {}. Retrying to resolve the host name...",
           hostname);
+      if (httpOperation instanceof AbfsAHCHttpOperation) {
+        registerApacheHttpClientIoException();
+      }
       if (!retryPolicy.shouldRetry(retryCount, -1)) {
         throw new InvalidAbfsRestOperationException(ex, retryCount);
       }
@@ -363,6 +383,13 @@ public class AbfsRestOperation {
 
       failureReason = RetryReason.getAbbreviation(ex, -1, "");
       retryPolicy = client.getRetryPolicy(failureReason);
+      if (httpOperation instanceof AbfsAHCHttpOperation) {
+        registerApacheHttpClientIoException();
+        if (ex instanceof RequestAbortedException
+            && ex.getCause() instanceof ClosedIOException) {
+          throw new AbfsDriverException((IOException) ex.getCause());
+        }
+      }
       if (!retryPolicy.shouldRetry(retryCount, -1)) {
         throw new InvalidAbfsRestOperationException(ex, retryCount);
       }
@@ -380,6 +407,18 @@ public class AbfsRestOperation {
     return true;
   }
 
+  /**
+   * Registers switch off of ApacheHttpClient in case of IOException retries increases
+   * more than the threshold.
+   */
+  private void registerApacheHttpClientIoException() {
+    apacheHttpClientIoExceptions++;
+    if (apacheHttpClientIoExceptions
+        >= abfsConfiguration.getMaxApacheHttpClientIoExceptionsRetries()) {
+      AbfsApacheHttpClient.registerFallback();
+    }
+  }
+
   /**
    * Sign an operation.
    * @param httpOperation operation to sign
@@ -388,11 +427,11 @@ public class AbfsRestOperation {
    */
   @VisibleForTesting
   public void signRequest(final AbfsHttpOperation httpOperation, int bytesToSign) throws IOException {
-    switch(client.getAuthType()) {
+    switch (client.getAuthType()) {
       case Custom:
       case OAuth:
         LOG.debug("Authenticating request with OAuth2 access token");
-        httpOperation.getConnection().setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION,
+        httpOperation.setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION,
             client.getAccessToken());
         break;
       case SAS:
@@ -405,23 +444,44 @@ public class AbfsRestOperation {
         LOG.debug("Signing request with shared key");
         // sign the HTTP request
         client.getSharedKeyCredentials().signRequest(
-            httpOperation.getConnection(),
+            httpOperation,
             bytesToSign);
         break;
     }
   }
 
   /**
-   * Creates new object of {@link AbfsHttpOperation} with the url, method, requestHeader fields and
-   * timeout values as set in configuration of the AbfsRestOperation object.
-   *
-   * @return {@link AbfsHttpOperation} to be used for sending requests
+   * 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,
-            client.getAbfsConfiguration().getHttpConnectionTimeout(),
-            client.getAbfsConfiguration().getHttpReadTimeout());
+    HttpOperationType httpOperationType
+        = abfsConfiguration.getPreferredHttpOperationType();
+    if (httpOperationType == HttpOperationType.APACHE_HTTP_CLIENT
+        && isApacheClientUsable()) {
+      return createAbfsAHCHttpOperation();
+    }
+    return createAbfsHttpOperation();
+  }
+
+  private boolean isApacheClientUsable() {
+    return AbfsApacheHttpClient.usable();
+  }
+
+  @VisibleForTesting
+  AbfsJdkHttpOperation createAbfsHttpOperation() throws IOException {
+    return new AbfsJdkHttpOperation(url, method, requestHeaders,
+        Duration.ofMillis(client.getAbfsConfiguration().getHttpConnectionTimeout()),
+        Duration.ofMillis(client.getAbfsConfiguration().getHttpReadTimeout()));
+  }
+
+  @VisibleForTesting
+  AbfsAHCHttpOperation createAbfsAHCHttpOperation() throws IOException {
+    return new AbfsAHCHttpOperation(url, method, requestHeaders,
+        Duration.ofMillis(client.getAbfsConfiguration().getHttpConnectionTimeout()),
+        Duration.ofMillis(client.getAbfsConfiguration().getHttpReadTimeout()),
+        client.getAbfsApacheHttpClient());
   }
 
   /**

+ 3 - 2
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsThrottlingIntercept.java

@@ -30,11 +30,12 @@ public interface AbfsThrottlingIntercept {
 
   /**
    * Updates the metrics for successful and failed read and write operations.
+   *
    * @param operationType Only applicable for read and write operations.
-   * @param abfsHttpOperation Used for status code and data transferred.
+   * @param httpOperation Used for status code and data transferred.
    */
   void updateMetrics(AbfsRestOperationType operationType,
-      AbfsHttpOperation abfsHttpOperation);
+      AbfsHttpOperation httpOperation);
 
   /**
    * Called before the request is sent.  Client-side throttling

+ 306 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/KeepAliveCache.java

@@ -0,0 +1,306 @@
+/**
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.Stack;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.fs.ClosedIOException;
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.http.HttpClientConnection;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_MAX_CONN_SYS_PROP;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.KEEP_ALIVE_CACHE_CLOSED;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_APACHE_HTTP_CLIENT_MAX_CACHE_CONNECTION_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS;
+
+/**
+ * Connection-pooling heuristics used by {@link AbfsConnectionManager}. Each
+ * instance of FileSystem has its own KeepAliveCache.
+ * <p>
+ * Why this implementation is required in comparison to {@link org.apache.http.impl.conn.PoolingHttpClientConnectionManager}
+ * connection-pooling:
+ * <ol>
+ * <li>PoolingHttpClientConnectionManager heuristic caches all the reusable connections it has created.
+ * JDK's implementation only caches a limited number of connections. The limit is given by JVM system
+ * property "http.maxConnections". If there is no system-property, it defaults to 5.</li>
+ * <li>In PoolingHttpClientConnectionManager, it expects the application to provide `setMaxPerRoute` and `setMaxTotal`,
+ * which the implementation uses as the total number of connections it can create. For application using ABFS, it is not
+ * feasible to provide a value in the initialisation of the connectionManager. JDK's implementation has no cap on the
+ * number of connections it can create.</li>
+ * </ol>
+ */
+class KeepAliveCache extends Stack<KeepAliveCache.KeepAliveEntry>
+    implements
+    Closeable {
+  private static final long serialVersionUID = 1L;
+
+  private static final Logger LOG = LoggerFactory.getLogger(KeepAliveCache.class);
+
+  /**
+   * Scheduled timer that evicts idle connections.
+   */
+  private final transient Timer timer;
+
+  /**
+   * Task provided to the timer that owns eviction logic.
+   */
+  private final transient TimerTask timerTask;
+
+  /**
+   * Flag to indicate if the cache is closed.
+   */
+  private final AtomicBoolean isClosed = new AtomicBoolean(false);
+
+  /**
+   * Counter to keep track of the number of KeepAliveCache instances created.
+   */
+  private static final AtomicInteger KAC_COUNTER = new AtomicInteger(0);
+
+  /**
+   * Maximum number of connections that can be cached.
+   */
+  private final int maxConn;
+
+  /**
+   * Time-to-live for an idle connection.
+   */
+  private final long connectionIdleTTL;
+
+  /**
+   * Flag to indicate if the eviction thread is paused.
+   */
+  private final AtomicBoolean isPaused = new AtomicBoolean(false);
+
+  /**
+   * Account name for which the cache is created. To be used only in exception
+   * messages.
+   */
+  private final String accountNamePath;
+
+  @VisibleForTesting
+  synchronized void pauseThread() {
+    isPaused.set(true);
+  }
+
+  @VisibleForTesting
+  synchronized void resumeThread() {
+    isPaused.set(false);
+  }
+
+  /**
+   * @return connectionIdleTTL.
+   */
+  @VisibleForTesting
+  public long getConnectionIdleTTL() {
+    return connectionIdleTTL;
+  }
+
+  /**
+   * Creates an {@link KeepAliveCache} instance using filesystem's configuration.
+   * <p>
+   * The size of the cache is determined by the configuration
+   * {@value org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys#FS_AZURE_APACHE_HTTP_CLIENT_MAX_CACHE_CONNECTION_SIZE}.
+   * If the configuration is not set, the system-property {@value org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants#HTTP_MAX_CONN_SYS_PROP}.
+   * If the system-property is not set or set to 0, the default value
+   * {@value org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations#DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS} is used.
+   * <p>
+   * This schedules an eviction thread to run every connectionIdleTTL milliseconds
+   * given by the configuration {@link AbfsConfiguration#getMaxApacheHttpClientConnectionIdleTime()}.
+   * @param abfsConfiguration Configuration of the filesystem.
+   */
+  KeepAliveCache(AbfsConfiguration abfsConfiguration) {
+    accountNamePath = abfsConfiguration.getAccountName();
+    this.timer = new Timer("abfs-kac-" + KAC_COUNTER.getAndIncrement(), true);
+
+    int sysPropMaxConn = Integer.parseInt(System.getProperty(HTTP_MAX_CONN_SYS_PROP, "0"));
+    final int defaultMaxConn;
+    if (sysPropMaxConn > 0) {
+      defaultMaxConn = sysPropMaxConn;
+    } else {
+      defaultMaxConn = DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS;
+    }
+    this.maxConn = abfsConfiguration.getInt(
+        FS_AZURE_APACHE_HTTP_CLIENT_MAX_CACHE_CONNECTION_SIZE,
+        defaultMaxConn);
+
+    this.connectionIdleTTL
+        = abfsConfiguration.getMaxApacheHttpClientConnectionIdleTime();
+    this.timerTask = new TimerTask() {
+      @Override
+      public void run() {
+          if (isPaused.get() || isClosed.get()) {
+            return;
+          }
+          evictIdleConnection();
+      }
+    };
+    timer.schedule(timerTask, 0, connectionIdleTTL);
+  }
+
+  /**
+   * Iterate over the cache and evict the idle connections. An idle connection is
+   * one that has been in the cache for more than connectionIdleTTL milliseconds.
+   */
+  synchronized void evictIdleConnection() {
+    long currentTime = System.currentTimeMillis();
+    int i;
+    for (i = 0; i < size(); i++) {
+      KeepAliveEntry e = elementAt(i);
+      if ((currentTime - e.idleStartTime) > connectionIdleTTL
+          || e.httpClientConnection.isStale()) {
+        HttpClientConnection hc = e.httpClientConnection;
+        closeHttpClientConnection(hc);
+      } else {
+        break;
+      }
+    }
+    subList(0, i).clear();
+  }
+
+  /**
+   * Safe close of the HttpClientConnection.
+   *
+   * @param hc HttpClientConnection to be closed
+   */
+  private void closeHttpClientConnection(final HttpClientConnection hc) {
+    try {
+      hc.close();
+    } catch (IOException ex) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Close failed for connection: {}", hc, ex);
+      }
+    }
+  }
+
+  /**
+   * Close all connections in cache and cancel the eviction timer.
+   */
+  @Override
+  public synchronized void close() {
+    boolean closed = isClosed.getAndSet(true);
+    if (closed) {
+      return;
+    }
+    closeInternal();
+  }
+
+  @VisibleForTesting
+  void closeInternal() {
+    timerTask.cancel();
+    timer.purge();
+    while (!empty()) {
+      KeepAliveEntry e = pop();
+      closeHttpClientConnection(e.httpClientConnection);
+    }
+  }
+
+  /**
+   * <p>
+   * Gets the latest added HttpClientConnection from the cache. The returned connection
+   * is non-stale and has been in the cache for less than connectionIdleTTL milliseconds.
+   * <p>
+   * The cache is checked from the top of the stack. If the connection is stale or has been
+   * in the cache for more than connectionIdleTTL milliseconds, it is closed and the next
+   * connection is checked. Once a valid connection is found, it is returned.
+   * @return HttpClientConnection: if a valid connection is found, else null.
+   * @throws IOException if the cache is closed.
+   */
+  public synchronized HttpClientConnection get()
+      throws IOException {
+    if (isClosed.get()) {
+      throw new ClosedIOException(accountNamePath, KEEP_ALIVE_CACHE_CLOSED);
+    }
+    if (empty()) {
+      return null;
+    }
+    HttpClientConnection hc = null;
+    long currentTime = System.currentTimeMillis();
+    do {
+      KeepAliveEntry e = pop();
+      if ((currentTime - e.idleStartTime) > connectionIdleTTL
+          || e.httpClientConnection.isStale()) {
+        closeHttpClientConnection(e.httpClientConnection);
+      } else {
+        hc = e.httpClientConnection;
+      }
+    } while ((hc == null) && (!empty()));
+    return hc;
+  }
+
+  /**
+   * Puts the HttpClientConnection in the cache. If the size of cache is equal to
+   * maxConn, the oldest connection is closed and removed from the cache, which
+   * will make space for the new connection. If the cache is closed or of zero size,
+   * the connection is closed and not added to the cache.
+   *
+   * @param httpClientConnection HttpClientConnection to be cached
+   * @return true if the HttpClientConnection is added in active cache, false otherwise.
+   */
+  public synchronized boolean put(HttpClientConnection httpClientConnection) {
+    if (isClosed.get() || maxConn == 0) {
+      closeHttpClientConnection(httpClientConnection);
+      return false;
+    }
+    if (size() == maxConn) {
+      closeHttpClientConnection(get(0).httpClientConnection);
+      subList(0, 1).clear();
+    }
+    KeepAliveEntry entry = new KeepAliveEntry(httpClientConnection,
+        System.currentTimeMillis());
+    push(entry);
+    return true;
+  }
+
+  @Override
+  public synchronized boolean equals(final Object o) {
+    return super.equals(o);
+  }
+
+  @Override
+  public synchronized int hashCode() {
+    return super.hashCode();
+  }
+
+  /**
+   * Entry data-structure in the cache.
+   */
+  static class KeepAliveEntry {
+
+    /**HttpClientConnection in the cache entry.*/
+    private final HttpClientConnection httpClientConnection;
+
+    /**Time at which the HttpClientConnection was added to the cache.*/
+    private final long idleStartTime;
+
+    KeepAliveEntry(HttpClientConnection hc, long idleStartTime) {
+      this.httpClientConnection = hc;
+      this.idleStartTime = idleStartTime;
+    }
+  }
+}

+ 8 - 9
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java

@@ -21,7 +21,6 @@ package org.apache.hadoop.fs.azurebfs.services;
 import javax.crypto.Mac;
 import javax.crypto.spec.SecretKeySpec;
 import java.io.UnsupportedEncodingException;
-import java.net.HttpURLConnection;
 import java.net.URL;
 import java.net.URLDecoder;
 import java.text.DateFormat;
@@ -80,7 +79,7 @@ public class SharedKeyCredentials {
     initializeMac();
   }
 
-  public void signRequest(HttpURLConnection connection, final long contentLength) throws UnsupportedEncodingException {
+  public void signRequest(AbfsHttpOperation connection, final long contentLength) throws UnsupportedEncodingException {
 
     String gmtTime = getGMTTime();
     connection.setRequestProperty(HttpHeaderConfigurations.X_MS_DATE, gmtTime);
@@ -117,7 +116,7 @@ public class SharedKeyCredentials {
    * @param conn                the HttpURLConnection for the operation
    * @param canonicalizedString the canonicalized string to add the canonicalized headerst to.
    */
-  private static void addCanonicalizedHeaders(final HttpURLConnection conn, final StringBuilder canonicalizedString) {
+  private static void addCanonicalizedHeaders(final AbfsHttpOperation conn, final StringBuilder canonicalizedString) {
     // Look for header names that start with
     // HeaderNames.PrefixForStorageHeader
     // Then sort them in case-insensitive manner.
@@ -205,13 +204,13 @@ public class SharedKeyCredentials {
    */
   private static String canonicalizeHttpRequest(final URL address,
       final String accountName, final String method, final String contentType,
-      final long contentLength, final String date, final HttpURLConnection conn)
+      final long contentLength, final String date, final AbfsHttpOperation conn)
       throws UnsupportedEncodingException {
 
     // The first element should be the Method of the request.
     // I.e. GET, POST, PUT, or HEAD.
     final StringBuilder canonicalizedString = new StringBuilder(EXPECTED_BLOB_QUEUE_CANONICALIZED_STRING_LENGTH);
-    canonicalizedString.append(conn.getRequestMethod());
+    canonicalizedString.append(conn.getMethod());
 
     // The next elements are
     // If any element is missing it may be empty.
@@ -450,7 +449,7 @@ public class SharedKeyCredentials {
     return value.substring(spaceDex);
   }
 
-  private static String getHeaderValue(final HttpURLConnection conn, final String headerName, final String defaultValue) {
+  private static String getHeaderValue(final AbfsHttpOperation conn, final String headerName, final String defaultValue) {
     final String headerValue = conn.getRequestProperty(headerName);
     return headerValue == null ? defaultValue : headerValue;
   }
@@ -465,7 +464,7 @@ public class SharedKeyCredentials {
    *                      -1 if unknown
    * @return a canonicalized string.
    */
-  private String canonicalize(final HttpURLConnection conn,
+  private String canonicalize(final AbfsHttpOperation conn,
                               final String accountName,
                               final Long contentLength) throws UnsupportedEncodingException {
 
@@ -476,8 +475,8 @@ public class SharedKeyCredentials {
 
     String contentType = getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_TYPE, "");
 
-    return canonicalizeHttpRequest(conn.getURL(), accountName,
-        conn.getRequestMethod(), contentType, contentLength, null, conn);
+    return canonicalizeHttpRequest(conn.getConnUrl(), accountName,
+        conn.getMethod(), contentType, contentLength, null, conn);
   }
 
   /**

+ 1 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/TracingContext.java

@@ -181,6 +181,7 @@ public class TracingContext {
               + getPrimaryRequestIdForHeader(retryCount > 0) + ":" + streamID
               + ":" + opType + ":" + retryCount;
       header = addFailureReasons(header, previousFailure, retryPolicyAbbreviation);
+      header += (":" + httpOperation.getTracingContextSuffix());
       break;
     case TWO_ID_FORMAT:
       header = clientCorrelationID + ":" + clientRequestId;

+ 39 - 0
hadoop-tools/hadoop-azure/src/site/markdown/abfs.md

@@ -866,6 +866,45 @@ and all associated tests to see how to make use of these extension points.
 
 _Warning_ These extension points are unstable.
 
+### <a href="networking"></a>Networking Layer:
+
+ABFS Driver can use the following networking libraries:
+- ApacheHttpClient:
+  -  <a href = "https://hc.apache.org/httpcomponents-client-4.5.x/index.html">Library Documentation</a>.
+  - Default networking library.
+- JDK networking library:
+  - <a href="https://docs.oracle.com/javase/8/docs/api/java/net/HttpURLConnection.html">Library documentation</a>.
+
+The networking library can be configured using the configuration `fs.azure.networking.library`
+while initializing the filesystem.
+Following are the supported values:
+- `APACHE_HTTP_CLIENT` : Use Apache HttpClient [Default]
+- `JDK_HTTP_URL_CONNECTION` : Use JDK networking library
+
+#### <a href="ahc_networking_conf"></a>ApacheHttpClient networking layer configuration Options:
+
+Following are the configuration options for ApacheHttpClient networking layer that
+can be provided at the initialization of the filesystem:
+1. `fs.azure.apache.http.client.idle.connection.ttl`:
+   1. Maximum idle time in milliseconds for a connection to be kept alive in the connection pool.
+      If the connection is not reused within the time limit, the connection shall be closed.
+   2. Default value: 5000 milliseconds.
+2. `fs.azure.apache.http.client.max.cache.connection.size`:
+   1. Maximum number of connections that can be cached in the connection pool for
+      a filesystem instance. Total number of concurrent connections has no limit.
+   2. Default value: 5.
+3. `fs.azure.apache.http.client.max.io.exception.retries`:
+   1. Maximum number of times the client will retry on IOExceptions for a single request
+      with ApacheHttpClient networking-layer. Breach of this limit would turn off
+      the future uses of the ApacheHttpClient library in the current JVM instance.
+   2. Default value: 3.
+
+#### <a href="ahc_classpath"></a> ApacheHttpClient classpath requirements:
+
+ApacheHttpClient is a `compile` maven dependency in hadoop-azure and would be
+included in the hadoop-azure jar. For using hadoop-azure with ApacheHttpClient no
+additional information is required in the classpath.
+
 ## <a href="options"></a> Other configuration options
 
 Consult the javadocs for `org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys`,

+ 1 - 1
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
 import org.apache.hadoop.fs.azurebfs.security.EncodingHelper;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClientUtils;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.assertj.core.api.Assertions;
 import org.assertj.core.api.Assumptions;
@@ -51,7 +52,6 @@ import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider;
 import org.apache.hadoop.fs.azurebfs.extensions.MockEncryptionContextProvider;
 import org.apache.hadoop.fs.azurebfs.security.ContextProviderEncryptionAdapter;
 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.utils.EncryptionType;
 import org.apache.hadoop.fs.impl.OpenFileParameters;

+ 57 - 7
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
 import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
 import org.apache.hadoop.fs.statistics.IOStatisticsSource;
 
@@ -55,22 +56,71 @@ public class ITestAbfsReadWriteAndSeek extends AbstractAbfsScaleTest {
    * For test performance, a full x*y test matrix is not used.
    * @return the test parameters
    */
-  @Parameterized.Parameters(name = "Size={0}-readahead={1}")
+  @Parameterized.Parameters(name = "Size={0}-readahead={1}-Client={2}")
   public static Iterable<Object[]> sizes() {
-    return Arrays.asList(new Object[][]{{MIN_BUFFER_SIZE, true},
-        {DEFAULT_READ_BUFFER_SIZE, false},
-        {DEFAULT_READ_BUFFER_SIZE, true},
-        {APPENDBLOB_MAX_WRITE_BUFFER_SIZE, false},
-        {MAX_BUFFER_SIZE, true}});
+    return Arrays.asList(new Object[][]{
+        {
+            MIN_BUFFER_SIZE,
+            true,
+            HttpOperationType.JDK_HTTP_URL_CONNECTION
+        },
+        {
+            MIN_BUFFER_SIZE,
+            true,
+            HttpOperationType.APACHE_HTTP_CLIENT
+        },
+        {
+            DEFAULT_READ_BUFFER_SIZE,
+            false,
+            HttpOperationType.JDK_HTTP_URL_CONNECTION
+        },
+        {
+            DEFAULT_READ_BUFFER_SIZE,
+            false,
+            HttpOperationType.APACHE_HTTP_CLIENT
+        },
+        {
+            DEFAULT_READ_BUFFER_SIZE,
+            true,
+            HttpOperationType.JDK_HTTP_URL_CONNECTION
+        },
+        {
+            DEFAULT_READ_BUFFER_SIZE,
+            true,
+            HttpOperationType.APACHE_HTTP_CLIENT
+        },
+        {
+            APPENDBLOB_MAX_WRITE_BUFFER_SIZE,
+            false,
+            HttpOperationType.JDK_HTTP_URL_CONNECTION
+        },
+        {
+            APPENDBLOB_MAX_WRITE_BUFFER_SIZE,
+            false,
+            HttpOperationType.APACHE_HTTP_CLIENT
+        },
+        {
+            MAX_BUFFER_SIZE,
+            true,
+            HttpOperationType.JDK_HTTP_URL_CONNECTION
+        },
+        {
+            MAX_BUFFER_SIZE,
+            true,
+            HttpOperationType.APACHE_HTTP_CLIENT
+        }
+    });
   }
 
   private final int size;
   private final boolean readaheadEnabled;
+  private final HttpOperationType httpOperationType;
 
   public ITestAbfsReadWriteAndSeek(final int size,
-      final boolean readaheadEnabled) throws Exception {
+      final boolean readaheadEnabled, final HttpOperationType httpOperationType) throws Exception {
     this.size = size;
     this.readaheadEnabled = readaheadEnabled;
+    this.httpOperationType = httpOperationType;
   }
 
   @Test

+ 5 - 2
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
 import org.apache.hadoop.fs.azurebfs.security.ContextEncryptionAdapter;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -44,7 +45,6 @@ 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;
 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.ITestAbfsClient;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
@@ -56,6 +56,7 @@ import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
 import static java.net.HttpURLConnection.HTTP_OK;
 import static java.net.HttpURLConnection.HTTP_PRECON_FAILED;
 
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.ArgumentMatchers.nullable;
@@ -238,7 +239,9 @@ public class ITestAzureBlobFileSystemCreate extends
     intercept(FileNotFoundException.class,
         () -> {
           try (FilterOutputStream fos = new FilterOutputStream(out)) {
-            fos.write('a');
+            byte[] bytes = new byte[8*ONE_MB];
+            fos.write(bytes);
+            fos.write(bytes);
             fos.flush();
             out.hsync();
             fs.delete(testPath, false);

+ 1 - 1
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java

@@ -40,9 +40,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
 import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
 import org.apache.hadoop.fs.azurebfs.extensions.MockDelegationSASTokenProvider;
-import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
 import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
 import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntryScope;
 import org.apache.hadoop.fs.permission.AclStatus;

+ 2 - 2
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelete.java

@@ -37,8 +37,8 @@ import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClientTestUtil;
-import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
 import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
 import org.apache.hadoop.fs.azurebfs.services.ITestAbfsClient;
 import org.apache.hadoop.fs.azurebfs.services.TestAbfsPerfTracker;
 import org.apache.hadoop.fs.azurebfs.utils.TestMockHelpers;
@@ -264,7 +264,7 @@ public class ITestAzureBlobFileSystemDelete extends
     AbfsRestOperation idempotencyRetOp = Mockito.spy(ITestAbfsClient.getRestOp(
         DeletePath, mockClient, HTTP_METHOD_DELETE,
         ITestAbfsClient.getTestUrl(mockClient, "/NonExistingPath"),
-        ITestAbfsClient.getTestRequestHeaders(mockClient)));
+        ITestAbfsClient.getTestRequestHeaders(mockClient), getConfiguration()));
     idempotencyRetOp.hardSetResult(HTTP_OK);
 
     doReturn(idempotencyRetOp).when(mockClient).deleteIdempotencyCheckOp(any());

+ 10 - 7
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemE2E.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
 
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_HTTP_CONNECTION_TIMEOUT;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_HTTP_READ_TIMEOUT;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_MAX_IO_RETRIES;
@@ -257,22 +258,24 @@ public class ITestAzureBlobFileSystemE2E extends AbstractAbfsIntegrationTest {
   }
 
   public void testHttpTimeouts(int connectionTimeoutMs, int readTimeoutMs)
-          throws Exception {
+      throws Exception {
     Configuration conf = this.getRawConfiguration();
     // set to small values that will cause timeouts
     conf.setInt(AZURE_HTTP_CONNECTION_TIMEOUT, connectionTimeoutMs);
     conf.setInt(AZURE_HTTP_READ_TIMEOUT, readTimeoutMs);
+    conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION,
+        false);
     // Reduce retry count to reduce test run time
     conf.setInt(AZURE_MAX_IO_RETRIES, 1);
     final AzureBlobFileSystem fs = getFileSystem(conf);
     Assertions.assertThat(
-                    fs.getAbfsStore().getAbfsConfiguration().getHttpConnectionTimeout())
-            .describedAs("HTTP connection time should be picked from config")
-            .isEqualTo(connectionTimeoutMs);
+            fs.getAbfsStore().getAbfsConfiguration().getHttpConnectionTimeout())
+        .describedAs("HTTP connection time should be picked from config")
+        .isEqualTo(connectionTimeoutMs);
     Assertions.assertThat(
-                    fs.getAbfsStore().getAbfsConfiguration().getHttpReadTimeout())
-            .describedAs("HTTP Read time should be picked from config")
-            .isEqualTo(readTimeoutMs);
+            fs.getAbfsStore().getAbfsConfiguration().getHttpReadTimeout())
+        .describedAs("HTTP Read time should be picked from config")
+        .isEqualTo(readTimeoutMs);
     Path testPath = path(methodName.getMethodName());
     ContractTestUtils.createFile(fs, testPath, false, new byte[0]);
   }

+ 23 - 2
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemLease.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.fs.azurebfs;
 
 import java.io.IOException;
+import java.util.concurrent.Callable;
 import java.util.concurrent.RejectedExecutionException;
 
 import org.junit.Assert;
@@ -28,6 +29,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsDriverException;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsLease;
@@ -302,11 +305,29 @@ public class ITestAzureBlobFileSystemLease extends AbstractAbfsIntegrationTest {
     fs.close();
     Assert.assertTrue("Store leases were not freed", fs.getAbfsStore().areLeasesFreed());
 
-    LambdaTestUtils.intercept(RejectedExecutionException.class, () -> {
+    Callable<String> exceptionRaisingCallable = () -> {
       try (FSDataOutputStream out2 = fs.append(testFilePath)) {
       }
       return "Expected exception on new append after closed FS";
-    });
+    };
+    /*
+     * For ApacheHttpClient, the failure would happen when trying to get a connection
+     * from KeepAliveCache, which is not possible after the FS is closed, as that
+     * also closes the cache.
+     *
+     * For JDK_Client, the failure happens when trying to submit a task to the
+     * executor service, which is not possible after the FS is closed, as that
+     * also shuts down the executor service.
+     */
+
+    if (getConfiguration().getPreferredHttpOperationType()
+        == HttpOperationType.APACHE_HTTP_CLIENT) {
+      LambdaTestUtils.intercept(AbfsDriverException.class,
+          exceptionRaisingCallable);
+    } else {
+      LambdaTestUtils.intercept(RejectedExecutionException.class,
+          exceptionRaisingCallable);
+    }
   }
 
   @Test(timeout = TEST_EXECUTION_TIMEOUT)

+ 11 - 3
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java

@@ -38,9 +38,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 import org.apache.hadoop.fs.azurebfs.enums.Trilean;
-import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
 import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
 import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat;
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
@@ -51,6 +51,8 @@ import org.apache.hadoop.util.Preconditions;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_CORRELATIONID;
 import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.JDK_HTTP_URL_CONNECTION;
 import static org.apache.hadoop.fs.azurebfs.services.RetryPolicyConstants.EXPONENTIAL_RETRY_POLICY_ABBREVIATION;
 import static org.apache.hadoop.fs.azurebfs.services.RetryPolicyConstants.STATIC_RETRY_POLICY_ABBREVIATION;
 import static org.apache.hadoop.fs.azurebfs.services.RetryReasonConstants.CONNECTION_TIMEOUT_ABBREVIATION;
@@ -135,10 +137,16 @@ public class TestTracingContext extends AbstractAbfsIntegrationTest {
 
     testClasses.put(new ITestAzureBlobFileSystemListStatus(), //liststatus
         ITestAzureBlobFileSystemListStatus.class.getMethod("testListPath"));
-    testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, true), //open,
+    testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, true, JDK_HTTP_URL_CONNECTION), //open,
+        // read, write
+        ITestAbfsReadWriteAndSeek.class.getMethod("testReadAheadRequestID"));
+    testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, true, APACHE_HTTP_CLIENT), //open,
         // read, write
         ITestAbfsReadWriteAndSeek.class.getMethod("testReadAheadRequestID"));
-    testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, false), //read (bypassreadahead)
+    testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, false, JDK_HTTP_URL_CONNECTION), //read (bypassreadahead)
+        ITestAbfsReadWriteAndSeek.class
+            .getMethod("testReadAndWriteWithDifferentBufferSizesAndSeek"));
+    testClasses.put(new ITestAbfsReadWriteAndSeek(MIN_BUFFER_SIZE, false, APACHE_HTTP_CLIENT), //read (bypassreadahead)
         ITestAbfsReadWriteAndSeek.class
             .getMethod("testReadAndWriteWithDifferentBufferSizesAndSeek"));
     testClasses.put(new ITestAzureBlobFileSystemAppend(), //append

+ 5 - 4
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockDelegationSASTokenProvider.java

@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.net.MalformedURLException;
 import java.net.URL;
 import java.nio.charset.StandardCharsets;
+import java.time.Duration;
 import java.time.Instant;
 import java.util.ArrayList;
 import java.util.List;
@@ -33,7 +34,7 @@ import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
 import org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider;
 import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader;
-import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsJdkHttpOperation;
 import org.apache.hadoop.fs.azurebfs.utils.Base64;
 import org.apache.hadoop.fs.azurebfs.utils.DelegationSASGenerator;
 import org.apache.hadoop.fs.azurebfs.utils.SASGenerator;
@@ -106,11 +107,11 @@ public class MockDelegationSASTokenProvider implements SASTokenProvider {
     requestBody.append(ske);
     requestBody.append("</Expiry></KeyInfo>");
 
-    AbfsHttpOperation op = new AbfsHttpOperation(url, method, requestHeaders,
-            DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
+    AbfsJdkHttpOperation op = new AbfsJdkHttpOperation(url, method, requestHeaders,
+        Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
 
     byte[] requestBuffer = requestBody.toString().getBytes(StandardCharsets.UTF_8.toString());
-    op.sendRequest(requestBuffer, 0, requestBuffer.length);
+    op.sendPayload(requestBuffer, 0, requestBuffer.length);
 
     byte[] responseBuffer = new byte[4 * 1024];
     op.processResponse(responseBuffer, 0, responseBuffer.length);

+ 4 - 4
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientTestUtil.java

@@ -60,18 +60,19 @@ public final class AbfsClientTestUtil {
 
   public static void setMockAbfsRestOperationForListPathOperation(
       final AbfsClient spiedClient,
-      FunctionRaisingIOE<AbfsHttpOperation, AbfsHttpOperation> functionRaisingIOE)
+      FunctionRaisingIOE<AbfsJdkHttpOperation, AbfsJdkHttpOperation> functionRaisingIOE)
       throws Exception {
     ExponentialRetryPolicy exponentialRetryPolicy = Mockito.mock(ExponentialRetryPolicy.class);
     StaticRetryPolicy staticRetryPolicy = Mockito.mock(StaticRetryPolicy.class);
     AbfsThrottlingIntercept intercept = Mockito.mock(AbfsThrottlingIntercept.class);
-    AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class);
+    AbfsJdkHttpOperation httpOperation = Mockito.mock(AbfsJdkHttpOperation.class);
     AbfsRestOperation abfsRestOperation = Mockito.spy(new AbfsRestOperation(
         AbfsRestOperationType.ListPaths,
         spiedClient,
         HTTP_METHOD_GET,
         null,
-        new ArrayList<>()
+        new ArrayList<>(),
+        spiedClient.getAbfsConfiguration()
     ));
 
     Mockito.doReturn(abfsRestOperation).when(spiedClient).getAbfsRestOperation(
@@ -96,7 +97,6 @@ public final class AbfsClientTestUtil {
     HttpURLConnection httpURLConnection = Mockito.mock(HttpURLConnection.class);
     Mockito.doNothing().when(httpURLConnection)
         .setRequestProperty(nullable(String.class), nullable(String.class));
-    Mockito.doReturn(httpURLConnection).when(httpOperation).getConnection();
     Mockito.doReturn("").when(abfsRestOperation).getClientLatency();
     Mockito.doReturn(httpOperation).when(abfsRestOperation).createHttpOperation();
   }

+ 87 - 43
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java

@@ -20,23 +20,29 @@ package org.apache.hadoop.fs.azurebfs.services;
 
 import java.io.IOException;
 import java.lang.reflect.Field;
-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 java.util.regex.Pattern;
 
 import org.assertj.core.api.Assertions;
+import org.junit.Assume;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 import org.mockito.Mockito;
 
+import org.apache.hadoop.fs.FileSystem;
 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.constants.HttpOperationType;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsApacheHttpExpect100Exception;
 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;
@@ -45,6 +51,7 @@ 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 org.apache.http.HttpResponse;
 
 import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
 import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APPEND_ACTION;
@@ -58,6 +65,8 @@ import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X
 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.HttpOperationType.APACHE_HTTP_CLIENT;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.JDK_HTTP_URL_CONNECTION;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.eq;
@@ -84,6 +93,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST
  * Test useragent of abfs client.
  *
  */
+@RunWith(Parameterized.class)
 public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
 
   private static final String ACCOUNT_NAME = "bogusAccountName.dfs.core.windows.net";
@@ -97,6 +107,17 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
 
   private final Pattern userAgentStringPattern;
 
+  @Parameterized.Parameter
+  public HttpOperationType httpOperationType;
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Iterable<Object[]> params() {
+    return Arrays.asList(new Object[][]{
+        {HttpOperationType.JDK_HTTP_URL_CONNECTION},
+        {APACHE_HTTP_CLIENT}
+    });
+  }
+
   public ITestAbfsClient() throws Exception {
     StringBuilder regEx = new StringBuilder();
     regEx.append("^");
@@ -147,6 +168,7 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
 
   @Test
   public void verifyBasicInfo() throws Exception {
+    Assume.assumeTrue(JDK_HTTP_URL_CONNECTION == httpOperationType);
     final Configuration configuration = new Configuration();
     configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
     AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration,
@@ -176,6 +198,7 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
   @Test
   public void verifyUserAgentPrefix()
       throws IOException, IllegalAccessException {
+    Assume.assumeTrue(JDK_HTTP_URL_CONNECTION == httpOperationType);
     final Configuration configuration = new Configuration();
     configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
     configuration.set(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, FS_AZURE_USER_AGENT_PREFIX);
@@ -210,6 +233,7 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
   @Test
   public void verifyUserAgentExpectHeader()
           throws IOException, IllegalAccessException {
+    Assume.assumeTrue(JDK_HTTP_URL_CONNECTION == httpOperationType);
     final Configuration configuration = new Configuration();
     configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
     configuration.set(ConfigurationKeys.FS_AZURE_USER_AGENT_PREFIX_KEY, FS_AZURE_USER_AGENT_PREFIX);
@@ -236,6 +260,7 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
 
   @Test
   public void verifyUserAgentWithoutSSLProvider() throws Exception {
+    Assume.assumeTrue(JDK_HTTP_URL_CONNECTION == httpOperationType);
     final Configuration configuration = new Configuration();
     configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
     configuration.set(ConfigurationKeys.FS_AZURE_SSL_CHANNEL_MODE_KEY,
@@ -259,6 +284,7 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
 
   @Test
   public void verifyUserAgentClusterName() throws Exception {
+    Assume.assumeTrue(JDK_HTTP_URL_CONNECTION == httpOperationType);
     final String clusterName = "testClusterName";
     final Configuration configuration = new Configuration();
     configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
@@ -287,6 +313,7 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
 
   @Test
   public void verifyUserAgentClusterType() throws Exception {
+    Assume.assumeTrue(JDK_HTTP_URL_CONNECTION == httpOperationType);
     final String clusterType = "testClusterType";
     final Configuration configuration = new Configuration();
     configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
@@ -380,6 +407,8 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
         AbfsThrottlingInterceptFactory.getInstance(
             abfsConfig.getAccountName().substring(0,
                 abfsConfig.getAccountName().indexOf(DOT)), abfsConfig));
+    Mockito.doReturn(baseAbfsClientInstance.getAbfsApacheHttpClient()).when(client).getAbfsApacheHttpClient();
+
     // override baseurl
     client = ITestAbfsClient.setAbfsClientField(client, "abfsConfiguration",
         abfsConfig);
@@ -464,13 +493,14 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
       AbfsClient client,
       String method,
       URL url,
-      List<AbfsHttpHeader> requestHeaders) {
+      List<AbfsHttpHeader> requestHeaders, AbfsConfiguration abfsConfiguration) {
     return new AbfsRestOperation(
         type,
         client,
         method,
         url,
-        requestHeaders);
+        requestHeaders,
+        abfsConfiguration);
   }
 
   public static AccessTokenProvider getAccessTokenProvider(AbfsClient client) {
@@ -488,6 +518,14 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
     return b;
   }
 
+  @Override
+  public AzureBlobFileSystem getFileSystem(final Configuration configuration)
+      throws Exception {
+    Configuration conf = new Configuration(configuration);
+    conf.set(ConfigurationKeys.FS_AZURE_NETWORKING_LIBRARY, httpOperationType.toString());
+    return (AzureBlobFileSystem) FileSystem.newInstance(conf);
+  }
+
   /**
    * Test to verify that client retries append request without
    * expect header enabled if append with expect header enabled fails
@@ -497,9 +535,10 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
   @Test
   public void testExpectHundredContinue() throws Exception {
     // Get the filesystem.
-    final AzureBlobFileSystem fs = getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem(getRawConfiguration());
 
-    final Configuration configuration = new Configuration();
+    final Configuration configuration = fs.getAbfsStore().getAbfsConfiguration()
+        .getRawConfiguration();
     configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
     AbfsClient abfsClient = fs.getAbfsStore().getClient();
 
@@ -559,44 +598,49 @@ public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
         url,
         requestHeaders, buffer,
         appendRequestParameters.getoffset(),
-        appendRequestParameters.getLength(), null));
-
-    AbfsHttpOperation abfsHttpOperation = Mockito.spy(new AbfsHttpOperation(url,
-        HTTP_METHOD_PUT, requestHeaders, DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT));
-
-    // Sets the expect request property if expect header is enabled.
-    if (appendRequestParameters.isExpectHeaderEnabled()) {
-      Mockito.doReturn(HUNDRED_CONTINUE).when(abfsHttpOperation)
-          .getConnProperty(EXPECT);
-    }
-
-    HttpURLConnection urlConnection = mock(HttpURLConnection.class);
-    Mockito.doNothing().when(urlConnection).setRequestProperty(Mockito
-        .any(), Mockito.any());
-    Mockito.doReturn(HTTP_METHOD_PUT).when(urlConnection).getRequestMethod();
-    Mockito.doReturn(url).when(urlConnection).getURL();
-    Mockito.doReturn(urlConnection).when(abfsHttpOperation).getConnection();
-
-    Mockito.doNothing().when(abfsHttpOperation).setRequestProperty(Mockito
-        .any(), Mockito.any());
-    Mockito.doReturn(url).when(abfsHttpOperation).getConnUrl();
-
-    // Give user error code 404 when processResponse is called.
-    Mockito.doReturn(HTTP_METHOD_PUT).when(abfsHttpOperation).getConnRequestMethod();
-    Mockito.doReturn(HTTP_NOT_FOUND).when(abfsHttpOperation).getConnResponseCode();
-    Mockito.doReturn("Resource Not Found")
-        .when(abfsHttpOperation)
-        .getConnResponseMessage();
-
-    // Make the getOutputStream throw IOException to see it returns from the sendRequest correctly.
-    Mockito.doThrow(new ProtocolException(EXPECT_100_JDK_ERROR))
-        .when(abfsHttpOperation)
-        .getConnOutputStream();
-
-    // Sets the httpOperation for the rest operation.
-    Mockito.doReturn(abfsHttpOperation)
-        .when(op)
-        .createHttpOperation();
+        appendRequestParameters.getLength(), null, abfsConfig));
+
+    Mockito.doAnswer(answer -> {
+      AbfsHttpOperation httpOperation = Mockito.spy((AbfsHttpOperation) answer.callRealMethod());
+      // Sets the expect request property if expect header is enabled.
+      if (appendRequestParameters.isExpectHeaderEnabled()) {
+        Mockito.doReturn(HUNDRED_CONTINUE).when(httpOperation)
+            .getConnProperty(EXPECT);
+      }
+      Mockito.doNothing().when(httpOperation).setRequestProperty(Mockito
+          .any(), Mockito.any());
+      Mockito.doReturn(url).when(httpOperation).getConnUrl();
+
+      // Give user error code 404 when processResponse is called.
+      Mockito.doReturn(HTTP_METHOD_PUT).when(httpOperation).getMethod();
+      Mockito.doReturn(HTTP_NOT_FOUND).when(httpOperation).getStatusCode();
+      Mockito.doReturn("Resource Not Found")
+          .when(httpOperation)
+          .getConnResponseMessage();
+
+      if (httpOperation instanceof AbfsJdkHttpOperation) {
+        // Make the getOutputStream throw IOException to see it returns from the sendRequest correctly.
+        Mockito.doThrow(new ProtocolException(EXPECT_100_JDK_ERROR))
+            .when((AbfsJdkHttpOperation) httpOperation)
+            .getConnOutputStream();
+      }
+
+      if (httpOperation instanceof AbfsAHCHttpOperation) {
+        Mockito.doNothing()
+            .when((AbfsAHCHttpOperation) httpOperation)
+            .parseResponseHeaderAndBody(Mockito.any(byte[].class),
+                Mockito.anyInt(), Mockito.anyInt());
+        Mockito.doReturn(HTTP_NOT_FOUND)
+            .when((AbfsAHCHttpOperation) httpOperation)
+            .parseStatusCode(Mockito.nullable(
+                HttpResponse.class));
+        Mockito.doThrow(
+                new AbfsApacheHttpExpect100Exception(Mockito.mock(HttpResponse.class)))
+            .when((AbfsAHCHttpOperation) httpOperation)
+            .executeRequest();
+      }
+      return httpOperation;
+    }).when(op).createHttpOperation();
 
     // Mock the restOperation for the client.
     Mockito.doReturn(op)

+ 406 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsHttpClientRequestExecutor.java

@@ -0,0 +1,406 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URL;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.apache.http.HttpClientConnection;
+import org.apache.http.HttpEntityEnclosingRequest;
+import org.apache.http.HttpException;
+import org.apache.http.HttpRequest;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.protocol.HttpClientContext;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_NETWORKING_LIBRARY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public class ITestAbfsHttpClientRequestExecutor extends
+    AbstractAbfsIntegrationTest {
+
+  public ITestAbfsHttpClientRequestExecutor() throws Exception {
+    super();
+  }
+
+  /**
+   * Verify the correctness of expect 100 continue handling by ApacheHttpClient
+   * with AbfsManagedHttpRequestExecutor.
+   */
+  @Test
+  public void testExpect100ContinueHandling() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    Path path = new Path("/testExpect100ContinueHandling");
+
+    Configuration conf = new Configuration(fs.getConf());
+    conf.set(FS_AZURE_NETWORKING_LIBRARY, APACHE_HTTP_CLIENT.toString());
+    AzureBlobFileSystem fs2 = Mockito.spy(
+        (AzureBlobFileSystem) FileSystem.newInstance(conf));
+
+    AzureBlobFileSystemStore store = Mockito.spy(fs2.getAbfsStore());
+    Mockito.doReturn(store).when(fs2).getAbfsStore();
+
+    AbfsClient client = Mockito.spy(store.getClient());
+    Mockito.doReturn(client).when(store).getClient();
+
+    final int[] invocation = {0};
+    Mockito.doAnswer(answer -> {
+      AbfsRestOperation op = Mockito.spy(
+          (AbfsRestOperation) answer.callRealMethod());
+      final ConnectionInfo connectionInfo = new ConnectionInfo();
+
+      /*
+       * Assert that correct actions are taking place over the connection to handle
+       * expect100 assertions, failure and success.
+       *
+       * The test would make two calls to the server. The first two calls would
+       * be because of attempt to write in a non-existing file. The first call would have
+       * expect100 header, and the server would respond with 404. The second call would
+       * be a retry from AbfsOutputStream, and would not have expect100 header.
+       *
+       * The third call would be because of attempt to write in an existing file. The call
+       * would have expect100 assertion pass and would send the data.
+       *
+       * Following is the expectation from the first attempt:
+       * 1. sendHeaders should be called once. This is for expect100 assertion invocation.
+       * 2. receiveResponse should be called once. This is to receive expect100 assertion.
+       * 2. sendBody should not be called.
+       *
+       * Following is the expectation from the second attempt:
+       * 1. sendHeaders should be called once. This is not for expect100 assertion invocation.
+       * 2. sendBody should be called once. It will not have any expect100 assertion.
+       * Once headers are sent, body is sent.
+       * 3. receiveResponse should be called once. This is to receive the response from the server.
+       *
+       * Following is the expectation from the third attempt:
+       * 1. sendHeaders should be called once. This is for expect100 assertion invocation.
+       * 2. receiveResponse should be called. This is to receive the response from the server for expect100 assertion.
+       * 3. sendBody called as expect100 assertion is pass.
+       * 4. receiveResponse should be called. This is to receive the response from the server.
+       */
+      mockHttpOperationBehavior(connectionInfo, op);
+      Mockito.doAnswer(executeAnswer -> {
+        invocation[0]++;
+        final Throwable throwable;
+        if (invocation[0] == 3) {
+          executeAnswer.callRealMethod();
+          throwable = null;
+        } else {
+          throwable = intercept(IOException.class, () -> {
+            try {
+              executeAnswer.callRealMethod();
+            } catch (IOException ex) {
+              //This exception is expected to be thrown by the op.execute() method.
+              throw ex;
+            } catch (Throwable interceptedAssertedThrowable) {
+              //Any other throwable thrown by Mockito's callRealMethod would be
+              //considered as an assertion error.
+            }
+          });
+        }
+        /*
+         * The first call would be with expect headers, and expect 100 continue assertion has to happen which would fail.
+         * For expect100 assertion to happen, header IO happens before body IO. If assertion fails, no body IO happens.
+         * The second call would not be using expect headers.
+         *
+         * The third call would be with expect headers, and expect 100 continue assertion has to happen which would pass.
+         */
+        if (invocation[0] == 1) {
+          Assertions.assertThat(connectionInfo.getSendHeaderInvocation())
+              .isEqualTo(1);
+          Assertions.assertThat(connectionInfo.getSendBodyInvocation())
+              .isEqualTo(0);
+          Assertions.assertThat(connectionInfo.getReceiveResponseInvocation())
+              .isEqualTo(1);
+          Assertions.assertThat(
+                  connectionInfo.getReceiveResponseBodyInvocation())
+              .isEqualTo(1);
+        }
+        if (invocation[0] == 2) {
+          Assertions.assertThat(connectionInfo.getSendHeaderInvocation())
+              .isEqualTo(1);
+          Assertions.assertThat(connectionInfo.getSendBodyInvocation())
+              .isEqualTo(1);
+          Assertions.assertThat(connectionInfo.getReceiveResponseInvocation())
+              .isEqualTo(1);
+          Assertions.assertThat(
+                  connectionInfo.getReceiveResponseBodyInvocation())
+              .isEqualTo(1);
+        }
+        if (invocation[0] == 3) {
+          Assertions.assertThat(connectionInfo.getSendHeaderInvocation())
+              .isEqualTo(1);
+          Assertions.assertThat(connectionInfo.getSendBodyInvocation())
+              .isEqualTo(1);
+          Assertions.assertThat(connectionInfo.getReceiveResponseInvocation())
+              .isEqualTo(2);
+          Assertions.assertThat(
+                  connectionInfo.getReceiveResponseBodyInvocation())
+              .isEqualTo(1);
+        }
+        Assertions.assertThat(invocation[0]).isLessThanOrEqualTo(3);
+        if (throwable != null) {
+          throw throwable;
+        }
+        return null;
+      }).when(op).execute(Mockito.any(TracingContext.class));
+      return op;
+    }).when(client).getAbfsRestOperation(
+        Mockito.any(AbfsRestOperationType.class),
+        Mockito.anyString(),
+        Mockito.any(URL.class),
+        Mockito.anyList(),
+        Mockito.any(byte[].class),
+        Mockito.anyInt(),
+        Mockito.anyInt(),
+        Mockito.nullable(String.class));
+
+    final OutputStream os = fs2.create(path);
+    fs.delete(path, true);
+    intercept(FileNotFoundException.class, () -> {
+      /*
+       * This would lead to two server calls.
+       * First call would be with expect headers, and expect 100 continue
+       *  assertion has to happen which would fail with 404.
+       * Second call would be a retry from AbfsOutputStream, and would not be using expect headers.
+       */
+      os.write(1);
+      os.close();
+    });
+
+    final OutputStream os2 = fs2.create(path);
+    /*
+     * This would lead to third server call. This would be with expect headers,
+     * and the expect 100 continue assertion would pass.
+     */
+    os2.write(1);
+    os2.close();
+  }
+
+  /**
+   * Creates a mock of HttpOperation that would be returned for AbfsRestOperation
+   * to use to execute server call. To make call via ApacheHttpClient, an object
+   * of {@link HttpClientContext} is required. This method would create a mock
+   * of HttpClientContext that would be able to register the actions taken on
+   * {@link HttpClientConnection} object. This would help in asserting the
+   * order of actions taken on the connection object for making an append call with
+   * expect100 header.
+   */
+  private void mockHttpOperationBehavior(final ConnectionInfo connectionInfo,
+      final AbfsRestOperation op) throws IOException {
+    Mockito.doAnswer(httpOpCreationAnswer -> {
+      AbfsAHCHttpOperation httpOperation = Mockito.spy(
+          (AbfsAHCHttpOperation) httpOpCreationAnswer.callRealMethod());
+
+      Mockito.doAnswer(createContextAnswer -> {
+            AbfsManagedHttpClientContext context = Mockito.spy(
+                (AbfsManagedHttpClientContext) createContextAnswer.callRealMethod());
+            Mockito.doAnswer(connectionSpyIntercept -> {
+              return interceptedConn(connectionInfo,
+                  (HttpClientConnection) connectionSpyIntercept.getArgument(0));
+            }).when(context).interceptConnectionActivity(Mockito.any(
+                HttpClientConnection.class));
+            return context;
+          })
+          .when(httpOperation).getHttpClientContext();
+      return httpOperation;
+    }).when(op).createHttpOperation();
+  }
+
+  private HttpClientConnection interceptedConn(final ConnectionInfo connectionInfo,
+      final HttpClientConnection connection) throws IOException, HttpException {
+    HttpClientConnection interceptedConn = Mockito.spy(connection);
+
+    Mockito.doAnswer(answer -> {
+      connectionInfo.incrementSendHeaderInvocation();
+      long start = System.currentTimeMillis();
+      Object result = answer.callRealMethod();
+      connectionInfo.addSendTime(System.currentTimeMillis() - start);
+      return result;
+    }).when(interceptedConn).sendRequestHeader(Mockito.any(HttpRequest.class));
+
+    Mockito.doAnswer(answer -> {
+      connectionInfo.incrementSendBodyInvocation();
+      long start = System.currentTimeMillis();
+      Object result = answer.callRealMethod();
+      connectionInfo.addSendTime(System.currentTimeMillis() - start);
+      return result;
+    }).when(interceptedConn).sendRequestEntity(Mockito.any(
+        HttpEntityEnclosingRequest.class));
+
+    Mockito.doAnswer(answer -> {
+      connectionInfo.incrementReceiveResponseInvocation();
+      long start = System.currentTimeMillis();
+      Object result = answer.callRealMethod();
+      connectionInfo.addReadTime(System.currentTimeMillis() - start);
+      return result;
+    }).when(interceptedConn).receiveResponseHeader();
+
+    Mockito.doAnswer(answer -> {
+      connectionInfo.incrementReceiveResponseBodyInvocation();
+      long start = System.currentTimeMillis();
+      Object result = answer.callRealMethod();
+      connectionInfo.addReadTime(System.currentTimeMillis() - start);
+      return result;
+    }).when(interceptedConn).receiveResponseEntity(Mockito.any(
+        HttpResponse.class));
+    return interceptedConn;
+  }
+
+  @Test
+  public void testConnectionReadRecords() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    Path path = new Path("/testConnectionRecords");
+
+    Configuration conf = new Configuration(fs.getConf());
+    conf.set(FS_AZURE_NETWORKING_LIBRARY, APACHE_HTTP_CLIENT.toString());
+    AzureBlobFileSystem fs2 = Mockito.spy(
+        (AzureBlobFileSystem) FileSystem.newInstance(conf));
+
+    AzureBlobFileSystemStore store = Mockito.spy(fs2.getAbfsStore());
+    Mockito.doReturn(store).when(fs2).getAbfsStore();
+
+    AbfsClient client = Mockito.spy(store.getClient());
+    Mockito.doReturn(client).when(store).getClient();
+
+    try (OutputStream os = fs.create(path)) {
+      os.write(1);
+    }
+
+    InputStream is = fs2.open(path);
+
+    Mockito.doAnswer(answer -> {
+      AbfsRestOperation op = Mockito.spy(
+          (AbfsRestOperation) answer.callRealMethod());
+      final ConnectionInfo connectionInfo = new ConnectionInfo();
+      mockHttpOperationBehavior(connectionInfo, op);
+      Mockito.doAnswer(executeAnswer -> {
+        executeAnswer.callRealMethod();
+        Assertions.assertThat(connectionInfo.getSendHeaderInvocation())
+            .isEqualTo(1);
+        Assertions.assertThat(connectionInfo.getSendBodyInvocation())
+            .isEqualTo(0);
+        Assertions.assertThat(connectionInfo.getReceiveResponseInvocation())
+            .isEqualTo(1);
+        Assertions.assertThat(connectionInfo.getReceiveResponseBodyInvocation())
+            .isEqualTo(1);
+        return null;
+      }).when(op).execute(Mockito.any(TracingContext.class));
+      return op;
+    }).when(client).getAbfsRestOperation(
+        Mockito.any(AbfsRestOperationType.class),
+        Mockito.anyString(),
+        Mockito.any(URL.class),
+        Mockito.anyList(),
+        Mockito.any(byte[].class),
+        Mockito.anyInt(),
+        Mockito.anyInt(),
+        Mockito.nullable(String.class));
+
+    is.read();
+    is.close();
+  }
+
+  private static class ConnectionInfo {
+
+    private long connectTime;
+
+    private long readTime;
+
+    private long sendTime;
+
+    private int sendHeaderInvocation;
+
+    private int sendBodyInvocation;
+
+    private int receiveResponseInvocation;
+
+    private int receiveResponseBodyInvocation;
+
+    private void incrementSendHeaderInvocation() {
+      sendHeaderInvocation++;
+    }
+
+    private void incrementSendBodyInvocation() {
+      sendBodyInvocation++;
+    }
+
+    private void incrementReceiveResponseInvocation() {
+      receiveResponseInvocation++;
+    }
+
+    private void incrementReceiveResponseBodyInvocation() {
+      receiveResponseBodyInvocation++;
+    }
+
+    private void addConnectTime(long connectTime) {
+      this.connectTime += connectTime;
+    }
+
+    private void addReadTime(long readTime) {
+      this.readTime += readTime;
+    }
+
+    private void addSendTime(long sendTime) {
+      this.sendTime += sendTime;
+    }
+
+    private long getConnectTime() {
+      return connectTime;
+    }
+
+    private long getReadTime() {
+      return readTime;
+    }
+
+    private long getSendTime() {
+      return sendTime;
+    }
+
+    private int getSendHeaderInvocation() {
+      return sendHeaderInvocation;
+    }
+
+    private int getSendBodyInvocation() {
+      return sendBodyInvocation;
+    }
+
+    private int getReceiveResponseInvocation() {
+      return receiveResponseInvocation;
+    }
+
+    private int getReceiveResponseBodyInvocation() {
+      return receiveResponseBodyInvocation;
+    }
+  }
+}

+ 38 - 8
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsOutputStream.java

@@ -23,9 +23,12 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
+import java.util.Arrays;
 
 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;
@@ -36,6 +39,7 @@ import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
 import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
 import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
 import org.apache.hadoop.test.LambdaTestUtils;
 
 import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED;
@@ -43,15 +47,36 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE
 /**
  * Test create operation.
  */
+@RunWith(Parameterized.class)
 public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
 
   private static final int TEST_EXECUTION_TIMEOUT = 2 * 60 * 1000;
   private static final String TEST_FILE_PATH = "testfile";
 
+  @Parameterized.Parameter
+  public HttpOperationType httpOperationType;
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Iterable<Object[]> params() {
+    return Arrays.asList(new Object[][]{
+        {HttpOperationType.JDK_HTTP_URL_CONNECTION},
+        {HttpOperationType.APACHE_HTTP_CLIENT}
+    });
+  }
+
+
   public ITestAbfsOutputStream() throws Exception {
     super();
   }
 
+  @Override
+  public AzureBlobFileSystem getFileSystem(final Configuration configuration)
+      throws Exception {
+    Configuration conf = new Configuration(configuration);
+    conf.set(ConfigurationKeys.FS_AZURE_NETWORKING_LIBRARY, httpOperationType.toString());
+    return (AzureBlobFileSystem) FileSystem.newInstance(conf);
+  }
+
   @Test
   public void testMaxRequestsAndQueueCapacityDefaults() throws Exception {
     Configuration conf = getRawConfiguration();
@@ -158,8 +183,7 @@ public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
   public void testExpect100ContinueFailureInAppend() throws Exception {
     Configuration configuration = new Configuration(getRawConfiguration());
     configuration.set(FS_AZURE_ACCOUNT_IS_EXPECT_HEADER_ENABLED, "true");
-    AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(
-        configuration);
+    AzureBlobFileSystem fs = getFileSystem(configuration);
     Path path = new Path("/testFile");
     AbfsOutputStream os = Mockito.spy(
         (AbfsOutputStream) fs.create(path).getWrappedStream());
@@ -175,17 +199,23 @@ public class ITestAbfsOutputStream extends AbstractAbfsIntegrationTest {
     Assertions.assertThat(httpOpForAppendTest[0].getConnectionDisconnectedOnError())
         .describedAs("First try from AbfsClient will have expect-100 "
             + "header and should fail with expect-100 error.").isTrue();
-    Mockito.verify(httpOpForAppendTest[0], Mockito.times(0))
-        .processConnHeadersAndInputStreams(Mockito.any(byte[].class),
-            Mockito.anyInt(), Mockito.anyInt());
+    if (httpOpForAppendTest[0] instanceof AbfsJdkHttpOperation) {
+      Mockito.verify((AbfsJdkHttpOperation) httpOpForAppendTest[0],
+              Mockito.times(0))
+          .processConnHeadersAndInputStreams(Mockito.any(byte[].class),
+              Mockito.anyInt(), Mockito.anyInt());
+    }
 
     Assertions.assertThat(httpOpForAppendTest[1].getConnectionDisconnectedOnError())
         .describedAs("The retried operation from AbfsClient should not "
             + "fail with expect-100 error. The retried operation does not have"
             + "expect-100 header.").isFalse();
-    Mockito.verify(httpOpForAppendTest[1], Mockito.times(1))
-        .processConnHeadersAndInputStreams(Mockito.any(byte[].class),
-            Mockito.anyInt(), Mockito.anyInt());
+    if (httpOpForAppendTest[1] instanceof AbfsJdkHttpOperation) {
+      Mockito.verify((AbfsJdkHttpOperation) httpOpForAppendTest[1],
+              Mockito.times(1))
+          .processConnHeadersAndInputStreams(Mockito.any(byte[].class),
+              Mockito.anyInt(), Mockito.anyInt());
+    }
   }
 
   private void mockSetupForAppend(final AbfsHttpOperation[] httpOpForAppendTest,

+ 90 - 48
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsRestOperation.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.fs.azurebfs.services;
 
 import java.io.IOException;
 import java.io.OutputStream;
-import java.net.HttpURLConnection;
 import java.net.ProtocolException;
 import java.net.URL;
 import java.util.Arrays;
@@ -34,16 +33,21 @@ import org.junit.runners.Parameterized;
 import org.mockito.Mockito;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 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.ConfigurationKeys;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+import org.apache.hadoop.fs.azurebfs.constants.HttpOperationType;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsApacheHttpExpect100Exception;
 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 org.apache.http.HttpResponse;
 
 import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
 import static java.net.HttpURLConnection.HTTP_OK;
@@ -53,17 +57,16 @@ import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EXPECT_1
 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.FileSystemConfigurations.DEFAULT_HTTP_CONNECTION_TIMEOUT;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_HTTP_READ_TIMEOUT;
 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;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.JDK_HTTP_URL_CONNECTION;
 import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.EGRESS_OVER_ACCOUNT_LIMIT;
 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;
 
@@ -98,6 +101,9 @@ public class ITestAbfsRestOperation extends AbstractAbfsIntegrationTest {
   @Parameterized.Parameter(3)
   public ErrorType errorType;
 
+  @Parameterized.Parameter(4)
+  public HttpOperationType httpOperationType;
+
   // The intercept.
   private AbfsThrottlingIntercept intercept;
 
@@ -108,15 +114,26 @@ public class ITestAbfsRestOperation extends AbstractAbfsIntegrationTest {
     HTTP_EXPECTATION_FAILED = 417,
     HTTP_ERROR = 0.
    */
-  @Parameterized.Parameters(name = "expect={0}-code={1}-ErrorType={3}")
+  @Parameterized.Parameters(name = "expect={0}-code={1}-ErrorType={3}=NetLib={4}")
   public static Iterable<Object[]> params() {
     return Arrays.asList(new Object[][]{
-        {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}
+        {true, HTTP_OK, "OK", ErrorType.WRITE, JDK_HTTP_URL_CONNECTION},
+        {true, HTTP_OK, "OK", ErrorType.WRITE, APACHE_HTTP_CLIENT},
+
+        {false, HTTP_OK, "OK", ErrorType.WRITE, JDK_HTTP_URL_CONNECTION},
+        {false, HTTP_OK, "OK", ErrorType.WRITE, APACHE_HTTP_CLIENT},
+
+        {true, HTTP_UNAVAILABLE, "ServerBusy", ErrorType.OUTPUTSTREAM, JDK_HTTP_URL_CONNECTION},
+        {true, HTTP_UNAVAILABLE, "ServerBusy", ErrorType.OUTPUTSTREAM, APACHE_HTTP_CLIENT},
+
+        {true, HTTP_NOT_FOUND, "Resource Not Found", ErrorType.OUTPUTSTREAM, JDK_HTTP_URL_CONNECTION},
+        {true, HTTP_NOT_FOUND, "Resource Not Found", ErrorType.OUTPUTSTREAM, APACHE_HTTP_CLIENT},
+
+        {true, HTTP_EXPECTATION_FAILED, "Expectation Failed", ErrorType.OUTPUTSTREAM, JDK_HTTP_URL_CONNECTION},
+        {true, HTTP_EXPECTATION_FAILED, "Expectation Failed", ErrorType.OUTPUTSTREAM, APACHE_HTTP_CLIENT},
+
+        {true, HTTP_ERROR, "Error", ErrorType.OUTPUTSTREAM, JDK_HTTP_URL_CONNECTION},
+        {true, HTTP_ERROR, "Error", ErrorType.OUTPUTSTREAM, APACHE_HTTP_CLIENT}
     });
   }
 
@@ -135,15 +152,23 @@ public class ITestAbfsRestOperation extends AbstractAbfsIntegrationTest {
     return b;
   }
 
+  @Override
+  public AzureBlobFileSystem getFileSystem(final Configuration configuration)
+      throws Exception {
+    Configuration conf = new Configuration(configuration);
+    conf.set(ConfigurationKeys.FS_AZURE_NETWORKING_LIBRARY, httpOperationType.toString());
+    return (AzureBlobFileSystem) FileSystem.newInstance(conf);
+  }
+
   /**
    * Gives the AbfsRestOperation.
    * @return abfsRestOperation.
    */
   private AbfsRestOperation getRestOperation() throws Exception {
     // Get the filesystem.
-    final AzureBlobFileSystem fs = getFileSystem();
+    final AzureBlobFileSystem fs = getFileSystem(getRawConfiguration());
 
-    final Configuration configuration = new Configuration();
+    final Configuration configuration = fs.getConf();
     configuration.addResource(TEST_CONFIGURATION_FILE_NAME);
     AbfsClient abfsClient = fs.getAbfsStore().getClient();
 
@@ -196,36 +221,38 @@ public class ITestAbfsRestOperation extends AbstractAbfsIntegrationTest {
     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(
+    final AbfsRestOperation op = Mockito.spy(new AbfsRestOperation(
         AbfsRestOperationType.Append,
         testClient,
         HTTP_METHOD_PUT,
         url,
         requestHeaders, buffer,
         appendRequestParameters.getoffset(),
-        appendRequestParameters.getLength(), null));
-
-    AbfsHttpOperation abfsHttpOperation = Mockito.spy(new AbfsHttpOperation(url, HTTP_METHOD_PUT, requestHeaders,
-            DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT));
+        appendRequestParameters.getLength(), null, abfsConfig));
+
+    Mockito.doAnswer(answer -> {
+      AbfsHttpOperation httpOperation = Mockito.spy(
+          (AbfsHttpOperation) answer.callRealMethod());
+      mockHttpOperation(appendRequestParameters, buffer, url, httpOperation);
+      Mockito.doReturn(httpOperation).when(op).getResult();
+      return httpOperation;
+    }).when(op).createHttpOperation();
+    return op;
+  }
 
+  private void mockHttpOperation(final AppendRequestParameters appendRequestParameters,
+      final byte[] buffer,
+      final URL url,
+      final AbfsHttpOperation httpOperation) throws IOException {
     // Sets the expect request property if expect header is enabled.
     if (expectHeaderEnabled) {
       Mockito.doReturn(HUNDRED_CONTINUE)
-          .when(abfsHttpOperation)
+          .when(httpOperation)
           .getConnProperty(EXPECT);
     }
 
-    HttpURLConnection urlConnection = mock(HttpURLConnection.class);
-    Mockito.doNothing().when(urlConnection).setRequestProperty(Mockito
+    Mockito.doNothing().when(httpOperation).setRequestProperty(Mockito
         .any(), Mockito.any());
-    Mockito.doReturn(HTTP_METHOD_PUT).when(urlConnection).getRequestMethod();
-    Mockito.doReturn(url).when(urlConnection).getURL();
-    Mockito.doReturn(urlConnection).when(abfsHttpOperation).getConnection();
-
-    Mockito.doNothing().when(abfsHttpOperation).setRequestProperty(Mockito
-        .any(), Mockito.any());
-    Mockito.doReturn(url).when(abfsHttpOperation).getConnUrl();
-    Mockito.doReturn(HTTP_METHOD_PUT).when(abfsHttpOperation).getConnRequestMethod();
 
     switch (errorType) {
     case OUTPUTSTREAM:
@@ -233,28 +260,51 @@ public class ITestAbfsRestOperation extends AbstractAbfsIntegrationTest {
       // 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(abfsHttpOperation).getConnResponseCode();
+
+      Mockito.doReturn(responseCode).when(httpOperation).getStatusCode();
       if (responseCode == HTTP_UNAVAILABLE) {
         Mockito.doReturn(EGRESS_OVER_ACCOUNT_LIMIT.getErrorMessage())
-            .when(abfsHttpOperation)
+            .when(httpOperation)
             .getStorageErrorMessage();
       }
       Mockito.doReturn(responseMessage)
-          .when(abfsHttpOperation)
+          .when(httpOperation)
           .getConnResponseMessage();
-      Mockito.doThrow(new ProtocolException(EXPECT_100_JDK_ERROR))
-          .when(abfsHttpOperation)
-          .getConnOutputStream();
+      if (httpOperation instanceof AbfsJdkHttpOperation) {
+        Mockito.doThrow(new ProtocolException(EXPECT_100_JDK_ERROR))
+            .when((AbfsJdkHttpOperation) httpOperation)
+            .getConnOutputStream();
+      }
+      if (httpOperation instanceof AbfsAHCHttpOperation) {
+        Mockito.doNothing()
+            .when((AbfsAHCHttpOperation) httpOperation)
+            .parseResponseHeaderAndBody(Mockito.any(byte[].class),
+                Mockito.anyInt(), Mockito.anyInt());
+        Mockito.doReturn(HTTP_NOT_FOUND)
+            .when((AbfsAHCHttpOperation) httpOperation)
+            .parseStatusCode(Mockito.nullable(
+                HttpResponse.class));
+        Mockito.doThrow(
+                new AbfsApacheHttpExpect100Exception(Mockito.mock(HttpResponse.class)))
+            .when((AbfsAHCHttpOperation) httpOperation).executeRequest();
+      }
       break;
     case WRITE:
       // If write() throws IOException and Expect Header is
       // enabled or not, it should throw back the exception.
+      if (httpOperation instanceof AbfsAHCHttpOperation) {
+        Mockito.doThrow(new IOException())
+            .when((AbfsAHCHttpOperation) httpOperation).executeRequest();
+        return;
+      }
       OutputStream outputStream = Mockito.spy(new OutputStream() {
         @Override
         public void write(final int i) throws IOException {
         }
       });
-      Mockito.doReturn(outputStream).when(abfsHttpOperation).getConnOutputStream();
+      Mockito.doReturn(outputStream)
+          .when((AbfsJdkHttpOperation) httpOperation)
+          .getConnOutputStream();
       Mockito.doThrow(new IOException())
           .when(outputStream)
           .write(buffer, appendRequestParameters.getoffset(),
@@ -263,12 +313,6 @@ public class ITestAbfsRestOperation extends AbstractAbfsIntegrationTest {
     default:
       break;
     }
-
-    // Sets the httpOperation for the rest operation.
-    Mockito.doReturn(abfsHttpOperation)
-        .when(op)
-        .createHttpOperation();
-    return op;
   }
 
   void assertTraceContextState(int retryCount, int assertRetryCount, int bytesSent, int assertBytesSent,
@@ -295,8 +339,6 @@ public class ITestAbfsRestOperation extends AbstractAbfsIntegrationTest {
   public void testExpectHundredContinue() throws Exception {
     // Gets the AbfsRestOperation.
     AbfsRestOperation op = getRestOperation();
-    AbfsHttpOperation httpOperation = op.createHttpOperation();
-
     TracingContext tracingContext = Mockito.spy(new TracingContext("abcd",
         "abcde", FSOperationType.APPEND,
         TracingHeaderFormat.ALL_ID_FORMAT, null));
@@ -311,7 +353,7 @@ public class ITestAbfsRestOperation extends AbstractAbfsIntegrationTest {
           () -> op.execute(tracingContext));
 
       // Asserting update of metrics and retries.
-      assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, httpOperation.getBytesSent(), BUFFER_LENGTH,
+      assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, op.getResult().getBytesSent(), BUFFER_LENGTH,
               0, 0);
       break;
     case OUTPUTSTREAM:
@@ -322,8 +364,8 @@ public class ITestAbfsRestOperation extends AbstractAbfsIntegrationTest {
             () -> op.execute(tracingContext));
 
         // Asserting update of metrics and retries.
-        assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, httpOperation.getBytesSent(), ZERO,
-                httpOperation.getExpectedBytesToBeSent(), BUFFER_LENGTH);
+        assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, op.getResult().getBytesSent(), ZERO,
+                op.getResult().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.
@@ -336,7 +378,7 @@ public class ITestAbfsRestOperation extends AbstractAbfsIntegrationTest {
             () -> op.execute(tracingContext));
 
         // Asserting update of metrics and retries.
-        assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, httpOperation.getBytesSent(),
+        assertTraceContextState(tracingContext.getRetryCount(), REDUCED_RETRY_COUNT, op.getResult().getBytesSent(),
                 ZERO, 0, 0);
 
         // Verifies that update Metrics call is made for ErrorType case and for the first without retry +

+ 63 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestApacheClientConnectionPool.java

@@ -0,0 +1,63 @@
+/**
+ * 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.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ClosedIOException;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsDriverException;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.KEEP_ALIVE_CACHE_CLOSED;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_NETWORKING_LIBRARY;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.apache.hadoop.test.LambdaTestUtils.verifyCause;
+
+/**
+ * This test class tests the exception handling in ABFS thrown by the
+ * {@link KeepAliveCache}.
+ */
+public class ITestApacheClientConnectionPool extends
+    AbstractAbfsIntegrationTest {
+
+  public ITestApacheClientConnectionPool() throws Exception {
+    super();
+  }
+
+  @Test
+  public void testKacIsClosed() throws Throwable {
+    Configuration configuration = new Configuration(getRawConfiguration());
+    configuration.set(FS_AZURE_NETWORKING_LIBRARY, APACHE_HTTP_CLIENT.name());
+    try (AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(
+        configuration)) {
+      KeepAliveCache kac = fs.getAbfsStore().getClient().getKeepAliveCache();
+      kac.close();
+      AbfsDriverException ex = intercept(AbfsDriverException.class,
+          KEEP_ALIVE_CACHE_CLOSED, () -> {
+            fs.create(new Path("/test"));
+          });
+      verifyCause(ClosedIOException.class, ex);
+    }
+  }
+}

+ 18 - 18
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsPerfTracker.java

@@ -76,8 +76,8 @@ public final class TestAbfsPerfTracker {
 
     try (AbfsPerfInfo tracker = new AbfsPerfInfo(abfsPerfTracker, "disablingCaller",
             "disablingCallee")) {
-      AbfsHttpOperation op = new AbfsHttpOperation(url, "GET", new ArrayList<>(),
-              DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
+      AbfsJdkHttpOperation op = new AbfsJdkHttpOperation(url, "GET", new ArrayList<>(),
+          Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
       tracker.registerResult(op).registerSuccess(true);
     }
 
@@ -95,8 +95,8 @@ public final class TestAbfsPerfTracker {
     assertThat(latencyDetails).describedAs("AbfsPerfTracker should be empty").isNull();
 
     List<Callable<Integer>> tasks = new ArrayList<>();
-    AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList<>(),
-            DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
+    AbfsJdkHttpOperation httpOperation = new AbfsJdkHttpOperation(url, "GET", new ArrayList<>(),
+        Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
 
     for (int i = 0; i < numTasks; i++) {
       tasks.add(() -> {
@@ -135,8 +135,8 @@ public final class TestAbfsPerfTracker {
     assertThat(latencyDetails).describedAs("AbfsPerfTracker should be empty").isNull();
 
     List<Callable<Integer>> tasks = new ArrayList<>();
-    AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList<>(),
-            DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
+    AbfsJdkHttpOperation httpOperation = new AbfsJdkHttpOperation(url, "GET", new ArrayList<>(),
+        Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
 
     for (int i = 0; i < numTasks; i++) {
       tasks.add(() -> {
@@ -175,8 +175,8 @@ public final class TestAbfsPerfTracker {
     long aggregateLatency = 0;
     AbfsPerfTracker abfsPerfTracker = new AbfsPerfTracker(accountName, filesystemName, false);
     List<Callable<Long>> tasks = new ArrayList<>();
-    final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList<>(),
-            DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
+    final AbfsJdkHttpOperation httpOperation = new AbfsJdkHttpOperation(url, "GET", new ArrayList<>(),
+        Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
 
     for (int i = 0; i < numTasks; i++) {
       tasks.add(() -> {
@@ -211,8 +211,8 @@ public final class TestAbfsPerfTracker {
     long aggregateLatency = 0;
     AbfsPerfTracker abfsPerfTracker = new AbfsPerfTracker(accountName, filesystemName, false);
     List<Callable<Long>> tasks = new ArrayList<>();
-    final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList<>(),
-            DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
+    final AbfsJdkHttpOperation httpOperation = new AbfsJdkHttpOperation(url, "GET", new ArrayList<>(),
+        Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
 
     for (int i = 0; i < numTasks; i++) {
       tasks.add(() -> {
@@ -276,8 +276,8 @@ public final class TestAbfsPerfTracker {
     long aggregateLatency = 0;
     AbfsPerfTracker abfsPerfTracker = new AbfsPerfTracker(accountName, filesystemName, true);
     List<Callable<Long>> tasks = new ArrayList<>();
-    final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList<>(),
-            DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
+    final AbfsJdkHttpOperation httpOperation = new AbfsJdkHttpOperation(url, "GET", new ArrayList<>(),
+        Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
 
     for (int i = 0; i < numTasks; i++) {
       tasks.add(() -> {
@@ -311,8 +311,8 @@ public final class TestAbfsPerfTracker {
     long aggregateLatency = 0;
     AbfsPerfTracker abfsPerfTracker = new AbfsPerfTracker(accountName, filesystemName, true);
     List<Callable<Long>> tasks = new ArrayList<>();
-    final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList<>(),
-            DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
+    final AbfsJdkHttpOperation httpOperation = new AbfsJdkHttpOperation(url, "GET", new ArrayList<>(),
+        Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
 
     for (int i = 0; i < numTasks; i++) {
       tasks.add(() -> {
@@ -372,8 +372,8 @@ public final class TestAbfsPerfTracker {
     Instant testInstant = Instant.now();
     AbfsPerfTracker abfsPerfTrackerDisabled = new AbfsPerfTracker(accountName, filesystemName, false);
     AbfsPerfTracker abfsPerfTrackerEnabled = new AbfsPerfTracker(accountName, filesystemName, true);
-    final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList<AbfsHttpHeader>(),
-            DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
+    final AbfsJdkHttpOperation httpOperation = new AbfsJdkHttpOperation(url, "GET", new ArrayList<AbfsHttpHeader>(),
+        Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
 
     verifyNoException(abfsPerfTrackerDisabled);
     verifyNoException(abfsPerfTrackerEnabled);
@@ -381,8 +381,8 @@ public final class TestAbfsPerfTracker {
 
   private void verifyNoException(AbfsPerfTracker abfsPerfTracker) throws Exception {
     Instant testInstant = Instant.now();
-    final AbfsHttpOperation httpOperation = new AbfsHttpOperation(url, "GET", new ArrayList<AbfsHttpHeader>(),
-            DEFAULT_HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_READ_TIMEOUT);
+    final AbfsJdkHttpOperation httpOperation = new AbfsJdkHttpOperation(url, "GET", new ArrayList<AbfsHttpHeader>(),
+        Duration.ofMillis(DEFAULT_HTTP_CONNECTION_TIMEOUT), Duration.ofMillis(DEFAULT_HTTP_READ_TIMEOUT));
 
     try (
             AbfsPerfInfo tracker01 = new AbfsPerfInfo(abfsPerfTracker, null, null);

+ 16 - 11
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java

@@ -99,14 +99,14 @@ public class TestAbfsRenameRetryRecovery extends AbstractAbfsIntegrationTest {
     // SuccessFul Result.
     AbfsRestOperation successOp =
         new AbfsRestOperation(AbfsRestOperationType.RenamePath, mockClient,
-            HTTP_METHOD_PUT, null, null);
+            HTTP_METHOD_PUT, null, null, mockClient.getAbfsConfiguration());
     AbfsClientRenameResult successResult = mock(AbfsClientRenameResult.class);
     doReturn(successOp).when(successResult).getOp();
     when(successResult.isIncompleteMetadataState()).thenReturn(false);
 
     // Failed Result.
     AbfsRestOperation failedOp = new AbfsRestOperation(AbfsRestOperationType.RenamePath, mockClient,
-        HTTP_METHOD_PUT, null, null);
+        HTTP_METHOD_PUT, null, null, mockClient.getAbfsConfiguration());
     AbfsClientRenameResult recoveredMetaDataIncompleteResult =
         mock(AbfsClientRenameResult.class);
 
@@ -167,12 +167,17 @@ public class TestAbfsRenameRetryRecovery extends AbstractAbfsIntegrationTest {
     Mockito.doReturn(spiedConf).when(spyClient).getAbfsConfiguration();
 
     Mockito.doAnswer(answer -> {
-      AbfsRestOperation op = new AbfsRestOperation(AbfsRestOperationType.RenamePath,
-              spyClient, HTTP_METHOD_PUT, answer.getArgument(0), answer.getArgument(1));
-      AbfsRestOperation spiedOp = Mockito.spy(op);
-      addSpyBehavior(spiedOp, op, spyClient);
-      return spiedOp;
-    }).when(spyClient).createRenameRestOperation(Mockito.any(URL.class), anyList());
+          AbfsRestOperation op = new AbfsRestOperation(
+              AbfsRestOperationType.RenamePath,
+              spyClient, HTTP_METHOD_PUT, answer.getArgument(0),
+              answer.getArgument(1),
+              spyClient.getAbfsConfiguration());
+          AbfsRestOperation spiedOp = Mockito.spy(op);
+          addSpyBehavior(spiedOp, op, spyClient);
+          return spiedOp;
+        })
+        .when(spyClient)
+        .createRenameRestOperation(Mockito.any(URL.class), anyList());
 
     return spyClient;
 
@@ -195,7 +200,7 @@ public class TestAbfsRenameRetryRecovery extends AbstractAbfsIntegrationTest {
     AbfsHttpOperation normalOp1 = normalRestOp.createHttpOperation();
     executeThenFail(client, normalRestOp, failingOperation, normalOp1);
     AbfsHttpOperation normalOp2 = normalRestOp.createHttpOperation();
-    normalOp2.getConnection().setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION,
+    normalOp2.setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION,
             client.getAccessToken());
 
     Mockito.doReturn(failingOperation).doReturn(normalOp2).when(spiedRestOp).createHttpOperation();
@@ -221,14 +226,14 @@ public class TestAbfsRenameRetryRecovery extends AbstractAbfsIntegrationTest {
       final int offset = answer.getArgument(1);
       final int length = answer.getArgument(2);
       normalRestOp.signRequest(normalOp, length);
-      normalOp.sendRequest(buffer, offset, length);
+      normalOp.sendPayload(buffer, offset, length);
       normalOp.processResponse(buffer, offset, length);
       LOG.info("Actual outcome is {} \"{}\" \"{}\"; injecting failure",
           normalOp.getStatusCode(),
           normalOp.getStorageErrorCode(),
           normalOp.getStorageErrorMessage());
       throw new SocketException("connection-reset");
-    }).when(failingOperation).sendRequest(Mockito.nullable(byte[].class),
+    }).when(failingOperation).sendPayload(Mockito.nullable(byte[].class),
         Mockito.nullable(int.class), Mockito.nullable(int.class));
 
   }

+ 11 - 4
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRestOperationMockFailures.java

@@ -29,6 +29,7 @@ import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.stubbing.Stubber;
 
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
 import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
 
@@ -208,7 +209,8 @@ public class TestAbfsRestOperationMockFailures {
         abfsClient,
         "PUT",
         null,
-        new ArrayList<>()
+        new ArrayList<>(),
+        Mockito.mock(AbfsConfiguration.class)
     ));
 
     AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class);
@@ -225,6 +227,8 @@ public class TestAbfsRestOperationMockFailures {
     Mockito.doReturn("").when(httpOperation).getStorageErrorMessage();
     Mockito.doReturn("").when(httpOperation).getStorageErrorCode();
     Mockito.doReturn("HEAD").when(httpOperation).getMethod();
+    Mockito.doReturn("").when(httpOperation).getMaskedUrl();
+    Mockito.doReturn("").when(httpOperation).getRequestId();
     Mockito.doReturn(EGRESS_OVER_ACCOUNT_LIMIT.getErrorMessage()).when(httpOperation).getStorageErrorMessage();
     Mockito.doReturn(tracingContext).when(abfsRestOperation).createNewTracingContext(any());
 
@@ -271,7 +275,8 @@ public class TestAbfsRestOperationMockFailures {
 
     // Assert that intercept.updateMetrics was called 2 times. Both the retried request fails with EGR.
     Mockito.verify(intercept, Mockito.times(2))
-        .updateMetrics(nullable(AbfsRestOperationType.class), nullable(AbfsHttpOperation.class));
+        .updateMetrics(nullable(AbfsRestOperationType.class), nullable(
+            AbfsHttpOperation.class));
   }
 
   private void testClientRequestIdForStatusRetry(int status,
@@ -293,7 +298,8 @@ public class TestAbfsRestOperationMockFailures {
         abfsClient,
         "PUT",
         null,
-        new ArrayList<>()
+        new ArrayList<>(),
+        Mockito.mock(AbfsConfiguration.class)
     ));
 
     AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class);
@@ -358,7 +364,8 @@ public class TestAbfsRestOperationMockFailures {
         abfsClient,
         "PUT",
         null,
-        new ArrayList<>()
+        new ArrayList<>(),
+        Mockito.mock(AbfsConfiguration.class)
     ));
 
     AbfsHttpOperation httpOperation = Mockito.mock(AbfsHttpOperation.class);

+ 290 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestApacheClientConnectionPool.java

@@ -0,0 +1,290 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ClosedIOException;
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsTestWithTimeout;
+
+import org.apache.http.HttpClientConnection;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.EMPTY_STRING;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_MAX_CONN_SYS_PROP;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.KEEP_ALIVE_CACHE_CLOSED;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_APACHE_HTTP_CLIENT_IDLE_CONNECTION_TTL;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_APACHE_HTTP_CLIENT_MAX_CACHE_CONNECTION_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_HTTP_CLIENT_CONN_MAX_IDLE_TIME;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.HUNDRED;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public class TestApacheClientConnectionPool extends
+    AbstractAbfsTestWithTimeout {
+
+  public TestApacheClientConnectionPool() throws Exception {
+    super();
+  }
+
+  @Override
+  protected int getTestTimeoutMillis() {
+    return (int) DEFAULT_HTTP_CLIENT_CONN_MAX_IDLE_TIME * 4;
+  }
+
+  @Test
+  public void testBasicPool() throws Exception {
+    System.clearProperty(HTTP_MAX_CONN_SYS_PROP);
+    validatePoolSize(DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS);
+  }
+
+  @Test
+  public void testSysPropAppliedPool() throws Exception {
+    final String customPoolSize = "10";
+    System.setProperty(HTTP_MAX_CONN_SYS_PROP, customPoolSize);
+    validatePoolSize(Integer.parseInt(customPoolSize));
+  }
+
+  @Test
+  public void testPoolWithZeroSysProp() throws Exception {
+    final String customPoolSize = "0";
+    System.setProperty(HTTP_MAX_CONN_SYS_PROP, customPoolSize);
+    validatePoolSize(DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS);
+  }
+
+  @Test
+  public void testEmptySizePool() throws Exception {
+    Configuration configuration = new Configuration();
+    configuration.set(FS_AZURE_APACHE_HTTP_CLIENT_MAX_CACHE_CONNECTION_SIZE,
+        "0");
+    AbfsConfiguration abfsConfiguration = new AbfsConfiguration(configuration,
+        EMPTY_STRING);
+    try (KeepAliveCache keepAliveCache = new KeepAliveCache(
+        abfsConfiguration)) {
+      assertCachePutFail(keepAliveCache,
+          Mockito.mock(HttpClientConnection.class));
+      assertCacheGetIsNull(keepAliveCache);
+    }
+  }
+
+  private void assertCacheGetIsNull(final KeepAliveCache keepAliveCache)
+      throws IOException {
+    Assertions.assertThat(keepAliveCache.get())
+        .describedAs("cache.get()")
+        .isNull();
+  }
+
+  private void assertCacheGetIsNonNull(final KeepAliveCache keepAliveCache)
+      throws IOException {
+    Assertions.assertThat(keepAliveCache.get())
+        .describedAs("cache.get()")
+        .isNotNull();
+  }
+
+  private void assertCachePutFail(final KeepAliveCache keepAliveCache,
+      final HttpClientConnection mock) {
+    Assertions.assertThat(keepAliveCache.put(mock))
+        .describedAs("cache.put()")
+        .isFalse();
+  }
+
+  private void assertCachePutSuccess(final KeepAliveCache keepAliveCache,
+      final HttpClientConnection connections) {
+    Assertions.assertThat(keepAliveCache.put(connections))
+        .describedAs("cache.put()")
+        .isTrue();
+  }
+
+  private void validatePoolSize(int size) throws Exception {
+    try (KeepAliveCache keepAliveCache = new KeepAliveCache(
+        new AbfsConfiguration(new Configuration(), EMPTY_STRING))) {
+      keepAliveCache.clear();
+      final HttpClientConnection[] connections = new HttpClientConnection[size
+          * 2];
+
+      for (int i = 0; i < size * 2; i++) {
+        connections[i] = Mockito.mock(HttpClientConnection.class);
+      }
+
+      for (int i = 0; i < size; i++) {
+        assertCachePutSuccess(keepAliveCache, connections[i]);
+        Mockito.verify(connections[i], Mockito.times(0)).close();
+      }
+
+      for (int i = size; i < size * 2; i++) {
+        assertCachePutSuccess(keepAliveCache, connections[i]);
+        Mockito.verify(connections[i - size], Mockito.times(1)).close();
+      }
+
+      for (int i = 0; i < size * 2; i++) {
+        if (i < size) {
+          assertCacheGetIsNonNull(keepAliveCache);
+        } else {
+          assertCacheGetIsNull(keepAliveCache);
+        }
+      }
+      System.clearProperty(HTTP_MAX_CONN_SYS_PROP);
+    }
+  }
+
+  @Test
+  public void testKeepAliveCache() throws Exception {
+    try (KeepAliveCache keepAliveCache = new KeepAliveCache(
+        new AbfsConfiguration(new Configuration(), EMPTY_STRING))) {
+      keepAliveCache.clear();
+      HttpClientConnection connection = Mockito.mock(
+          HttpClientConnection.class);
+
+      keepAliveCache.put(connection);
+
+      assertCacheGetIsNonNull(keepAliveCache);
+    }
+  }
+
+  @Test
+  public void testKeepAliveCacheCleanup() throws Exception {
+    Configuration configuration = new Configuration();
+    configuration.set(FS_AZURE_APACHE_HTTP_CLIENT_IDLE_CONNECTION_TTL,
+        HUNDRED + EMPTY_STRING);
+    try (KeepAliveCache keepAliveCache = new KeepAliveCache(
+        new AbfsConfiguration(configuration, EMPTY_STRING))) {
+      keepAliveCache.clear();
+      HttpClientConnection connection = Mockito.mock(
+          HttpClientConnection.class);
+
+
+      // Eviction thread would close the TTL-elapsed connection and remove it from cache.
+      AtomicBoolean isConnClosed = new AtomicBoolean(false);
+      Mockito.doAnswer(closeInvocation -> {
+        isConnClosed.set(true);
+        return null;
+      }).when(connection).close();
+      keepAliveCache.put(connection);
+
+      while (!isConnClosed.get()) {
+        Thread.sleep(HUNDRED);
+      }
+
+      // Assert that the closed connection is removed from the cache.
+      assertCacheGetIsNull(keepAliveCache);
+      Mockito.verify(connection, Mockito.times(1)).close();
+    }
+  }
+
+  @Test
+  public void testKeepAliveCacheCleanupWithConnections() throws Exception {
+    Configuration configuration = new Configuration();
+    configuration.set(FS_AZURE_APACHE_HTTP_CLIENT_IDLE_CONNECTION_TTL,
+        HUNDRED + EMPTY_STRING);
+    try (KeepAliveCache keepAliveCache = new KeepAliveCache(
+        new AbfsConfiguration(configuration, EMPTY_STRING))) {
+      keepAliveCache.pauseThread();
+      keepAliveCache.clear();
+      HttpClientConnection connection = Mockito.mock(
+          HttpClientConnection.class);
+      keepAliveCache.put(connection);
+
+      Thread.sleep(2 * keepAliveCache.getConnectionIdleTTL());
+      /*
+       * Eviction thread is switched off, the get() on the cache would close and
+       * remove the TTL-elapsed connection.
+       */
+      Mockito.verify(connection, Mockito.times(0)).close();
+      assertCacheGetIsNull(keepAliveCache);
+      Mockito.verify(connection, Mockito.times(1)).close();
+      keepAliveCache.resumeThread();
+    }
+  }
+
+  @Test
+  public void testKeepAliveCacheConnectionRecache() throws Exception {
+    try (KeepAliveCache keepAliveCache = new KeepAliveCache(
+        new AbfsConfiguration(new Configuration(), EMPTY_STRING))) {
+      keepAliveCache.clear();
+      HttpClientConnection connection = Mockito.mock(
+          HttpClientConnection.class);
+      keepAliveCache.put(connection);
+
+      assertCacheGetIsNonNull(keepAliveCache);
+      keepAliveCache.put(connection);
+      assertCacheGetIsNonNull(keepAliveCache);
+    }
+  }
+
+  @Test
+  public void testKeepAliveCacheRemoveStaleConnection() throws Exception {
+    try (KeepAliveCache keepAliveCache = new KeepAliveCache(
+        new AbfsConfiguration(new Configuration(), EMPTY_STRING))) {
+      keepAliveCache.clear();
+      HttpClientConnection[] connections = new HttpClientConnection[5];
+
+      // Fill up the cache.
+      for (int i = 0;
+          i < DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS;
+          i++) {
+        connections[i] = Mockito.mock(HttpClientConnection.class);
+        keepAliveCache.put(connections[i]);
+      }
+
+      // Mark all but the last two connections as stale.
+      for (int i = 0;
+          i < DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS - 2;
+          i++) {
+        Mockito.doReturn(true).when(connections[i]).isStale();
+      }
+
+      // Verify that the stale connections are removed.
+      for (int i = DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS - 1;
+          i >= 0;
+          i--) {
+        // The last two connections are not stale and would be returned.
+        if (i >= (DEFAULT_HTTP_CLIENT_CONN_MAX_CACHED_CONNECTIONS - 2)) {
+          assertCacheGetIsNonNull(keepAliveCache);
+        } else {
+          // Stale connections are closed and removed.
+          assertCacheGetIsNull(keepAliveCache);
+          Mockito.verify(connections[i], Mockito.times(1)).close();
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testKeepAliveCacheClosed() throws Exception {
+    KeepAliveCache keepAliveCache = Mockito.spy(new KeepAliveCache(
+        new AbfsConfiguration(new Configuration(), EMPTY_STRING)));
+    keepAliveCache.put(Mockito.mock(HttpClientConnection.class));
+    keepAliveCache.close();
+    intercept(ClosedIOException.class,
+        KEEP_ALIVE_CACHE_CLOSED,
+        () -> keepAliveCache.get());
+
+    HttpClientConnection conn = Mockito.mock(HttpClientConnection.class);
+    assertCachePutFail(keepAliveCache, conn);
+    Mockito.verify(conn, Mockito.times(1)).close();
+    keepAliveCache.close();
+    Mockito.verify(keepAliveCache, Mockito.times(1)).closeInternal();
+  }
+}

+ 226 - 0
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestApacheHttpClientFallback.java

@@ -0,0 +1,226 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsTestWithTimeout;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderFormat;
+
+import static java.net.HttpURLConnection.HTTP_OK;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.APACHE_IMPL;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.JDK_FALLBACK;
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.JDK_IMPL;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpOperationType.APACHE_HTTP_CLIENT;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+
+public class TestApacheHttpClientFallback extends AbstractAbfsTestWithTimeout {
+
+  public TestApacheHttpClientFallback() throws Exception {
+    super();
+  }
+
+  private TracingContext getSampleTracingContext(int[] jdkCallsRegister,
+      int[] apacheCallsRegister) {
+    String correlationId, fsId;
+    TracingHeaderFormat format;
+    correlationId = "test-corr-id";
+    fsId = "test-filesystem-id";
+    format = TracingHeaderFormat.ALL_ID_FORMAT;
+    TracingContext tc = Mockito.spy(new TracingContext(correlationId, fsId,
+        FSOperationType.TEST_OP, true, format, null));
+    Mockito.doAnswer(answer -> {
+          answer.callRealMethod();
+          AbfsHttpOperation op = answer.getArgument(0);
+          if (op instanceof AbfsAHCHttpOperation) {
+            Assertions.assertThat(tc.getHeader()).endsWith(APACHE_IMPL);
+            apacheCallsRegister[0]++;
+          }
+          if (op instanceof AbfsJdkHttpOperation) {
+            jdkCallsRegister[0]++;
+            if (AbfsApacheHttpClient.usable()) {
+              Assertions.assertThat(tc.getHeader()).endsWith(JDK_IMPL);
+            } else {
+              Assertions.assertThat(tc.getHeader()).endsWith(JDK_FALLBACK);
+            }
+          }
+          return null;
+        })
+        .when(tc)
+        .constructHeader(Mockito.any(AbfsHttpOperation.class),
+            Mockito.nullable(String.class), Mockito.nullable(String.class));
+    return tc;
+  }
+
+  @Test
+  public void testMultipleFailureLeadToFallback()
+      throws Exception {
+    int[] apacheCallsTest1 = {0};
+    int[] jdkCallsTest1 = {0};
+    TracingContext tcTest1 = getSampleTracingContext(jdkCallsTest1,
+        apacheCallsTest1);
+    int[] retryIterationTest1 = {0};
+    intercept(IOException.class, () -> {
+      getMockRestOperation(retryIterationTest1).execute(tcTest1);
+    });
+    Assertions.assertThat(apacheCallsTest1[0])
+        .isEqualTo(DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES);
+    Assertions.assertThat(jdkCallsTest1[0]).isEqualTo(1);
+
+    int[] retryIteration1 = {0};
+    int[] apacheCallsTest2 = {0};
+    int[] jdkCallsTest2 = {0};
+    TracingContext tcTest2 = getSampleTracingContext(jdkCallsTest2,
+        apacheCallsTest2);
+    intercept(IOException.class, () -> {
+      getMockRestOperation(retryIteration1).execute(tcTest2);
+    });
+    Assertions.assertThat(apacheCallsTest2[0]).isEqualTo(0);
+    Assertions.assertThat(jdkCallsTest2[0])
+        .isEqualTo(DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES + 1);
+  }
+
+  private AbfsRestOperation getMockRestOperation(int[] retryIteration)
+      throws IOException {
+    AbfsConfiguration configuration = Mockito.mock(AbfsConfiguration.class);
+    Mockito.doReturn(APACHE_HTTP_CLIENT)
+        .when(configuration)
+        .getPreferredHttpOperationType();
+    Mockito.doReturn(DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES)
+        .when(configuration)
+        .getMaxApacheHttpClientIoExceptionsRetries();
+    AbfsClient client = Mockito.mock(AbfsClient.class);
+    Mockito.doReturn(Mockito.mock(ExponentialRetryPolicy.class))
+        .when(client)
+        .getExponentialRetryPolicy();
+
+    AbfsRetryPolicy retryPolicy = Mockito.mock(AbfsRetryPolicy.class);
+    Mockito.doReturn(retryPolicy)
+        .when(client)
+        .getRetryPolicy(Mockito.nullable(String.class));
+
+    Mockito.doAnswer(answer -> {
+          if (retryIteration[0]
+              < DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES) {
+            retryIteration[0]++;
+            return true;
+          } else {
+            return false;
+          }
+        })
+        .when(retryPolicy)
+        .shouldRetry(Mockito.anyInt(), Mockito.nullable(Integer.class));
+
+    AbfsThrottlingIntercept abfsThrottlingIntercept = Mockito.mock(
+        AbfsThrottlingIntercept.class);
+    Mockito.doNothing()
+        .when(abfsThrottlingIntercept)
+        .updateMetrics(Mockito.any(AbfsRestOperationType.class),
+            Mockito.any(AbfsHttpOperation.class));
+    Mockito.doNothing()
+        .when(abfsThrottlingIntercept)
+        .sendingRequest(Mockito.any(AbfsRestOperationType.class),
+            Mockito.nullable(AbfsCounters.class));
+    Mockito.doReturn(abfsThrottlingIntercept).when(client).getIntercept();
+
+
+    AbfsRestOperation op = Mockito.spy(new AbfsRestOperation(
+        AbfsRestOperationType.ReadFile,
+        client,
+        AbfsHttpConstants.HTTP_METHOD_GET,
+        new URL("http://localhost"),
+        new ArrayList<>(),
+        null,
+        configuration
+    ));
+
+    Mockito.doReturn(null).when(op).getClientLatency();
+
+    Mockito.doReturn(createApacheHttpOp())
+        .when(op)
+        .createAbfsHttpOperation();
+    Mockito.doReturn(createAhcHttpOp())
+        .when(op)
+        .createAbfsAHCHttpOperation();
+
+    Mockito.doAnswer(answer -> {
+      return answer.getArgument(0);
+    }).when(op).createNewTracingContext(Mockito.nullable(TracingContext.class));
+
+    Mockito.doNothing()
+        .when(op)
+        .signRequest(Mockito.any(AbfsHttpOperation.class), Mockito.anyInt());
+
+    Mockito.doAnswer(answer -> {
+      AbfsHttpOperation operation = Mockito.spy(
+          (AbfsHttpOperation) answer.callRealMethod());
+      Assertions.assertThat(operation).isInstanceOf(
+          (retryIteration[0]
+              < DEFAULT_APACHE_HTTP_CLIENT_MAX_IO_EXCEPTION_RETRIES
+              && AbfsApacheHttpClient.usable())
+              ? AbfsAHCHttpOperation.class
+              : AbfsJdkHttpOperation.class);
+      Mockito.doReturn(HTTP_OK).when(operation).getStatusCode();
+      Mockito.doThrow(new IOException("Test Exception"))
+          .when(operation)
+          .processResponse(Mockito.nullable(byte[].class), Mockito.anyInt(),
+              Mockito.anyInt());
+      Mockito.doCallRealMethod().when(operation).getTracingContextSuffix();
+      return operation;
+    }).when(op).createHttpOperation();
+    return op;
+  }
+
+  private AbfsAHCHttpOperation createAhcHttpOp() {
+    AbfsAHCHttpOperation ahcOp = Mockito.mock(AbfsAHCHttpOperation.class);
+    Mockito.doCallRealMethod().when(ahcOp).getTracingContextSuffix();
+    return ahcOp;
+  }
+
+  private AbfsJdkHttpOperation createApacheHttpOp() {
+    AbfsJdkHttpOperation httpOperationMock = Mockito.mock(AbfsJdkHttpOperation.class);
+    Mockito.doCallRealMethod()
+        .when(httpOperationMock)
+        .getTracingContextSuffix();
+    return httpOperationMock;
+  }
+
+  @Test
+  public void testTcHeaderOnJDKClientUse() {
+    int[] jdkCallsRegister = {0};
+    int[] apacheCallsRegister = {0};
+    TracingContext tc = getSampleTracingContext(jdkCallsRegister,
+        apacheCallsRegister);
+    AbfsJdkHttpOperation op = Mockito.mock(AbfsJdkHttpOperation.class);
+    Mockito.doCallRealMethod().when(op).getTracingContextSuffix();
+    tc.constructHeader(op, null, null);
+  }
+}

+ 1 - 1
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/TracingHeaderValidator.java

@@ -93,7 +93,7 @@ public class TracingHeaderValidator implements Listener {
   private void validateBasicFormat(String[] idList) {
     if (format == TracingHeaderFormat.ALL_ID_FORMAT) {
       Assertions.assertThat(idList)
-          .describedAs("header should have 7 elements").hasSize(7);
+          .describedAs("header should have 8 elements").hasSize(8);
     } else if (format == TracingHeaderFormat.TWO_ID_FORMAT) {
       Assertions.assertThat(idList)
           .describedAs("header should have 2 elements").hasSize(2);