Browse Source

Merge branch 'HADOOP-15407' into trunk

Sean Mackrory 6 năm trước cách đây
mục cha
commit
0def61482b
100 tập tin đã thay đổi với 11208 bổ sung77 xóa
  1. 1 0
      .gitignore
  2. 3 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
  3. 40 1
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  4. 1 10
      hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
  5. 3 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
  6. 7 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigRedactor.java
  7. 9 25
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractConcatTest.java
  8. 7 19
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java
  9. 8 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMkdirTest.java
  10. 0 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContract.java
  11. 17 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
  12. 18 0
      hadoop-project/pom.xml
  13. 387 16
      hadoop-tools/hadoop-azure/pom.xml
  14. 49 0
      hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml
  15. 1 1
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/ClientThrottlingAnalyzer.java
  16. 46 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java
  17. 576 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java
  18. 46 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java
  19. 953 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
  20. 1028 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java
  21. 39 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.java
  22. 91 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java
  23. 89 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java
  24. 67 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java
  25. 42 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemUriSchemes.java
  26. 63 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java
  27. 40 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.java
  28. 22 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java
  29. 104 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java
  30. 22 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/package-info.java
  31. 37 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java
  32. 22 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/package-info.java
  33. 84 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AbfsRestOperationException.java
  34. 56 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/AzureBlobFileSystemException.java
  35. 32 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/ConfigurationPropertyNotFoundException.java
  36. 33 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/FileSystemOperationUnhandledException.java
  37. 40 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.java
  38. 33 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAclOperationException.java
  39. 37 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidConfigurationValueException.java
  40. 33 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidFileSystemPropertyException.java
  41. 33 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriAuthorityException.java
  42. 33 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidUriException.java
  43. 42 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/KeyProviderException.java
  44. 33 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/TimeoutException.java
  45. 36 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/TokenAccessProviderException.java
  46. 22 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/package-info.java
  47. 22 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/package-info.java
  48. 115 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java
  49. 239 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java
  50. 58 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java
  51. 29 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ReadBufferStatus.java
  52. 22 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/package-info.java
  53. 50 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java
  54. 50 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java
  55. 67 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java
  56. 68 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java
  57. 63 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.java
  58. 43 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java
  59. 22 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java
  60. 70 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomDelegationTokenManager.java
  61. 75 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomTokenProviderAdaptee.java
  62. 32 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/package-info.java
  63. 98 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AccessTokenProvider.java
  64. 344 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java
  65. 47 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADToken.java
  66. 62 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/ClientCredsTokenProvider.java
  67. 58 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdapter.java
  68. 48 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/MsiTokenProvider.java
  69. 69 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/QueryParams.java
  70. 57 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/RefreshTokenBasedTokenProvider.java
  71. 56 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/UserPasswordTokenProvider.java
  72. 18 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/package-info.java
  73. 31 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/package.html
  74. 49 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsDelegationTokenIdentifier.java
  75. 88 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsDelegationTokenManager.java
  76. 96 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsTokenRenewer.java
  77. 23 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/package-info.java
  78. 202 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsAclHelper.java
  79. 581 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java
  80. 272 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingAnalyzer.java
  81. 135 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientThrottlingIntercept.java
  82. 40 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java
  83. 446 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java
  84. 381 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java
  85. 378 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java
  86. 114 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsPermission.java
  87. 193 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperation.java
  88. 42 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsRestOperationType.java
  89. 64 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsUriQueryBuilder.java
  90. 27 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AuthType.java
  91. 144 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.java
  92. 43 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/KeyProvider.java
  93. 139 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBuffer.java
  94. 395 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferManager.java
  95. 72 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ReadBufferWorker.java
  96. 510 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SharedKeyCredentials.java
  97. 71 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ShellDecryptionKeyProvider.java
  98. 54 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SimpleKeyProvider.java
  99. 22 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/package-info.java
  100. 329 0
      hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Base64.java

+ 1 - 0
.gitignore

@@ -22,6 +22,7 @@ make-build-debug
 # Filesystem contract test options and credentials
 auth-keys.xml
 azure-auth-keys.xml
+azure-bfs-auth-keys.xml
 
 # External tool builders
 */.externalToolBuilders

+ 3 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java

@@ -886,7 +886,9 @@ public class CommonConfigurationKeysPublic {
           "fs.s3a.*.server-side-encryption.key",
           "fs.azure\\.account.key.*",
           "credential$",
-          "oauth.*token$",
+          "oauth.*secret",
+          "oauth.*password",
+          "oauth.*token",
           HADOOP_SECURITY_SENSITIVE_CONFIG_KEYS);
 
   /**

+ 40 - 1
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -603,7 +603,9 @@
       fs.s3a.*.server-side-encryption.key
       fs.azure.account.key.*
       credential$
-      oauth.*token$
+      oauth.*secret
+      oauth.*password
+      oauth.*token
       hadoop.security.sensitive-config-keys
   </value>
   <description>A comma-separated or multi-line list of regular expressions to
@@ -1618,6 +1620,18 @@
 </property>
 
 <!-- Azure file system properties -->
+<property>
+  <name>fs.AbstractFileSystem.wasb.impl</name>
+  <value>org.apache.hadoop.fs.azure.Wasb</value>
+  <description>AbstractFileSystem implementation class of wasb://</description>
+</property>
+
+<property>
+  <name>fs.AbstractFileSystem.wasbs.impl</name>
+  <value>org.apache.hadoop.fs.azure.Wasbs</value>
+  <description>AbstractFileSystem implementation class of wasbs://</description>
+</property>
+
 <property>
   <name>fs.wasb.impl</name>
   <value>org.apache.hadoop.fs.azure.NativeAzureFileSystem</value>
@@ -1639,6 +1653,31 @@
     SAS keys to communicate with Azure storage.
   </description>
 </property>
+
+<property>
+  <name>fs.abfs.impl</name>
+  <value>org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem</value>
+  <description>The implementation class of the Azure Blob Filesystem</description>
+</property>
+
+<property>
+  <name>fs.abfss.impl</name>
+  <value>org.apache.hadoop.fs.azurebfs.SecureAzureBlobFileSystem</value>
+  <description>The implementation class of the Secure Azure Blob Filesystem</description>
+</property>
+
+<property>
+  <name>fs.AbstractFileSystem.abfs.impl</name>
+  <value>org.apache.hadoop.fs.azurebfs.Abfs</value>
+  <description>AbstractFileSystem implementation class of abfs://</description>
+</property>
+
+<property>
+  <name>fs.AbstractFileSystem.abfss.impl</name>
+  <value>org.apache.hadoop.fs.azurebfs.Abfss</value>
+  <description>AbstractFileSystem implementation class of abfss://</description>
+</property>
+
 <property>
   <name>fs.azure.local.sas.key.mode</name>
   <value>false</value>

+ 1 - 10
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md

@@ -544,15 +544,6 @@ atomic. The combined operation, including `mkdirs(parent(F))` MAY be atomic.
 The return value is always true&mdash;even if a new directory is not created
  (this is defined in HDFS).
 
-#### Implementation Notes: Local FileSystem
-
-The local FileSystem does not raise an exception if `mkdirs(p)` is invoked
-on a path that exists and is a file. Instead the operation returns false.
-
-    if isFile(FS, p):
-       FS' = FS
-       result = False
-
 ### <a name='FileSystem.create'></a> `FSDataOutputStream create(Path, ...)`
 
 
@@ -641,7 +632,7 @@ Implementations without a compliant call SHOULD throw `UnsupportedOperationExcep
 
     if not exists(FS, p) : raise FileNotFoundException
 
-    if not isFile(FS, p) : raise [FileNotFoundException, IOException]
+    if not isFile(FS, p) : raise [FileAlreadyExistsException, FileNotFoundException, IOException]
 
 #### Postconditions
 

+ 3 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java

@@ -113,6 +113,9 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase {
     xmlPrefixToSkipCompare.add("fs.wasb.impl");
     xmlPrefixToSkipCompare.add("fs.wasbs.impl");
     xmlPrefixToSkipCompare.add("fs.azure.");
+    xmlPrefixToSkipCompare.add("fs.abfs.impl");
+    xmlPrefixToSkipCompare.add("fs.abfss.impl");
+
 
     // ADL properties are in a different subtree
     // - org.apache.hadoop.hdfs.web.ADLConfKeys

+ 7 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigRedactor.java

@@ -55,6 +55,13 @@ public class TestConfigRedactor {
         "fs.s3a.server-side-encryption.key",
         "fs.s3a.bucket.engineering.server-side-encryption.key",
         "fs.azure.account.key.abcdefg.blob.core.windows.net",
+        "fs.azure.account.key.abcdefg.dfs.core.windows.net",
+        "fs.azure.account.oauth2.client.secret",
+        "fs.azure.account.oauth2.client.secret.account.dfs.core.windows.net",
+        "fs.azure.account.oauth2.user.password",
+        "fs.azure.account.oauth2.user.password.account.dfs.core.windows.net",
+        "fs.azure.account.oauth2.refresh.token",
+        "fs.azure.account.oauth2.refresh.token.account.dfs.core.windows.net",
         "fs.adl.oauth2.refresh.token",
         "fs.adl.oauth2.credential",
         "dfs.adls.oauth2.refresh.token",

+ 9 - 25
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractConcatTest.java

@@ -19,15 +19,16 @@
 package org.apache.hadoop.fs.contract;
 
 import org.apache.hadoop.fs.Path;
+
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertFileHasLength;
-import static org.apache.hadoop.fs.contract.ContractTestUtils.cleanup;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Test concat -if supported
@@ -60,25 +61,15 @@ public abstract class AbstractContractConcatTest extends AbstractFSContractTestB
   @Test
   public void testConcatEmptyFiles() throws Throwable {
     touch(getFileSystem(), target);
-    try {
-      getFileSystem().concat(target, new Path[0]);
-      fail("expected a failure");
-    } catch (Exception e) {
-      //expected
-      handleExpectedException(e);
-    }
+    handleExpectedException(intercept(Exception.class,
+        () -> getFileSystem().concat(target, new Path[0])));
   }
 
   @Test
   public void testConcatMissingTarget() throws Throwable {
-    try {
-      getFileSystem().concat(target,
-                             new Path[] { zeroByteFile});
-      fail("expected a failure");
-    } catch (Exception e) {
-      //expected
-      handleExpectedException(e);
-    }
+    handleExpectedException(
+        intercept(Exception.class,
+            () -> getFileSystem().concat(target, new Path[]{zeroByteFile})));
   }
 
   @Test
@@ -98,15 +89,8 @@ public abstract class AbstractContractConcatTest extends AbstractFSContractTestB
   public void testConcatOnSelf() throws Throwable {
     byte[] block = dataset(TEST_FILE_LEN, 0, 255);
     createFile(getFileSystem(), target, false, block);
-    try {
-      getFileSystem().concat(target,
-                             new Path[]{target});
-    } catch (Exception e) {
-      //expected
-      handleExpectedException(e);
-    }
+    handleExpectedException(intercept(Exception.class,
+        () -> getFileSystem().concat(target, new Path[]{target})));
   }
 
-
-
 }

+ 7 - 19
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.RemoteIterator;
 import org.junit.Test;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Test getFileStatus and related listing operations.
@@ -275,35 +276,22 @@ public abstract class AbstractContractGetFileStatusTest extends
   @Test
   public void testLocatedStatusNoDir() throws Throwable {
     describe("test the LocatedStatus call on a path which is not present");
-    try {
-      RemoteIterator<LocatedFileStatus> iterator
-          = getFileSystem().listLocatedStatus(path("missing"));
-      fail("Expected an exception, got an iterator: " + iterator);
-    } catch (FileNotFoundException expected) {
-      // expected
-    }
+    intercept(FileNotFoundException.class,
+        () -> getFileSystem().listLocatedStatus(path("missing")));
   }
 
   @Test
   public void testListStatusNoDir() throws Throwable {
     describe("test the listStatus(path) call on a path which is not present");
-    try {
-      getFileSystem().listStatus(path("missing"));
-      fail("Expected an exception");
-    } catch (FileNotFoundException expected) {
-      // expected
-    }
+    intercept(FileNotFoundException.class,
+        () -> getFileSystem().listStatus(path("missing")));
   }
 
   @Test
   public void testListStatusFilteredNoDir() throws Throwable {
     describe("test the listStatus(path, filter) call on a missing path");
-    try {
-      getFileSystem().listStatus(path("missing"), ALL_PATHS);
-      fail("Expected an exception");
-    } catch (FileNotFoundException expected) {
-      // expected
-    }
+    intercept(FileNotFoundException.class,
+        () -> getFileSystem().listStatus(path("missing"), ALL_PATHS));
   }
 
   @Test

+ 8 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMkdirTest.java

@@ -26,6 +26,7 @@ import org.junit.Test;
 
 import java.io.IOException;
 
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 
@@ -175,4 +176,11 @@ public abstract class AbstractContractMkdirTest extends AbstractFSContractTestBa
     }
   }
 
+  @Test
+  public void testCreateDirWithExistingDir() throws Exception {
+    Path path = path("testCreateDirWithExistingDir");
+    final FileSystem fs = getFileSystem();
+    assertMkdirs(fs, path);
+    assertMkdirs(fs, path);
+  }
 }

+ 0 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContract.java

@@ -148,7 +148,6 @@ public abstract class AbstractFSContract extends Configured {
    * @param feature feature to query
    * @param defval default value
    * @return true if the feature is supported
-   * @throws IOException IO problems
    */
   public boolean isSupported(String feature, boolean defval) {
     return getConf().getBoolean(getConfKey(feature), defval);
@@ -160,7 +159,6 @@ public abstract class AbstractFSContract extends Configured {
    * @param feature feature to query
    * @param defval default value
    * @return true if the feature is supported
-   * @throws IOException IO problems
    */
   public int getLimit(String feature, int defval) {
     return getConf().getInt(getConfKey(feature), defval);

+ 17 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java

@@ -187,8 +187,11 @@ public class ContractTestUtils extends Assert {
           (short) 1,
           buffersize);
     }
-    out.write(src, 0, len);
-    out.close();
+    try {
+      out.write(src, 0, len);
+    } finally {
+      out.close();
+    }
     assertFileHasLength(fs, path, len);
   }
 
@@ -1021,6 +1024,18 @@ public class ContractTestUtils extends Assert {
                       found);
   }
 
+  /**
+   * Execute {@link FileSystem#mkdirs(Path)}; expect {@code true} back.
+   * (Note: does not work for localFS if the directory already exists)
+   * Does not perform any validation of the created directory.
+   * @param fs filesystem
+   * @param dir directory to create
+   * @throws IOException IO Problem
+   */
+  public static void assertMkdirs(FileSystem fs, Path dir) throws IOException {
+    assertTrue("mkdirs(" + dir + ") returned false", fs.mkdirs(dir));
+  }
+
   /**
    * Test for the host being an OSX machine.
    * @return true if the JVM thinks that is running on OSX

+ 18 - 0
hadoop-project/pom.xml

@@ -1209,6 +1209,11 @@
         <artifactId>jsch</artifactId>
         <version>0.1.54</version>
       </dependency>
+      <dependency>
+        <groupId>org.apache.htrace</groupId>
+        <artifactId>htrace-core</artifactId>
+        <version>3.1.0-incubating</version>
+      </dependency>
       <dependency>
         <groupId>org.apache.htrace</groupId>
         <artifactId>htrace-core4</artifactId>
@@ -1344,6 +1349,19 @@
         <version>7.0.0</version>
      </dependency>
 
+      <!--Wildfly openssl dependency is introduced by HADOOP-15669-->
+      <dependency>
+        <groupId>org.wildfly.openssl</groupId>
+        <artifactId>wildfly-openssl</artifactId>
+        <version>1.0.4.Final</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.threadly</groupId>
+        <artifactId>threadly</artifactId>
+        <version>4.9.0</version>
+      </dependency>
+
       <dependency>
         <groupId>com.aliyun.oss</groupId>
         <artifactId>aliyun-sdk-oss</artifactId>

+ 387 - 16
hadoop-tools/hadoop-azure/pom.xml

@@ -67,6 +67,7 @@
             <!-- To run with the default Sun ruleset, 
                  comment out the configLocation line -->
             <configLocation>src/config/checkstyle.xml</configLocation>
+            <suppressionsLocation>src/config/checkstyle-suppressions.xml</suppressionsLocation>
           </configuration>
           
       </plugin>
@@ -148,11 +149,6 @@
       <scope>provided</scope>
     </dependency>
 
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-core</artifactId>
-      <scope>compile</scope>
-    </dependency>
     
     <dependency>
       <groupId>org.apache.httpcomponents</groupId>
@@ -172,10 +168,22 @@
       </exclusions>
     </dependency>
 
+    <dependency>
+      <groupId>com.google.inject</groupId>
+      <artifactId>guice</artifactId>
+      <scope>compile</scope>
+      <!-- we have a dependency on a lower version -->
+      <exclusions>
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
     <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
-      <scope>compile</scope>
     </dependency>
 
     <dependency>
@@ -183,15 +191,26 @@
       <artifactId>jetty-util-ajax</artifactId>
       <scope>compile</scope>
     </dependency>
-    
-    
-    <!-- dependencies use for test only -->
+
     <dependency>
-      <groupId>commons-io</groupId>
-      <artifactId>commons-io</artifactId>
-      <scope>test</scope>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-core-asl</artifactId>
+      <scope>compile</scope>
     </dependency>
 
+    <dependency>
+      <groupId>org.wildfly.openssl</groupId>
+      <artifactId>wildfly-openssl</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <!-- dependencies use for test only -->
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
@@ -229,19 +248,363 @@
       <artifactId>mockito-all</artifactId>
       <scope>test</scope>
     </dependency>
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
-    </dependency>
 
   </dependencies>
 
   <profiles>
+    <profile>
+      <id>parallel-tests-wasb</id>
+      <activation>
+        <property>
+          <name>parallel-tests</name>
+          <value>wasb</value>
+        </property>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>create-parallel-tests-dirs</id>
+                <phase>test-compile</phase>
+                <configuration>
+                  <target>
+                    <script language="javascript"><![CDATA[
+                      var baseDirs = [
+                        project.getProperty("test.build.data"),
+                        project.getProperty("test.build.dir"),
+                        project.getProperty("hadoop.tmp.dir")
+                      ];
+                      for (var i in baseDirs) {
+                        for (var j = 1; j <= ${testsThreadCount}; ++j) {
+                          var mkdir = project.createTask("mkdir");
+                          mkdir.setDir(new java.io.File(baseDirs[i], j));
+                          mkdir.perform();
+                        }
+                      }
+                    ]]></script>
+                  </target>
+                </configuration>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-surefire-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>default-test</id>
+                <goals>
+                  <goal>test</goal>
+                </goals>
+                <configuration>
+                  <forkCount>1</forkCount>
+                  <forkCount>${testsThreadCount}</forkCount>
+                  <reuseForks>false</reuseForks>
+                  <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
+                  <forkedProcessTimeoutInSeconds>${fs.azure.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <systemPropertyVariables>
+                    <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
+                    <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
+                    <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>
+                    <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+                    <fs.azure.scale.test.enabled>${fs.azure.scale.test.enabled}</fs.azure.scale.test.enabled>
+                    <fs.azure.scale.test.huge.filesize>${fs.azure.scale.test.huge.filesize}</fs.azure.scale.test.huge.filesize>
+                    <fs.azure.scale.test.huge.huge.partitionsize>${fs.azure.scale.test.huge.partitionsize}</fs.azure.scale.test.huge.huge.partitionsize>
+                    <fs.azure.scale.test.timeout>${fs.azure.scale.test.timeout}</fs.azure.scale.test.timeout>
+                    <fs.azure.scale.test.list.performance.threads>${fs.azure.scale.test.list.performance.threads}</fs.azure.scale.test.list.performance.threads>
+                    <fs.azure.scale.test.list.performance.files>${fs.azure.scale.test.list.performance.files}</fs.azure.scale.test.list.performance.files>
+                  </systemPropertyVariables>
+                  <includes>
+                    <include>**/azure/Test*.java</include>
+                    <include>**/azure/**/Test*.java</include>
+                  </includes>
+                  <excludes>
+                    <exclude>**/azure/**/TestRollingWindowAverage*.java</exclude>
+                  </excludes>
+                </configuration>
+              </execution>
+              <execution>
+                <id>serialized-test-wasb</id>
+                <goals>
+                  <goal>test</goal>
+                </goals>
+                <configuration>
+                  <forkCount>1</forkCount>
+                  <reuseForks>false</reuseForks>
+                  <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
+                  <forkedProcessTimeoutInSeconds>${fs.azure.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <systemPropertyVariables>
+                    <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
+                    <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
+                    <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>
+                    <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+                    <fs.azure.scale.test.enabled>${fs.azure.scale.test.enabled}</fs.azure.scale.test.enabled>
+                    <fs.azure.scale.test.huge.filesize>${fs.azure.scale.test.huge.filesize}</fs.azure.scale.test.huge.filesize>
+                    <fs.azure.scale.test.huge.huge.partitionsize>${fs.azure.scale.test.huge.partitionsize}</fs.azure.scale.test.huge.huge.partitionsize>
+                    <fs.azure.scale.test.timeout>${fs.azure.scale.test.timeout}</fs.azure.scale.test.timeout>
+                    <fs.azure.scale.test.list.performance.threads>${fs.azure.scale.test.list.performance.threads}</fs.azure.scale.test.list.performance.threads>
+                    <fs.azure.scale.test.list.performance.files>${fs.azure.scale.test.list.performance.files}</fs.azure.scale.test.list.performance.files>
+                  </systemPropertyVariables>
+                  <includes>
+                    <include>**/azure/**/TestRollingWindowAverage*.java</include>
+                  </includes>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-failsafe-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>default-integration-test-wasb</id>
+                <goals>
+                  <goal>integration-test</goal>
+                  <goal>verify</goal>
+                </goals>
+                <configuration>
+                  <forkCount>1</forkCount>
+                  <forkCount>${testsThreadCount}</forkCount>
+                  <reuseForks>false</reuseForks>
+                  <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
+                  <forkedProcessTimeoutInSeconds>${fs.azure.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <trimStackTrace>false</trimStackTrace>
+                  <systemPropertyVariables>
+                    <!-- Tell tests that they are being executed in parallel -->
+                    <test.parallel.execution>true</test.parallel.execution>
+                    <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
+                    <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
+                    <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>
+
+                    <!-- Due to a Maven quirk, setting this to just -->
+                    <!-- surefire.forkNumber won't do the parameter -->
+                    <!-- substitution.  Putting a prefix in front of it like -->
+                    <!-- "fork-" makes it work. -->
+                    <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+                    <!-- Propagate scale parameters -->
+                    <fs.azure.scale.test.enabled>${fs.azure.scale.test.enabled}</fs.azure.scale.test.enabled>
+                    <fs.azure.scale.test.huge.filesize>${fs.azure.scale.test.huge.filesize}</fs.azure.scale.test.huge.filesize>
+                    <fs.azure.scale.test.huge.huge.partitionsize>${fs.azure.scale.test.huge.partitionsize}</fs.azure.scale.test.huge.huge.partitionsize>
+                    <fs.azure.scale.test.timeout>${fs.azure.scale.test.timeout}</fs.azure.scale.test.timeout>
+                    <fs.azure.scale.test.list.performance.threads>${fs.azure.scale.test.list.performance.threads}</fs.azure.scale.test.list.performance.threads>
+                    <fs.azure.scale.test.list.performance.files>${fs.azure.scale.test.list.performance.files}</fs.azure.scale.test.list.performance.files>
+                  </systemPropertyVariables>
+                  <!-- Some tests cannot run in parallel-->
+                  <includes>
+                    <include>**/azure/ITest*.java</include>
+                    <include>**/azure/**/ITest*.java</include>
+                  </includes>
+                  <excludes>
+                    <exclude>**/azure/ITestNativeFileSystemStatistics.java</exclude>
+                  </excludes>
+                </configuration>
+              </execution>
+              <!-- Do a sequential run for tests that cannot handle -->
+              <!-- parallel execution. -->
+              <execution>
+                <id>sequential-integration-tests-wasb</id>
+                <goals>
+                  <goal>integration-test</goal>
+                  <goal>verify</goal>
+                </goals>
+                <configuration>
+                  <forkedProcessTimeoutInSeconds>${fs.azure.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <trimStackTrace>false</trimStackTrace>
+                  <systemPropertyVariables>
+                    <test.parallel.execution>false</test.parallel.execution>
+                    <fs.azure.scale.test.enabled>${fs.azure.scale.test.enabled}</fs.azure.scale.test.enabled>
+                    <fs.azure.scale.test.huge.filesize>${fs.azure.scale.test.huge.filesize}</fs.azure.scale.test.huge.filesize>
+                    <fs.azure.scale.test.huge.huge.partitionsize>${fs.azure.scale.test.huge.partitionsize}</fs.azure.scale.test.huge.huge.partitionsize>
+                    <fs.azure.scale.test.timeout>${fs.azure.scale.test.timeout}</fs.azure.scale.test.timeout>
+                    <fs.azure.scale.test.list.performance.threads>${fs.azure.scale.test.list.performance.threads}</fs.azure.scale.test.list.performance.threads>
+                    <fs.azure.scale.test.list.performance.files>${fs.azure.scale.test.list.performance.files}</fs.azure.scale.test.list.performance.files>
+                  </systemPropertyVariables>
+                  <includes>
+                    <include>**/azure/ITestNativeFileSystemStatistics.java</include>
+                  </includes>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+
+    <profile>
+      <id>parallel-tests-abfs</id>
+      <activation>
+        <property>
+          <name>parallel-tests</name>
+          <value>abfs</value>
+        </property>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>create-parallel-tests-dirs</id>
+                <phase>test-compile</phase>
+                <configuration>
+                  <target>
+                    <script language="javascript"><![CDATA[
+                      var baseDirs = [
+                        project.getProperty("test.build.data"),
+                        project.getProperty("test.build.dir"),
+                        project.getProperty("hadoop.tmp.dir")
+                      ];
+                      for (var i in baseDirs) {
+                        for (var j = 1; j <= ${testsThreadCount}; ++j) {
+                          var mkdir = project.createTask("mkdir");
+                          mkdir.setDir(new java.io.File(baseDirs[i], j));
+                          mkdir.perform();
+                        }
+                      }
+                    ]]></script>
+                  </target>
+                </configuration>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-surefire-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>default-test</id>
+                <goals>
+                  <goal>test</goal>
+                </goals>
+                <configuration>
+                  <forkCount>${testsThreadCount}</forkCount>
+                  <reuseForks>false</reuseForks>
+                  <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
+                  <forkedProcessTimeoutInSeconds>${fs.azure.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <systemPropertyVariables>
+                    <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
+                    <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
+                    <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>
+                    <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+                    <fs.azure.scale.test.enabled>${fs.azure.scale.test.enabled}</fs.azure.scale.test.enabled>
+                    <fs.azure.scale.test.huge.filesize>${fs.azure.scale.test.huge.filesize}</fs.azure.scale.test.huge.filesize>
+                    <fs.azure.scale.test.huge.huge.partitionsize>${fs.azure.scale.test.huge.partitionsize}</fs.azure.scale.test.huge.huge.partitionsize>
+                    <fs.azure.scale.test.timeout>${fs.azure.scale.test.timeout}</fs.azure.scale.test.timeout>
+                    <fs.azure.scale.test.list.performance.threads>${fs.azure.scale.test.list.performance.threads}</fs.azure.scale.test.list.performance.threads>
+                    <fs.azure.scale.test.list.performance.files>${fs.azure.scale.test.list.performance.files}</fs.azure.scale.test.list.performance.files>
+                  </systemPropertyVariables>
+                  <includes>
+                    <include>**/azurebfs/Test*.java</include>
+                    <include>**/azurebfs/**/Test*.java</include>
+                  </includes>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-failsafe-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>integration-test-abfs-parallel-classesAndMethods</id>
+                <goals>
+                  <goal>integration-test</goal>
+                  <goal>verify</goal>
+                </goals>
+                <configuration>
+                  <forkCount>${testsThreadCount}</forkCount>
+                  <reuseForks>true</reuseForks>
+                  <parallel>both</parallel>
+                  <threadCount>${testsThreadCount}</threadCount>
+                  <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
+                  <forkedProcessTimeoutInSeconds>${fs.azure.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <trimStackTrace>false</trimStackTrace>
+                  <systemPropertyVariables>
+                    <!-- Tell tests that they are being executed in parallel -->
+                    <test.parallel.execution>true</test.parallel.execution>
+                    <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
+                    <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
+                    <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>
+                    <!-- Due to a Maven quirk, setting this to just -->
+                    <!-- surefire.forkNumber won't do the parameter -->
+                    <!-- substitution.  Putting a prefix in front of it like -->
+                    <!-- "fork-" makes it work. -->
+                    <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+                    <!-- Propagate scale parameters -->
+                    <fs.azure.scale.test.enabled>${fs.azure.scale.test.enabled}</fs.azure.scale.test.enabled>
+                    <fs.azure.scale.test.timeout>${fs.azure.scale.test.timeout}</fs.azure.scale.test.timeout>
+                  </systemPropertyVariables>
+
+                  <includes>
+                    <include>**/azurebfs/ITest*.java</include>
+                    <include>**/azurebfs/**/ITest*.java</include>
+                  </includes>
+                  <excludes>
+                    <exclude>**/azurebfs/contract/ITest*.java</exclude>
+                    <exclude>**/azurebfs/ITestAzureBlobFileSystemE2EScale.java</exclude>
+                    <exclude>**/azurebfs/ITestAbfsReadWriteAndSeek.java</exclude>
+                    <exclude>**/azurebfs/ITestAzureBlobFileSystemListStatus.java</exclude>
+                  </excludes>
+
+                </configuration>
+              </execution>
+              <execution>
+                <id>integration-test-abfs-parallel-classes</id>
+                <goals>
+                  <goal>integration-test</goal>
+                  <goal>verify</goal>
+                </goals>
+                <configuration>
+                  <forkCount>${testsThreadCount}</forkCount>
+                  <reuseForks>false</reuseForks>
+                  <!--NOTICE: hadoop contract tests methods can not be ran in parallel-->
+                  <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
+                  <forkedProcessTimeoutInSeconds>${fs.azure.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <trimStackTrace>false</trimStackTrace>
+                  <systemPropertyVariables>
+                    <!-- Tell tests that they are being executed in parallel -->
+                    <test.parallel.execution>true</test.parallel.execution>
+                    <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
+                    <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
+                    <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>
+
+                    <!-- Due to a Maven quirk, setting this to just -->
+                    <!-- surefire.forkNumber won't do the parameter -->
+                    <!-- substitution.  Putting a prefix in front of it like -->
+                    <!-- "fork-" makes it work. -->
+                    <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+                    <!-- Propagate scale parameters -->
+                    <fs.azure.scale.test.enabled>${fs.azure.scale.test.enabled}</fs.azure.scale.test.enabled>
+                    <fs.azure.scale.test.timeout>${fs.azure.scale.test.timeout}</fs.azure.scale.test.timeout>
+                  </systemPropertyVariables>
+                  <includes>
+                    <include>**/azurebfs/contract/ITest*.java</include>
+                    <include>**/azurebfs/ITestAzureBlobFileSystemE2EScale.java</include>
+                    <include>**/azurebfs/ITestAbfsReadWriteAndSeek.java</include>
+                    <include>**/azurebfs/ITestAzureBlobFileSystemListStatus.java</include>
+                  </includes>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+
     <profile>
       <id>parallel-tests</id>
       <activation>
         <property>
           <name>parallel-tests</name>
+          <value>both</value>
         </property>
       </activation>
       <build>
@@ -398,8 +761,11 @@
                     <exclude>**/ITestNativeAzureFileSystemConcurrencyLive.java</exclude>
                     <exclude>**/ITestNativeAzureFileSystemLive.java</exclude>
                     <exclude>**/ITestNativeAzureFSPageBlobLive.java</exclude>
+                    <exclude>**/ITestAzureBlobFileSystemRandomRead.java</exclude>
                     <exclude>**/ITestWasbRemoteCallHelper.java</exclude>
                     <exclude>**/ITestBlockBlobInputStream.java</exclude>
+                    <exclude>**/ITestWasbAbfsCompatibility.java</exclude>
+                    <exclude>**/ITestNativeFileSystemStatistics.java</exclude>
                   </excludes>
                 </configuration>
               </execution>
@@ -424,14 +790,18 @@
                     <fs.azure.scale.test.list.performance.files>${fs.azure.scale.test.list.performance.files}</fs.azure.scale.test.list.performance.files>
                   </systemPropertyVariables>
                   <includes>
+                    <include>**/ITestWasbAbfsCompatibility.java</include>
                     <include>**/ITestFileSystemOperationsExceptionHandlingMultiThreaded.java</include>
                     <include>**/ITestFileSystemOperationsWithThreads.java</include>
                     <include>**/ITestOutOfBandAzureBlobOperationsLive.java</include>
                     <include>**/ITestNativeAzureFileSystemAuthorizationWithOwner.java</include>
                     <include>**/ITestNativeAzureFileSystemConcurrencyLive.java</include>
                     <include>**/ITestNativeAzureFileSystemLive.java</include>
+                    <include>**/ITestNativeAzureFSPageBlobLive.java</include>
+                    <include>**/ITestAzureBlobFileSystemRandomRead.java</include>
                     <include>**/ITestWasbRemoteCallHelper.java</include>
                     <include>**/ITestBlockBlobInputStream.java</include>
+                    <include>**/ITestNativeFileSystemStatistics.java</include>
                   </includes>
                 </configuration>
               </execution>
@@ -440,6 +810,7 @@
         </plugins>
       </build>
     </profile>
+
     <profile>
       <id>sequential-tests</id>
       <activation>

+ 49 - 0
hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml

@@ -0,0 +1,49 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!DOCTYPE suppressions PUBLIC
+        "-//Puppy Crawl//DTD Suppressions 1.0//EN"
+        "http://www.puppycrawl.com/dtds/suppressions_1_0.dtd">
+
+
+<!--
+  Licensed 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.
+-->
+
+<!--
+  Checkstyle configuration that checks the sun coding conventions from:
+
+    - the Java Language Specification at
+      http://java.sun.com/docs/books/jls/second_edition/html/index.html
+
+    - the Sun Code Conventions at http://java.sun.com/docs/codeconv/
+
+    - the Javadoc guidelines at
+      http://java.sun.com/j2se/javadoc/writingdoccomments/index.html
+
+    - the JDK Api documentation http://java.sun.com/j2se/docs/api/index.html
+
+    - some best practices
+
+  Checkstyle is very configurable. Be sure to read the documentation at
+  http://checkstyle.sf.net (or in your downloaded distribution).
+
+  Most Checks are configurable, be sure to consult the documentation.
+  To completely disable a check, just comment it out or delete it from the file.
+  Finally, it is worth reading the documentation.
+-->
+
+<suppressions>
+    <suppress checks="ParameterNumber|MagicNumber"
+              files="org[\\/]apache[\\/]hadoop[\\/]fs[\\/]azurebfs[\\/]AzureBlobFileSystemStore.java"/>
+    <suppress checks="ParameterNumber|MagicNumber"
+              files="org[\\/]apache[\\/]hadoop[\\/]fs[\\/]azurebfs[\\/]utils[\\/]Base64.java"/>
+</suppressions>

+ 1 - 1
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/ClientThrottlingAnalyzer.java

@@ -99,7 +99,7 @@ class ClientThrottlingAnalyzer {
     this.blobMetrics = new AtomicReference<BlobOperationMetrics>(
         new BlobOperationMetrics(System.currentTimeMillis()));
     this.timer = new Timer(
-        String.format("wasb-timer-client-throttling-analyzer-%s", name));
+        String.format("wasb-timer-client-throttling-analyzer-%s", name), true);
     this.timer.schedule(new TimerTaskImpl(),
         analysisPeriodMs,
         analysisPeriodMs);

+ 46 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfs.java

@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegateToFileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+
+/**
+ * Azure Blob File System implementation of AbstractFileSystem.
+ * This impl delegates to the old FileSystem
+ */
+@InterfaceStability.Evolving
+public class Abfs extends DelegateToFileSystem {
+
+  Abfs(final URI theUri, final Configuration conf) throws IOException,
+      URISyntaxException {
+    super(theUri, new AzureBlobFileSystem(), conf, FileSystemUriSchemes.ABFS_SCHEME, false);
+  }
+
+  @Override
+  public int getUriDefaultPort() {
+    return -1;
+  }
+}

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

@@ -0,0 +1,576 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.util.Map;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.IntegerConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.LongConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.StringConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.Base64StringConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.annotations.ConfigurationValidationAnnotations.BooleanConfigurationValidatorAnnotation;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.ConfigurationPropertyNotFoundException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TokenAccessProviderException;
+import org.apache.hadoop.fs.azurebfs.diagnostics.Base64StringConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.diagnostics.BooleanConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.diagnostics.IntegerConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.diagnostics.LongConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.diagnostics.StringConfigurationBasicValidator;
+import org.apache.hadoop.fs.azurebfs.extensions.CustomTokenProviderAdaptee;
+import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
+import org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider;
+import org.apache.hadoop.fs.azurebfs.oauth2.CustomTokenProviderAdapter;
+import org.apache.hadoop.fs.azurebfs.oauth2.MsiTokenProvider;
+import org.apache.hadoop.fs.azurebfs.oauth2.RefreshTokenBasedTokenProvider;
+import org.apache.hadoop.fs.azurebfs.oauth2.UserPasswordTokenProvider;
+import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.KeyProvider;
+import org.apache.hadoop.fs.azurebfs.services.SimpleKeyProvider;
+import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
+import org.apache.hadoop.security.ProviderUtils;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.*;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.*;
+
+/**
+ * Configuration for Azure Blob FileSystem.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class AbfsConfiguration{
+  private final Configuration rawConfig;
+  private final String accountName;
+  private final boolean isSecure;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_WRITE_BUFFER_SIZE,
+      MinValue = MIN_BUFFER_SIZE,
+      MaxValue = MAX_BUFFER_SIZE,
+      DefaultValue = DEFAULT_WRITE_BUFFER_SIZE)
+  private int writeBufferSize;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_READ_BUFFER_SIZE,
+      MinValue = MIN_BUFFER_SIZE,
+      MaxValue = MAX_BUFFER_SIZE,
+      DefaultValue = DEFAULT_READ_BUFFER_SIZE)
+  private int readBufferSize;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_MIN_BACKOFF_INTERVAL,
+      DefaultValue = DEFAULT_MIN_BACKOFF_INTERVAL)
+  private int minBackoffInterval;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_MAX_BACKOFF_INTERVAL,
+      DefaultValue = DEFAULT_MAX_BACKOFF_INTERVAL)
+  private int maxBackoffInterval;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_BACKOFF_INTERVAL,
+      DefaultValue = DEFAULT_BACKOFF_INTERVAL)
+  private int backoffInterval;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_MAX_IO_RETRIES,
+      MinValue = 0,
+      DefaultValue = DEFAULT_MAX_RETRY_ATTEMPTS)
+  private int maxIoRetries;
+
+  @LongConfigurationValidatorAnnotation(ConfigurationKey = AZURE_BLOCK_SIZE_PROPERTY_NAME,
+      MinValue = 0,
+      MaxValue = MAX_AZURE_BLOCK_SIZE,
+      DefaultValue = MAX_AZURE_BLOCK_SIZE)
+  private long azureBlockSize;
+
+  @StringConfigurationValidatorAnnotation(ConfigurationKey = AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME,
+      DefaultValue = AZURE_BLOCK_LOCATION_HOST_DEFAULT)
+  private String azureBlockLocationHost;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_CONCURRENT_CONNECTION_VALUE_OUT,
+      MinValue = 1,
+      DefaultValue = MAX_CONCURRENT_WRITE_THREADS)
+  private int maxConcurrentWriteThreads;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = AZURE_CONCURRENT_CONNECTION_VALUE_IN,
+      MinValue = 1,
+      DefaultValue = MAX_CONCURRENT_READ_THREADS)
+  private int maxConcurrentReadThreads;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = AZURE_TOLERATE_CONCURRENT_APPEND,
+      DefaultValue = DEFAULT_READ_TOLERATE_CONCURRENT_APPEND)
+  private boolean tolerateOobAppends;
+
+  @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ATOMIC_RENAME_KEY,
+      DefaultValue = DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES)
+  private String azureAtomicDirs;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION,
+      DefaultValue = DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION)
+  private boolean createRemoteFileSystemDuringInitialization;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION,
+      DefaultValue = DEFAULT_AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION)
+  private boolean skipUserGroupMetadataDuringInitialization;
+
+  @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_READ_AHEAD_QUEUE_DEPTH,
+      DefaultValue = DEFAULT_READ_AHEAD_QUEUE_DEPTH)
+  private int readAheadQueueDepth;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ENABLE_FLUSH,
+      DefaultValue = DEFAULT_ENABLE_FLUSH)
+  private boolean enableFlush;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ENABLE_AUTOTHROTTLING,
+      DefaultValue = DEFAULT_ENABLE_AUTOTHROTTLING)
+  private boolean enableAutoThrottling;
+
+  @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_USER_AGENT_PREFIX_KEY,
+      DefaultValue = "")
+  private String userAgentId;
+
+  @BooleanConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ENABLE_DELEGATION_TOKEN,
+      DefaultValue = DEFAULT_ENABLE_DELEGATION_TOKEN)
+  private boolean enableDelegationToken;
+
+  private Map<String, String> storageAccountKeys;
+
+  public AbfsConfiguration(final Configuration rawConfig, String accountName)
+      throws IllegalAccessException, InvalidConfigurationValueException, IOException {
+    this.rawConfig = ProviderUtils.excludeIncompatibleCredentialProviders(
+        rawConfig, AzureBlobFileSystem.class);
+    this.accountName = accountName;
+    this.isSecure = getBoolean(FS_AZURE_SECURE_MODE, false);
+
+    validateStorageAccountKeys();
+    Field[] fields = this.getClass().getDeclaredFields();
+    for (Field field : fields) {
+      field.setAccessible(true);
+      if (field.isAnnotationPresent(IntegerConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateInt(field));
+      } else if (field.isAnnotationPresent(LongConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateLong(field));
+      } else if (field.isAnnotationPresent(StringConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateString(field));
+      } else if (field.isAnnotationPresent(Base64StringConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateBase64String(field));
+      } else if (field.isAnnotationPresent(BooleanConfigurationValidatorAnnotation.class)) {
+        field.set(this, validateBoolean(field));
+      }
+    }
+  }
+
+  /**
+   * Appends an account name to a configuration key yielding the
+   * account-specific form.
+   * @param key Account-agnostic configuration key
+   * @return Account-specific configuration key
+   */
+  public String accountConf(String key) {
+    return key + "." + accountName;
+  }
+
+  /**
+   * Returns the account-specific value if it exists, then looks for an
+   * account-agnostic value.
+   * @param key Account-agnostic configuration key
+   * @return value if one exists, else null
+   */
+  public String get(String key) {
+    return rawConfig.get(accountConf(key), rawConfig.get(key));
+  }
+
+  /**
+   * 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 boolean getBoolean(String key, boolean defaultValue) {
+    return rawConfig.getBoolean(accountConf(key), rawConfig.getBoolean(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 long getLong(String key, long defaultValue) {
+    return rawConfig.getLong(accountConf(key), rawConfig.getLong(key, defaultValue));
+  }
+
+  /**
+   * Returns the account-specific password in string form if it exists, then
+   * looks for an account-agnostic value.
+   * @param key Account-agnostic configuration key
+   * @return value in String form if one exists, else null
+   * @throws IOException
+   */
+  public String getPasswordString(String key) throws IOException {
+    char[] passchars = rawConfig.getPassword(accountConf(key));
+    if (passchars == null) {
+      passchars = rawConfig.getPassword(key);
+    }
+    if (passchars != null) {
+      return new String(passchars);
+    }
+    return null;
+  }
+
+  /**
+   * Returns the account-specific Class if it exists, then looks for an
+   * account-agnostic value, and finally tries the default value.
+   * @param name Account-agnostic configuration key
+   * @param defaultValue Class returned if none is configured
+   * @param xface Interface shared by all possible values
+   * @return Highest-precedence Class object that was found
+   */
+  public <U> Class<? extends U> getClass(String name, Class<? extends U> defaultValue, Class<U> xface) {
+    return rawConfig.getClass(accountConf(name),
+        rawConfig.getClass(name, defaultValue, xface),
+        xface);
+  }
+
+  /**
+   * Returns the account-specific password in string form if it exists, then
+   * looks for an account-agnostic value.
+   * @param name Account-agnostic configuration key
+   * @param defaultValue Value returned if none is configured
+   * @return value in String form if one exists, else null
+   */
+  public <T extends Enum<T>> T getEnum(String name, T defaultValue) {
+    return rawConfig.getEnum(accountConf(name),
+        rawConfig.getEnum(name, defaultValue));
+  }
+
+  /**
+   * Unsets parameter in the underlying Configuration object.
+   * Provided only as a convenience; does not add any account logic.
+   * @param key Configuration key
+   */
+  public void unset(String key) {
+    rawConfig.unset(key);
+  }
+
+  /**
+   * Sets String in the underlying Configuration object.
+   * Provided only as a convenience; does not add any account logic.
+   * @param key Configuration key
+   * @param value Configuration value
+   */
+  public void set(String key, String value) {
+    rawConfig.set(key, value);
+  }
+
+  /**
+   * Sets boolean in the underlying Configuration object.
+   * Provided only as a convenience; does not add any account logic.
+   * @param key Configuration key
+   * @param value Configuration value
+   */
+  public void setBoolean(String key, boolean value) {
+    rawConfig.setBoolean(key, value);
+  }
+
+  public boolean isSecureMode() {
+    return isSecure;
+  }
+
+  public String getStorageAccountKey() throws AzureBlobFileSystemException {
+    String key;
+    String keyProviderClass = get(AZURE_KEY_ACCOUNT_KEYPROVIDER);
+    KeyProvider keyProvider;
+
+    if (keyProviderClass == null) {
+      // No key provider was provided so use the provided key as is.
+      keyProvider = new SimpleKeyProvider();
+    } else {
+      // create an instance of the key provider class and verify it
+      // implements KeyProvider
+      Object keyProviderObject;
+      try {
+        Class<?> clazz = rawConfig.getClassByName(keyProviderClass);
+        keyProviderObject = clazz.newInstance();
+      } catch (Exception e) {
+        throw new KeyProviderException("Unable to load key provider class.", e);
+      }
+      if (!(keyProviderObject instanceof KeyProvider)) {
+        throw new KeyProviderException(keyProviderClass
+                + " specified in config is not a valid KeyProvider class.");
+      }
+      keyProvider = (KeyProvider) keyProviderObject;
+    }
+    key = keyProvider.getStorageAccountKey(accountName, rawConfig);
+
+    if (key == null) {
+      throw new ConfigurationPropertyNotFoundException(accountName);
+    }
+
+    return key;
+  }
+
+  public Configuration getRawConfiguration() {
+    return this.rawConfig;
+  }
+
+  public int getWriteBufferSize() {
+    return this.writeBufferSize;
+  }
+
+  public int getReadBufferSize() {
+    return this.readBufferSize;
+  }
+
+  public int getMinBackoffIntervalMilliseconds() {
+    return this.minBackoffInterval;
+  }
+
+  public int getMaxBackoffIntervalMilliseconds() {
+    return this.maxBackoffInterval;
+  }
+
+  public int getBackoffIntervalMilliseconds() {
+    return this.backoffInterval;
+  }
+
+  public int getMaxIoRetries() {
+    return this.maxIoRetries;
+  }
+
+  public long getAzureBlockSize() {
+    return this.azureBlockSize;
+  }
+
+  public String getAzureBlockLocationHost() {
+    return this.azureBlockLocationHost;
+  }
+
+  public int getMaxConcurrentWriteThreads() {
+    return this.maxConcurrentWriteThreads;
+  }
+
+  public int getMaxConcurrentReadThreads() {
+    return this.maxConcurrentReadThreads;
+  }
+
+  public boolean getTolerateOobAppends() {
+    return this.tolerateOobAppends;
+  }
+
+  public String getAzureAtomicRenameDirs() {
+    return this.azureAtomicDirs;
+  }
+
+  public boolean getCreateRemoteFileSystemDuringInitialization() {
+    return this.createRemoteFileSystemDuringInitialization;
+  }
+
+  public boolean getSkipUserGroupMetadataDuringInitialization() {
+    return this.skipUserGroupMetadataDuringInitialization;
+  }
+
+  public int getReadAheadQueueDepth() {
+    return this.readAheadQueueDepth;
+  }
+
+  public boolean isFlushEnabled() {
+    return this.enableFlush;
+  }
+
+  public boolean isAutoThrottlingEnabled() {
+    return this.enableAutoThrottling;
+  }
+
+  public String getCustomUserAgentPrefix() {
+    return this.userAgentId;
+  }
+
+  public SSLSocketFactoryEx.SSLChannelMode getPreferredSSLFactoryOption() {
+    return getEnum(FS_AZURE_SSL_CHANNEL_MODE_KEY, DEFAULT_FS_AZURE_SSL_CHANNEL_MODE);
+  }
+
+  public AuthType getAuthType(String accountName) {
+    return getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey);
+  }
+
+  public boolean isDelegationTokenManagerEnabled() {
+    return enableDelegationToken;
+  }
+
+  public AbfsDelegationTokenManager getDelegationTokenManager() throws IOException {
+    return new AbfsDelegationTokenManager(getRawConfiguration());
+  }
+
+  public AccessTokenProvider getTokenProvider() throws TokenAccessProviderException {
+    AuthType authType = getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey);
+    if (authType == AuthType.OAuth) {
+      try {
+        Class<? extends AccessTokenProvider> tokenProviderClass =
+                getClass(FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME, null,
+                        AccessTokenProvider.class);
+        AccessTokenProvider tokenProvider = null;
+        if (tokenProviderClass == ClientCredsTokenProvider.class) {
+          String authEndpoint = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT);
+          String clientId = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID);
+          String clientSecret = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET);
+          tokenProvider = new ClientCredsTokenProvider(authEndpoint, clientId, clientSecret);
+        } else if (tokenProviderClass == UserPasswordTokenProvider.class) {
+          String authEndpoint = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT);
+          String username = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_USER_NAME);
+          String password = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_USER_PASSWORD);
+          tokenProvider = new UserPasswordTokenProvider(authEndpoint, username, password);
+        } else if (tokenProviderClass == MsiTokenProvider.class) {
+          String tenantGuid = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT);
+          String clientId = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID);
+          tokenProvider = new MsiTokenProvider(tenantGuid, clientId);
+        } else if (tokenProviderClass == RefreshTokenBasedTokenProvider.class) {
+          String refreshToken = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN);
+          String clientId = getPasswordString(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID);
+          tokenProvider = new RefreshTokenBasedTokenProvider(clientId, refreshToken);
+        } else {
+          throw new IllegalArgumentException("Failed to initialize " + tokenProviderClass);
+        }
+        return tokenProvider;
+      } catch(IllegalArgumentException e) {
+        throw e;
+      } catch (Exception e) {
+        throw new TokenAccessProviderException("Unable to load key provider class.", e);
+      }
+
+    } else if (authType == AuthType.Custom) {
+      try {
+        String configKey = FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME;
+        Class<? extends CustomTokenProviderAdaptee> customTokenProviderClass =
+                getClass(configKey, null, CustomTokenProviderAdaptee.class);
+        if (customTokenProviderClass == null) {
+          throw new IllegalArgumentException(
+                  String.format("The configuration value for \"%s\" is invalid.", configKey));
+        }
+        CustomTokenProviderAdaptee azureTokenProvider = ReflectionUtils
+                .newInstance(customTokenProviderClass, rawConfig);
+        if (azureTokenProvider == null) {
+          throw new IllegalArgumentException("Failed to initialize " + customTokenProviderClass);
+        }
+        azureTokenProvider.initialize(rawConfig, accountName);
+        return new CustomTokenProviderAdapter(azureTokenProvider);
+      } catch(IllegalArgumentException e) {
+        throw e;
+      } catch (Exception e) {
+        throw new TokenAccessProviderException("Unable to load custom token provider class.", e);
+      }
+
+    } else {
+      throw new TokenAccessProviderException(String.format(
+              "Invalid auth type: %s is being used, expecting OAuth", authType));
+    }
+  }
+
+  void validateStorageAccountKeys() throws InvalidConfigurationValueException {
+    Base64StringConfigurationBasicValidator validator = new Base64StringConfigurationBasicValidator(
+        FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME, "", true);
+    this.storageAccountKeys = rawConfig.getValByRegex(FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX);
+
+    for (Map.Entry<String, String> account : storageAccountKeys.entrySet()) {
+      validator.validate(account.getValue());
+    }
+  }
+
+  int validateInt(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    IntegerConfigurationValidatorAnnotation validator = field.getAnnotation(IntegerConfigurationValidatorAnnotation.class);
+    String value = get(validator.ConfigurationKey());
+
+    // validate
+    return new IntegerConfigurationBasicValidator(
+        validator.MinValue(),
+        validator.MaxValue(),
+        validator.DefaultValue(),
+        validator.ConfigurationKey(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  long validateLong(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    LongConfigurationValidatorAnnotation validator = field.getAnnotation(LongConfigurationValidatorAnnotation.class);
+    String value = rawConfig.get(validator.ConfigurationKey());
+
+    // validate
+    return new LongConfigurationBasicValidator(
+        validator.MinValue(),
+        validator.MaxValue(),
+        validator.DefaultValue(),
+        validator.ConfigurationKey(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  String validateString(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    StringConfigurationValidatorAnnotation validator = field.getAnnotation(StringConfigurationValidatorAnnotation.class);
+    String value = rawConfig.get(validator.ConfigurationKey());
+
+    // validate
+    return new StringConfigurationBasicValidator(
+        validator.ConfigurationKey(),
+        validator.DefaultValue(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  String validateBase64String(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    Base64StringConfigurationValidatorAnnotation validator = field.getAnnotation((Base64StringConfigurationValidatorAnnotation.class));
+    String value = rawConfig.get(validator.ConfigurationKey());
+
+    // validate
+    return new Base64StringConfigurationBasicValidator(
+        validator.ConfigurationKey(),
+        validator.DefaultValue(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  boolean validateBoolean(Field field) throws IllegalAccessException, InvalidConfigurationValueException {
+    BooleanConfigurationValidatorAnnotation validator = field.getAnnotation(BooleanConfigurationValidatorAnnotation.class);
+    String value = rawConfig.get(validator.ConfigurationKey());
+
+    // validate
+    return new BooleanConfigurationBasicValidator(
+        validator.ConfigurationKey(),
+        validator.DefaultValue(),
+        validator.ThrowIfInvalid()).validate(value);
+  }
+
+  @VisibleForTesting
+  void setReadBufferSize(int bufferSize) {
+    this.readBufferSize = bufferSize;
+  }
+
+  @VisibleForTesting
+  void setWriteBufferSize(int bufferSize) {
+    this.writeBufferSize = bufferSize;
+  }
+
+  @VisibleForTesting
+  void setEnableFlush(boolean enableFlush) {
+    this.enableFlush = enableFlush;
+  }
+}

+ 46 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/Abfss.java

@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegateToFileSystem;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+
+/**
+ * Azure Blob File System implementation of AbstractFileSystem.
+ * This impl delegates to the old FileSystem
+ */
+@InterfaceStability.Evolving
+public class Abfss extends DelegateToFileSystem {
+
+  Abfss(final URI theUri, final Configuration conf) throws IOException,
+      URISyntaxException {
+    super(theUri, new SecureAzureBlobFileSystem(), conf, FileSystemUriSchemes.ABFS_SECURE_SCHEME, false);
+  }
+
+  @Override
+  public int getUriDefaultPort() {
+    return -1;
+  }
+}

+ 953 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java

@@ -0,0 +1,953 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClientThrottlingIntercept;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+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.FileSystemOperationUnhandledException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
+import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * A {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on <a
+ * href="http://store.azure.com/">Windows Azure</a>
+ */
+@InterfaceStability.Evolving
+public class AzureBlobFileSystem extends FileSystem {
+  public static final Logger LOG = LoggerFactory.getLogger(AzureBlobFileSystem.class);
+  private URI uri;
+  private Path workingDir;
+  private UserGroupInformation userGroupInformation;
+  private String user;
+  private String primaryUserGroup;
+  private AzureBlobFileSystemStore abfsStore;
+  private boolean isClosed;
+
+  private boolean delegationTokenEnabled = false;
+  private AbfsDelegationTokenManager delegationTokenManager;
+
+  @Override
+  public void initialize(URI uri, Configuration configuration)
+      throws IOException {
+    uri = ensureAuthority(uri, configuration);
+    super.initialize(uri, configuration);
+    setConf(configuration);
+
+    LOG.debug("Initializing AzureBlobFileSystem for {}", uri);
+
+    this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
+    this.userGroupInformation = UserGroupInformation.getCurrentUser();
+    this.user = userGroupInformation.getUserName();
+    this.abfsStore = new AzureBlobFileSystemStore(uri, this.isSecure(), configuration, userGroupInformation);
+    final AbfsConfiguration abfsConfiguration = abfsStore.getAbfsConfiguration();
+
+    this.setWorkingDirectory(this.getHomeDirectory());
+
+    if (abfsConfiguration.getCreateRemoteFileSystemDuringInitialization()) {
+      if (!this.fileSystemExists()) {
+        try {
+          this.createFileSystem();
+        } catch (AzureBlobFileSystemException ex) {
+          checkException(null, ex, AzureServiceErrorCode.FILE_SYSTEM_ALREADY_EXISTS);
+        }
+      }
+    }
+
+    if (!abfsConfiguration.getSkipUserGroupMetadataDuringInitialization()) {
+      this.primaryUserGroup = userGroupInformation.getPrimaryGroupName();
+    } else {
+      //Provide a default group name
+      this.primaryUserGroup = this.user;
+    }
+
+    if (UserGroupInformation.isSecurityEnabled()) {
+      this.delegationTokenEnabled = abfsConfiguration.isDelegationTokenManagerEnabled();
+
+      if (this.delegationTokenEnabled) {
+        LOG.debug("Initializing DelegationTokenManager for {}", uri);
+        this.delegationTokenManager = abfsConfiguration.getDelegationTokenManager();
+      }
+    }
+
+    AbfsClientThrottlingIntercept.initializeSingleton(abfsConfiguration.isAutoThrottlingEnabled());
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "AzureBlobFileSystem{");
+    sb.append("uri=").append(uri);
+    sb.append(", user='").append(user).append('\'');
+    sb.append(", primaryUserGroup='").append(primaryUserGroup).append('\'');
+    sb.append('}');
+    return sb.toString();
+  }
+
+  public boolean isSecure() {
+    return false;
+  }
+
+  @Override
+  public URI getUri() {
+    return this.uri;
+  }
+
+  @Override
+  public FSDataInputStream open(final Path path, final int bufferSize) throws IOException {
+    LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize);
+
+    try {
+      InputStream inputStream = abfsStore.openFileForRead(makeQualified(path), statistics);
+      return new FSDataInputStream(inputStream);
+    } catch(AzureBlobFileSystemException ex) {
+      checkException(path, ex);
+      return null;
+    }
+  }
+
+  @Override
+  public FSDataOutputStream create(final Path f, final FsPermission permission, final boolean overwrite, final int bufferSize,
+      final short replication, final long blockSize, final Progressable progress) throws IOException {
+    LOG.debug("AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}",
+        f,
+        permission,
+        overwrite,
+        blockSize);
+
+    try {
+      OutputStream outputStream = abfsStore.createFile(makeQualified(f), overwrite,
+          permission == null ? FsPermission.getFileDefault() : permission, FsPermission.getUMask(getConf()));
+      return new FSDataOutputStream(outputStream, statistics);
+    } catch(AzureBlobFileSystemException ex) {
+      checkException(f, ex);
+      return null;
+    }
+  }
+
+  @Override
+  @SuppressWarnings("deprecation")
+  public FSDataOutputStream createNonRecursive(final Path f, final FsPermission permission,
+      final boolean overwrite, final int bufferSize, final short replication, final long blockSize,
+      final Progressable progress) throws IOException {
+
+    final Path parent = f.getParent();
+    final FileStatus parentFileStatus = tryGetFileStatus(parent);
+
+    if (parentFileStatus == null) {
+      throw new FileNotFoundException("Cannot create file "
+          + f.getName() + " because parent folder does not exist.");
+    }
+
+    return create(f, permission, overwrite, bufferSize, replication, blockSize, progress);
+  }
+
+  @Override
+  @SuppressWarnings("deprecation")
+  public FSDataOutputStream createNonRecursive(final Path f, final FsPermission permission,
+      final EnumSet<CreateFlag> flags, final int bufferSize, final short replication, final long blockSize,
+      final Progressable progress) throws IOException {
+
+    // Check if file should be appended or overwritten. Assume that the file
+    // is overwritten on if the CREATE and OVERWRITE create flags are set.
+    final EnumSet<CreateFlag> createflags =
+        EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE);
+    final boolean overwrite = flags.containsAll(createflags);
+
+    // Delegate the create non-recursive call.
+    return this.createNonRecursive(f, permission, overwrite,
+        bufferSize, replication, blockSize, progress);
+  }
+
+  @Override
+  @SuppressWarnings("deprecation")
+  public FSDataOutputStream createNonRecursive(final Path f,
+      final boolean overwrite, final int bufferSize, final short replication, final long blockSize,
+      final Progressable progress) throws IOException {
+    return this.createNonRecursive(f, FsPermission.getFileDefault(),
+        overwrite, bufferSize, replication, blockSize, progress);
+  }
+
+  @Override
+  public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress) throws IOException {
+    LOG.debug(
+        "AzureBlobFileSystem.append path: {} bufferSize: {}",
+        f.toString(),
+        bufferSize);
+
+    try {
+      OutputStream outputStream = abfsStore.openFileForWrite(makeQualified(f), false);
+      return new FSDataOutputStream(outputStream, statistics);
+    } catch(AzureBlobFileSystemException ex) {
+      checkException(f, ex);
+      return null;
+    }
+  }
+
+  public boolean rename(final Path src, final Path dst) throws IOException {
+    LOG.debug(
+        "AzureBlobFileSystem.rename src: {} dst: {}", src.toString(), dst.toString());
+
+    Path parentFolder = src.getParent();
+    if (parentFolder == null) {
+      return false;
+    }
+
+    final FileStatus dstFileStatus = tryGetFileStatus(dst);
+    try {
+      String sourceFileName = src.getName();
+      Path adjustedDst = dst;
+
+      if (dstFileStatus != null) {
+        if (!dstFileStatus.isDirectory()) {
+          return src.equals(dst);
+        }
+
+        adjustedDst = new Path(dst, sourceFileName);
+      }
+
+      abfsStore.rename(makeQualified(src), makeQualified(adjustedDst));
+      return true;
+    } catch(AzureBlobFileSystemException ex) {
+      checkException(
+              src,
+              ex,
+              AzureServiceErrorCode.PATH_ALREADY_EXISTS,
+              AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH,
+              AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND,
+              AzureServiceErrorCode.INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE,
+              AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND,
+              AzureServiceErrorCode.INTERNAL_OPERATION_ABORT);
+      return false;
+    }
+
+  }
+
+  @Override
+  public boolean delete(final Path f, final boolean recursive) throws IOException {
+    LOG.debug(
+        "AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), recursive);
+
+    if (f.isRoot()) {
+      if (!recursive) {
+        return false;
+      }
+
+      return deleteRoot();
+    }
+
+    try {
+      abfsStore.delete(makeQualified(f), recursive);
+      return true;
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(f, ex, AzureServiceErrorCode.PATH_NOT_FOUND);
+      return false;
+    }
+
+  }
+
+  @Override
+  public FileStatus[] listStatus(final Path f) throws IOException {
+    LOG.debug(
+        "AzureBlobFileSystem.listStatus path: {}", f.toString());
+
+    try {
+      FileStatus[] result = abfsStore.listStatus(makeQualified(f));
+      return result;
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(f, ex);
+      return null;
+    }
+  }
+
+  @Override
+  public boolean mkdirs(final Path f, final FsPermission permission) throws IOException {
+    LOG.debug(
+        "AzureBlobFileSystem.mkdirs path: {} permissions: {}", f, permission);
+
+    final Path parentFolder = f.getParent();
+    if (parentFolder == null) {
+      // Cannot create root
+      return true;
+    }
+
+    try {
+      abfsStore.createDirectory(makeQualified(f), permission == null ? FsPermission.getDirDefault() : permission,
+          FsPermission.getUMask(getConf()));
+      return true;
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(f, ex, AzureServiceErrorCode.PATH_ALREADY_EXISTS);
+      return true;
+    }
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    if (isClosed) {
+      return;
+    }
+
+    super.close();
+    LOG.debug("AzureBlobFileSystem.close");
+    this.isClosed = true;
+  }
+
+  @Override
+  public FileStatus getFileStatus(final Path f) throws IOException {
+    LOG.debug("AzureBlobFileSystem.getFileStatus path: {}", f);
+
+    try {
+      return abfsStore.getFileStatus(makeQualified(f));
+    } catch(AzureBlobFileSystemException ex) {
+      checkException(f, ex);
+      return null;
+    }
+  }
+
+  /**
+   * Qualify a path to one which uses this FileSystem and, if relative,
+   * made absolute.
+   * @param path to qualify.
+   * @return this path if it contains a scheme and authority and is absolute, or
+   * a new path that includes a path and authority and is fully qualified
+   * @see Path#makeQualified(URI, Path)
+   * @throws IllegalArgumentException if the path has a schema/URI different
+   * from this FileSystem.
+   */
+  @Override
+  public Path makeQualified(Path path) {
+    // To support format: abfs://{dfs.nameservices}/file/path,
+    // path need to be first converted to URI, then get the raw path string,
+    // during which {dfs.nameservices} will be omitted.
+    if (path != null) {
+      String uriPath = path.toUri().getPath();
+      path = uriPath.isEmpty() ? path : new Path(uriPath);
+    }
+    return super.makeQualified(path);
+  }
+
+
+  @Override
+  public Path getWorkingDirectory() {
+    return this.workingDir;
+  }
+
+  @Override
+  public void setWorkingDirectory(final Path newDir) {
+    if (newDir.isAbsolute()) {
+      this.workingDir = newDir;
+    } else {
+      this.workingDir = new Path(workingDir, newDir);
+    }
+  }
+
+  @Override
+  public String getScheme() {
+    return FileSystemUriSchemes.ABFS_SCHEME;
+  }
+
+  @Override
+  public Path getHomeDirectory() {
+    return makeQualified(new Path(
+            FileSystemConfigurations.USER_HOME_DIRECTORY_PREFIX
+                + "/" + this.userGroupInformation.getShortUserName()));
+  }
+
+  /**
+   * Return an array containing hostnames, offset and size of
+   * portions of the given file. For ABFS we'll just lie and give
+   * fake hosts to make sure we get many splits in MR jobs.
+   */
+  @Override
+  public BlockLocation[] getFileBlockLocations(FileStatus file,
+      long start, long len) {
+    if (file == null) {
+      return null;
+    }
+
+    if ((start < 0) || (len < 0)) {
+      throw new IllegalArgumentException("Invalid start or len parameter");
+    }
+
+    if (file.getLen() < start) {
+      return new BlockLocation[0];
+    }
+    final String blobLocationHost = abfsStore.getAbfsConfiguration().getAzureBlockLocationHost();
+
+    final String[] name = { blobLocationHost };
+    final String[] host = { blobLocationHost };
+    long blockSize = file.getBlockSize();
+    if (blockSize <= 0) {
+      throw new IllegalArgumentException(
+          "The block size for the given file is not a positive number: "
+              + blockSize);
+    }
+    int numberOfLocations = (int) (len / blockSize)
+        + ((len % blockSize == 0) ? 0 : 1);
+    BlockLocation[] locations = new BlockLocation[numberOfLocations];
+    for (int i = 0; i < locations.length; i++) {
+      long currentOffset = start + (i * blockSize);
+      long currentLength = Math.min(blockSize, start + len - currentOffset);
+      locations[i] = new BlockLocation(name, host, currentOffset, currentLength);
+    }
+
+    return locations;
+  }
+
+  @Override
+  protected void finalize() throws Throwable {
+    LOG.debug("finalize() called.");
+    close();
+    super.finalize();
+  }
+
+  public String getOwnerUser() {
+    return user;
+  }
+
+  public String getOwnerUserPrimaryGroup() {
+    return primaryUserGroup;
+  }
+
+  private boolean deleteRoot() throws IOException {
+    LOG.debug("Deleting root content");
+
+    final ExecutorService executorService = Executors.newFixedThreadPool(10);
+
+    try {
+      final FileStatus[] ls = listStatus(makeQualified(new Path(File.separator)));
+      final ArrayList<Future> deleteTasks = new ArrayList<>();
+      for (final FileStatus fs : ls) {
+        final Future deleteTask = executorService.submit(new Callable<Void>() {
+          @Override
+          public Void call() throws Exception {
+            delete(fs.getPath(), fs.isDirectory());
+            return null;
+          }
+        });
+        deleteTasks.add(deleteTask);
+      }
+
+      for (final Future deleteTask : deleteTasks) {
+        execute("deleteRoot", new Callable<Void>() {
+          @Override
+          public Void call() throws Exception {
+            deleteTask.get();
+            return null;
+          }
+        });
+      }
+    }
+    finally {
+      executorService.shutdownNow();
+    }
+
+    return true;
+  }
+
+   /**
+   * Set owner of a path (i.e. a file or a directory).
+   * The parameters owner and group cannot both be null.
+   *
+   * @param path  The path
+   * @param owner If it is null, the original username remains unchanged.
+   * @param group If it is null, the original groupname remains unchanged.
+   */
+  @Override
+  public void setOwner(final Path path, final String owner, final String group)
+      throws IOException {
+    LOG.debug(
+        "AzureBlobFileSystem.setOwner path: {}", path);
+    if (!getIsNamespaceEnabeld()) {
+      super.setOwner(path, owner, group);
+      return;
+    }
+
+    if ((owner == null || owner.isEmpty()) && (group == null || group.isEmpty())) {
+      throw new IllegalArgumentException("A valid owner or group must be specified.");
+    }
+
+    try {
+      abfsStore.setOwner(makeQualified(path),
+              owner,
+              group);
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(path, ex);
+    }
+  }
+
+  /**
+   * Set permission of a path.
+   *
+   * @param path       The path
+   * @param permission Access permission
+   */
+  @Override
+  public void setPermission(final Path path, final FsPermission permission)
+      throws IOException {
+    LOG.debug("AzureBlobFileSystem.setPermission path: {}", path);
+    if (!getIsNamespaceEnabeld()) {
+      super.setPermission(path, permission);
+      return;
+    }
+
+    if (permission == null) {
+      throw new IllegalArgumentException("The permission can't be null");
+    }
+
+    try {
+      abfsStore.setPermission(makeQualified(path),
+              permission);
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(path, ex);
+    }
+  }
+
+  /**
+   * Modifies ACL entries of files and directories.  This method can add new ACL
+   * entries or modify the permissions on existing ACL entries.  All existing
+   * ACL entries that are not specified in this call are retained without
+   * changes.  (Modifications are merged into the current ACL.)
+   *
+   * @param path    Path to modify
+   * @param aclSpec List of AbfsAclEntry describing modifications
+   * @throws IOException if an ACL could not be modified
+   */
+  @Override
+  public void modifyAclEntries(final Path path, final List<AclEntry> aclSpec)
+      throws IOException {
+    LOG.debug("AzureBlobFileSystem.modifyAclEntries path: {}", path.toString());
+
+    if (!getIsNamespaceEnabeld()) {
+      throw new UnsupportedOperationException(
+          "modifyAclEntries is only supported by storage accounts with the "
+          + "hierarchical namespace enabled.");
+    }
+
+    if (aclSpec == null || aclSpec.isEmpty()) {
+      throw new IllegalArgumentException("The value of the aclSpec parameter is invalid.");
+    }
+
+    try {
+      abfsStore.modifyAclEntries(makeQualified(path),
+              aclSpec);
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(path, ex);
+    }
+  }
+
+  /**
+   * Removes ACL entries from files and directories.  Other ACL entries are
+   * retained.
+   *
+   * @param path    Path to modify
+   * @param aclSpec List of AclEntry describing entries to remove
+   * @throws IOException if an ACL could not be modified
+   */
+  @Override
+  public void removeAclEntries(final Path path, final List<AclEntry> aclSpec)
+      throws IOException {
+    LOG.debug("AzureBlobFileSystem.removeAclEntries path: {}", path);
+
+    if (!getIsNamespaceEnabeld()) {
+      throw new UnsupportedOperationException(
+          "removeAclEntries is only supported by storage accounts with the "
+          + "hierarchical namespace enabled.");
+    }
+
+    if (aclSpec == null || aclSpec.isEmpty()) {
+      throw new IllegalArgumentException("The aclSpec argument is invalid.");
+    }
+
+    try {
+      abfsStore.removeAclEntries(makeQualified(path), aclSpec);
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(path, ex);
+    }
+  }
+
+  /**
+   * Removes all default ACL entries from files and directories.
+   *
+   * @param path Path to modify
+   * @throws IOException if an ACL could not be modified
+   */
+  @Override
+  public void removeDefaultAcl(final Path path) throws IOException {
+    LOG.debug("AzureBlobFileSystem.removeDefaultAcl path: {}", path);
+
+    if (!getIsNamespaceEnabeld()) {
+      throw new UnsupportedOperationException(
+          "removeDefaultAcl is only supported by storage accounts with the "
+          + "hierarchical namespace enabled.");
+    }
+
+    try {
+      abfsStore.removeDefaultAcl(makeQualified(path));
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(path, ex);
+    }
+  }
+
+  /**
+   * Removes all but the base ACL entries of files and directories.  The entries
+   * for user, group, and others are retained for compatibility with permission
+   * bits.
+   *
+   * @param path Path to modify
+   * @throws IOException if an ACL could not be removed
+   */
+  @Override
+  public void removeAcl(final Path path) throws IOException {
+    LOG.debug("AzureBlobFileSystem.removeAcl path: {}", path);
+
+    if (!getIsNamespaceEnabeld()) {
+      throw new UnsupportedOperationException(
+          "removeAcl is only supported by storage accounts with the "
+          + "hierarchical namespace enabled.");
+    }
+
+    try {
+      abfsStore.removeAcl(makeQualified(path));
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(path, ex);
+    }
+  }
+
+  /**
+   * Fully replaces ACL of files and directories, discarding all existing
+   * entries.
+   *
+   * @param path    Path to modify
+   * @param aclSpec List of AclEntry describing modifications, must include
+   *                entries for user, group, and others for compatibility with
+   *                permission bits.
+   * @throws IOException if an ACL could not be modified
+   */
+  @Override
+  public void setAcl(final Path path, final List<AclEntry> aclSpec)
+      throws IOException {
+    LOG.debug("AzureBlobFileSystem.setAcl path: {}", path);
+
+    if (!getIsNamespaceEnabeld()) {
+      throw new UnsupportedOperationException(
+          "setAcl is only supported by storage accounts with the hierarchical "
+          + "namespace enabled.");
+    }
+
+    if (aclSpec == null || aclSpec.size() == 0) {
+      throw new IllegalArgumentException("The aclSpec argument is invalid.");
+    }
+
+    try {
+      abfsStore.setAcl(makeQualified(path), aclSpec);
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(path, ex);
+    }
+  }
+
+  /**
+   * Gets the ACL of a file or directory.
+   *
+   * @param path Path to get
+   * @return AbfsAclStatus describing the ACL of the file or directory
+   * @throws IOException if an ACL could not be read
+   */
+  @Override
+  public AclStatus getAclStatus(final Path path) throws IOException {
+    LOG.debug("AzureBlobFileSystem.getAclStatus path: {}", path.toString());
+
+    if (!getIsNamespaceEnabeld()) {
+      throw new UnsupportedOperationException(
+          "getAclStatus is only supported by storage account with the "
+          + "hierarchical namespace enabled.");
+    }
+
+    try {
+      return abfsStore.getAclStatus(makeQualified(path));
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(path, ex);
+      return null;
+    }
+  }
+
+  private FileStatus tryGetFileStatus(final Path f) {
+    try {
+      return getFileStatus(f);
+    } catch (IOException ex) {
+      LOG.debug("File not found {}", f);
+      return null;
+    }
+  }
+
+  private boolean fileSystemExists() throws IOException {
+    LOG.debug(
+            "AzureBlobFileSystem.fileSystemExists uri: {}", uri);
+    try {
+      abfsStore.getFilesystemProperties();
+    } catch (AzureBlobFileSystemException ex) {
+      try {
+        checkException(null, ex);
+        // Because HEAD request won't contain message body,
+        // there is not way to get the storage error code
+        // workaround here is to check its status code.
+      } catch (FileNotFoundException e) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private void createFileSystem() throws IOException {
+    LOG.debug(
+        "AzureBlobFileSystem.createFileSystem uri: {}", uri);
+    try {
+      abfsStore.createFilesystem();
+    } catch (AzureBlobFileSystemException ex) {
+      checkException(null, ex);
+    }
+  }
+
+  private URI ensureAuthority(URI uri, final Configuration conf) {
+
+    Preconditions.checkNotNull(uri, "uri");
+
+    if (uri.getAuthority() == null) {
+      final URI defaultUri = FileSystem.getDefaultUri(conf);
+
+      if (defaultUri != null && isAbfsScheme(defaultUri.getScheme())) {
+        try {
+          // Reconstruct the URI with the authority from the default URI.
+          uri = new URI(
+              uri.getScheme(),
+              defaultUri.getAuthority(),
+              uri.getPath(),
+              uri.getQuery(),
+              uri.getFragment());
+        } catch (URISyntaxException e) {
+          // This should never happen.
+          throw new IllegalArgumentException(new InvalidUriException(uri.toString()));
+        }
+      }
+    }
+
+    if (uri.getAuthority() == null) {
+      throw new IllegalArgumentException(new InvalidUriAuthorityException(uri.toString()));
+    }
+
+    return uri;
+  }
+
+  private boolean isAbfsScheme(final String scheme) {
+    if (scheme == null) {
+      return false;
+    }
+
+    if (scheme.equals(FileSystemUriSchemes.ABFS_SCHEME)
+        || scheme.equals(FileSystemUriSchemes.ABFS_SECURE_SCHEME)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  @VisibleForTesting
+  <T> FileSystemOperation<T> execute(
+      final String scopeDescription,
+      final Callable<T> callableFileOperation) throws IOException {
+    return execute(scopeDescription, callableFileOperation, null);
+  }
+
+  @VisibleForTesting
+  <T> FileSystemOperation<T> execute(
+      final String scopeDescription,
+      final Callable<T> callableFileOperation,
+      T defaultResultValue) throws IOException {
+
+    try {
+      final T executionResult = callableFileOperation.call();
+      return new FileSystemOperation<>(executionResult, null);
+    } catch (AbfsRestOperationException abfsRestOperationException) {
+      return new FileSystemOperation<>(defaultResultValue, abfsRestOperationException);
+    } catch (AzureBlobFileSystemException azureBlobFileSystemException) {
+      throw new IOException(azureBlobFileSystemException);
+    } catch (Exception exception) {
+      if (exception instanceof ExecutionException) {
+        exception = (Exception) getRootCause(exception);
+      }
+      final FileSystemOperationUnhandledException fileSystemOperationUnhandledException
+          = new FileSystemOperationUnhandledException(exception);
+      throw new IOException(fileSystemOperationUnhandledException);
+    }
+  }
+
+  /**
+   * Given a path and exception, choose which IOException subclass
+   * to create.
+   * Will return if and only iff the error code is in the list of allowed
+   * error codes.
+   * @param path path of operation triggering exception; may be null
+   * @param exception the exception caught
+   * @param allowedErrorCodesList varargs list of error codes.
+   * @throws IOException if the exception error code is not on the allowed list.
+   */
+  private void checkException(final Path path,
+                              final AzureBlobFileSystemException exception,
+                              final AzureServiceErrorCode... allowedErrorCodesList) throws IOException {
+    if (exception instanceof AbfsRestOperationException) {
+      AbfsRestOperationException ere = (AbfsRestOperationException) exception;
+
+      if (ArrayUtils.contains(allowedErrorCodesList, ere.getErrorCode())) {
+        return;
+      }
+      int statusCode = ere.getStatusCode();
+
+      //AbfsRestOperationException.getMessage() contains full error info including path/uri.
+      if (statusCode == HttpURLConnection.HTTP_NOT_FOUND) {
+        throw (IOException) new FileNotFoundException(ere.getMessage())
+            .initCause(exception);
+      } else if (statusCode == HttpURLConnection.HTTP_CONFLICT) {
+        throw (IOException) new FileAlreadyExistsException(ere.getMessage())
+            .initCause(exception);
+      } else {
+        throw ere;
+      }
+    } else {
+      if (path == null) {
+        throw exception;
+      }
+      // record info of path
+      throw new PathIOException(path.toString(), exception);
+    }
+  }
+
+  /**
+   * Gets the root cause of a provided {@link Throwable}.  If there is no cause for the
+   * {@link Throwable} provided into this function, the original {@link Throwable} is returned.
+   *
+   * @param throwable starting {@link Throwable}
+   * @return root cause {@link Throwable}
+   */
+  private Throwable getRootCause(Throwable throwable) {
+    if (throwable == null) {
+      throw new IllegalArgumentException("throwable can not be null");
+    }
+
+    Throwable result = throwable;
+    while (result.getCause() != null) {
+      result = result.getCause();
+    }
+
+    return result;
+  }
+
+  /**
+   * Get a delegation token from remote service endpoint if
+   * 'fs.azure.enable.kerberos.support' is set to 'true', and
+   * 'fs.azure.enable.delegation.token' is set to 'true'.
+   * @param renewer the account name that is allowed to renew the token.
+   * @return delegation token
+   * @throws IOException thrown when getting the current user.
+   */
+  @Override
+  public synchronized Token<?> getDelegationToken(final String renewer) throws IOException {
+    return this.delegationTokenEnabled ? this.delegationTokenManager.getDelegationToken(renewer)
+        : super.getDelegationToken(renewer);
+  }
+
+  @VisibleForTesting
+  FileSystem.Statistics getFsStatistics() {
+    return this.statistics;
+  }
+
+  @VisibleForTesting
+  static class FileSystemOperation<T> {
+    private final T result;
+    private final AbfsRestOperationException exception;
+
+    FileSystemOperation(final T result, final AbfsRestOperationException exception) {
+      this.result = result;
+      this.exception = exception;
+    }
+
+    public boolean failed() {
+      return this.exception != null;
+    }
+  }
+
+  @VisibleForTesting
+  AzureBlobFileSystemStore getAbfsStore() {
+    return abfsStore;
+  }
+
+  @VisibleForTesting
+  AbfsClient getAbfsClient() {
+    return abfsStore.getClient();
+  }
+
+  @VisibleForTesting
+  boolean getIsNamespaceEnabeld() throws AzureBlobFileSystemException {
+    return abfsStore.getIsNamespaceEnabled();
+  }
+}

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

@@ -0,0 +1,1028 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.azurebfs;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.MalformedURLException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.CharsetEncoder;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.FileSystemOperationUnhandledException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidFileSystemPropertyException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriAuthorityException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TimeoutException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultEntrySchema;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
+import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
+import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper;
+import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream;
+import org.apache.hadoop.fs.azurebfs.services.AbfsPermission;
+import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy;
+import org.apache.hadoop.fs.azurebfs.services.SharedKeyCredentials;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.http.client.utils.URIBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_ABFS_ENDPOINT;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME;
+import static org.apache.hadoop.util.Time.now;
+
+/**
+ * Provides the bridging logic between Hadoop's abstract filesystem and Azure Storage.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class AzureBlobFileSystemStore {
+  private static final Logger LOG = LoggerFactory.getLogger(AzureBlobFileSystemStore.class);
+
+  private AbfsClient client;
+  private URI uri;
+  private final UserGroupInformation userGroupInformation;
+  private static final String DATE_TIME_PATTERN = "E, dd MMM yyyy HH:mm:ss 'GMT'";
+  private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1";
+  private static final int LIST_MAX_RESULTS = 5000;
+  private static final int DELETE_DIRECTORY_TIMEOUT_MILISECONDS = 180000;
+  private static final int RENAME_TIMEOUT_MILISECONDS = 180000;
+
+  private final AbfsConfiguration abfsConfiguration;
+  private final Set<String> azureAtomicRenameDirSet;
+  private boolean isNamespaceEnabledSet;
+  private boolean isNamespaceEnabled;
+
+  public AzureBlobFileSystemStore(URI uri, boolean isSecure, Configuration configuration, UserGroupInformation userGroupInformation)
+          throws AzureBlobFileSystemException, IOException {
+    this.uri = uri;
+
+    String[] authorityParts = authorityParts(uri);
+    final String fileSystemName = authorityParts[0];
+    final String accountName = authorityParts[1];
+
+    try {
+      this.abfsConfiguration = new AbfsConfiguration(configuration, accountName);
+    } catch (IllegalAccessException exception) {
+      throw new FileSystemOperationUnhandledException(exception);
+    }
+
+    this.userGroupInformation = userGroupInformation;
+    this.azureAtomicRenameDirSet = new HashSet<>(Arrays.asList(
+        abfsConfiguration.getAzureAtomicRenameDirs().split(AbfsHttpConstants.COMMA)));
+
+    if (AuthType.OAuth == abfsConfiguration.getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SharedKey)
+            && !FileSystemUriSchemes.ABFS_SECURE_SCHEME.equals(uri.getScheme())) {
+      throw new IllegalArgumentException(
+              String.format("Incorrect URI %s, URI scheme must be abfss when authenticating using Oauth.", uri));
+    }
+
+    initializeClient(uri, fileSystemName, accountName, isSecure);
+  }
+
+  private String[] authorityParts(URI uri) throws InvalidUriAuthorityException, InvalidUriException {
+    final String authority = uri.getRawAuthority();
+    if (null == authority) {
+      throw new InvalidUriAuthorityException(uri.toString());
+    }
+
+    if (!authority.contains(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER)) {
+      throw new InvalidUriAuthorityException(uri.toString());
+    }
+
+    final String[] authorityParts = authority.split(AbfsHttpConstants.AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER, 2);
+
+    if (authorityParts.length < 2 || authorityParts[0] != null
+        && authorityParts[0].isEmpty()) {
+      final String errMsg = String
+              .format("'%s' has a malformed authority, expected container name. "
+                      + "Authority takes the form "
+                      + FileSystemUriSchemes.ABFS_SCHEME + "://[<container name>@]<account name>",
+                      uri.toString());
+      throw new InvalidUriException(errMsg);
+    }
+    return authorityParts;
+  }
+
+  public boolean getIsNamespaceEnabled() throws AzureBlobFileSystemException {
+    if (!isNamespaceEnabledSet) {
+      LOG.debug("getFilesystemProperties for filesystem: {}",
+          client.getFileSystem());
+
+      final AbfsRestOperation op = client.getFilesystemProperties();
+      isNamespaceEnabled = Boolean.parseBoolean(
+          op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_NAMESPACE_ENABLED));
+      isNamespaceEnabledSet = true;
+    }
+
+    return isNamespaceEnabled;
+  }
+
+  @VisibleForTesting
+  URIBuilder getURIBuilder(final String hostName, boolean isSecure) {
+    String scheme = isSecure ? FileSystemUriSchemes.HTTPS_SCHEME : FileSystemUriSchemes.HTTP_SCHEME;
+
+    final URIBuilder uriBuilder = new URIBuilder();
+    uriBuilder.setScheme(scheme);
+
+    // For testing purposes, an IP address and port may be provided to override
+    // the host specified in the FileSystem URI.  Also note that the format of
+    // the Azure Storage Service URI changes from
+    // http[s]://[account][domain-suffix]/[filesystem] to
+    // http[s]://[ip]:[port]/[account]/[filesystem].
+    String endPoint = abfsConfiguration.get(AZURE_ABFS_ENDPOINT);
+    if (endPoint == null || !endPoint.contains(AbfsHttpConstants.COLON)) {
+      uriBuilder.setHost(hostName);
+      return uriBuilder;
+    }
+
+    // Split ip and port
+    String[] data = endPoint.split(AbfsHttpConstants.COLON);
+    if (data.length != 2) {
+      throw new RuntimeException(String.format("ABFS endpoint is not set correctly : %s, "
+              + "Do not specify scheme when using {IP}:{PORT}", endPoint));
+    }
+    uriBuilder.setHost(data[0].trim());
+    uriBuilder.setPort(Integer.parseInt(data[1].trim()));
+    uriBuilder.setPath("/" + UriUtils.extractAccountNameFromHostName(hostName));
+
+    return uriBuilder;
+  }
+
+  public AbfsConfiguration getAbfsConfiguration() {
+    return this.abfsConfiguration;
+  }
+
+  public Hashtable<String, String> getFilesystemProperties() throws AzureBlobFileSystemException {
+    LOG.debug("getFilesystemProperties for filesystem: {}",
+            client.getFileSystem());
+
+    final Hashtable<String, String> parsedXmsProperties;
+
+    final AbfsRestOperation op = client.getFilesystemProperties();
+    final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES);
+
+    parsedXmsProperties = parseCommaSeparatedXmsProperties(xMsProperties);
+
+    return parsedXmsProperties;
+  }
+
+  public void setFilesystemProperties(final Hashtable<String, String> properties)
+      throws AzureBlobFileSystemException {
+    if (properties == null || properties.isEmpty()) {
+      return;
+    }
+
+    LOG.debug("setFilesystemProperties for filesystem: {} with properties: {}",
+            client.getFileSystem(),
+            properties);
+
+    final String commaSeparatedProperties;
+    try {
+      commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties);
+    } catch (CharacterCodingException ex) {
+      throw new InvalidAbfsRestOperationException(ex);
+    }
+
+    client.setFilesystemProperties(commaSeparatedProperties);
+  }
+
+  public Hashtable<String, String> getPathProperties(final Path path) throws AzureBlobFileSystemException {
+    LOG.debug("getPathProperties for filesystem: {} path: {}",
+            client.getFileSystem(),
+           path);
+
+    final Hashtable<String, String> parsedXmsProperties;
+    final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
+
+    final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES);
+
+    parsedXmsProperties = parseCommaSeparatedXmsProperties(xMsProperties);
+
+    return parsedXmsProperties;
+  }
+
+  public void setPathProperties(final Path path, final Hashtable<String, String> properties) throws AzureBlobFileSystemException {
+    LOG.debug("setFilesystemProperties for filesystem: {} path: {} with properties: {}",
+            client.getFileSystem(),
+            path,
+            properties);
+
+    final String commaSeparatedProperties;
+    try {
+      commaSeparatedProperties = convertXmsPropertiesToCommaSeparatedString(properties);
+    } catch (CharacterCodingException ex) {
+      throw new InvalidAbfsRestOperationException(ex);
+    }
+    client.setPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), commaSeparatedProperties);
+  }
+
+  public void createFilesystem() throws AzureBlobFileSystemException {
+    LOG.debug("createFilesystem for filesystem: {}",
+            client.getFileSystem());
+
+    client.createFilesystem();
+  }
+
+  public void deleteFilesystem() throws AzureBlobFileSystemException {
+    LOG.debug("deleteFilesystem for filesystem: {}",
+            client.getFileSystem());
+
+    client.deleteFilesystem();
+  }
+
+  public OutputStream createFile(final Path path, final boolean overwrite, final FsPermission permission,
+                                 final FsPermission umask) throws AzureBlobFileSystemException {
+    boolean isNamespaceEnabled = getIsNamespaceEnabled();
+    LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}",
+            client.getFileSystem(),
+            path,
+            overwrite,
+            permission.toString(),
+            umask.toString(),
+            isNamespaceEnabled);
+
+    client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), true, overwrite,
+        isNamespaceEnabled ? getOctalNotation(permission) : null,
+        isNamespaceEnabled ? getOctalNotation(umask) : null);
+
+    return new AbfsOutputStream(
+        client,
+        AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path),
+        0,
+        abfsConfiguration.getWriteBufferSize(),
+        abfsConfiguration.isFlushEnabled());
+  }
+
+  public void createDirectory(final Path path, final FsPermission permission, final FsPermission umask)
+      throws AzureBlobFileSystemException {
+    boolean isNamespaceEnabled = getIsNamespaceEnabled();
+    LOG.debug("createDirectory filesystem: {} path: {} permission: {} umask: {} isNamespaceEnabled: {}",
+            client.getFileSystem(),
+            path,
+            permission,
+            umask,
+            isNamespaceEnabled);
+
+    client.createPath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), false, true,
+        isNamespaceEnabled ? getOctalNotation(permission) : null,
+        isNamespaceEnabled ? getOctalNotation(umask) : null);
+  }
+
+  public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statistics statistics)
+      throws AzureBlobFileSystemException {
+    LOG.debug("openFileForRead filesystem: {} path: {}",
+            client.getFileSystem(),
+            path);
+
+    final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
+
+    final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
+    final long contentLength = Long.parseLong(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH));
+    final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
+
+    if (parseIsDirectory(resourceType)) {
+      throw new AbfsRestOperationException(
+              AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
+              AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
+              "openFileForRead must be used with files and not directories",
+              null);
+    }
+
+    // Add statistics for InputStream
+    return new AbfsInputStream(client, statistics,
+            AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), contentLength,
+                abfsConfiguration.getReadBufferSize(), abfsConfiguration.getReadAheadQueueDepth(), eTag);
+  }
+
+  public OutputStream openFileForWrite(final Path path, final boolean overwrite) throws
+          AzureBlobFileSystemException {
+    LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {}",
+            client.getFileSystem(),
+            path,
+            overwrite);
+
+    final AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
+
+    final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
+    final Long contentLength = Long.valueOf(op.getResult().getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH));
+
+    if (parseIsDirectory(resourceType)) {
+      throw new AbfsRestOperationException(
+              AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
+              AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
+              "openFileForRead must be used with files and not directories",
+              null);
+    }
+
+    final long offset = overwrite ? 0 : contentLength;
+
+    return new AbfsOutputStream(
+        client,
+        AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path),
+        offset,
+        abfsConfiguration.getWriteBufferSize(),
+        abfsConfiguration.isFlushEnabled());
+  }
+
+  public void rename(final Path source, final Path destination) throws
+          AzureBlobFileSystemException {
+
+    if (isAtomicRenameKey(source.getName())) {
+      LOG.warn("The atomic rename feature is not supported by the ABFS scheme; however rename,"
+              +" create and delete operations are atomic if Namespace is enabled for your Azure Storage account.");
+    }
+
+    LOG.debug("renameAsync filesystem: {} source: {} destination: {}",
+            client.getFileSystem(),
+            source,
+            destination);
+
+    String continuation = null;
+    long deadline = now() + RENAME_TIMEOUT_MILISECONDS;
+
+    do {
+      if (now() > deadline) {
+        LOG.debug("Rename {} to {} timed out.",
+                source,
+                destination);
+
+        throw new TimeoutException("Rename timed out.");
+      }
+
+      AbfsRestOperation op = client.renamePath(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(source),
+              AbfsHttpConstants.FORWARD_SLASH + getRelativePath(destination), continuation);
+      continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
+
+    } while (continuation != null && !continuation.isEmpty());
+  }
+
+  public void delete(final Path path, final boolean recursive)
+      throws AzureBlobFileSystemException {
+    LOG.debug("delete filesystem: {} path: {} recursive: {}",
+            client.getFileSystem(),
+            path,
+            String.valueOf(recursive));
+
+    String continuation = null;
+    long deadline = now() + DELETE_DIRECTORY_TIMEOUT_MILISECONDS;
+
+    do {
+      if (now() > deadline) {
+        LOG.debug("Delete directory {} timed out.", path);
+
+        throw new TimeoutException("Delete directory timed out.");
+      }
+
+      AbfsRestOperation op = client.deletePath(
+          AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path), recursive, continuation);
+      continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
+
+    } while (continuation != null && !continuation.isEmpty());
+  }
+
+  public FileStatus getFileStatus(final Path path) throws IOException {
+    boolean isNamespaceEnabled = getIsNamespaceEnabled();
+    LOG.debug("getFileStatus filesystem: {} path: {} isNamespaceEnabled: {}",
+            client.getFileSystem(),
+            path,
+            isNamespaceEnabled);
+
+    if (path.isRoot()) {
+      final AbfsRestOperation op = isNamespaceEnabled
+          ? client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + AbfsHttpConstants.ROOT_PATH)
+          : client.getFilesystemProperties();
+
+      final long blockSize = abfsConfiguration.getAzureBlockSize();
+      final String owner = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_OWNER);
+      final String group = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_GROUP);
+      final String permissions = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PERMISSIONS);
+      final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
+      final String lastModified = op.getResult().getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED);
+      final boolean hasAcl = AbfsPermission.isExtendedAcl(permissions);
+
+      return new VersionedFileStatus(
+              owner == null ? userGroupInformation.getUserName() : owner,
+              group == null ? userGroupInformation.getPrimaryGroupName() : group,
+              permissions == null ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL)
+                      : AbfsPermission.valueOf(permissions),
+              hasAcl,
+              0,
+              true,
+              1,
+              blockSize,
+              parseLastModifiedTime(lastModified),
+              path,
+              eTag);
+    } else {
+      AbfsRestOperation op = client.getPathProperties(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path));
+
+      final long blockSize = abfsConfiguration.getAzureBlockSize();
+      final AbfsHttpOperation result = op.getResult();
+      final String eTag = result.getResponseHeader(HttpHeaderConfigurations.ETAG);
+      final String lastModified = result.getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED);
+      final String contentLength = result.getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH);
+      final String resourceType = result.getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE);
+      final String owner = result.getResponseHeader(HttpHeaderConfigurations.X_MS_OWNER);
+      final String group = result.getResponseHeader(HttpHeaderConfigurations.X_MS_GROUP);
+      final String permissions = result.getResponseHeader((HttpHeaderConfigurations.X_MS_PERMISSIONS));
+      final boolean hasAcl = AbfsPermission.isExtendedAcl(permissions);
+
+      return new VersionedFileStatus(
+              owner == null ? userGroupInformation.getUserName() : owner,
+              group == null ? userGroupInformation.getPrimaryGroupName() : group,
+              permissions == null ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL)
+                      : AbfsPermission.valueOf(permissions),
+              hasAcl,
+              parseContentLength(contentLength),
+              parseIsDirectory(resourceType),
+              1,
+              blockSize,
+              parseLastModifiedTime(lastModified),
+              path,
+              eTag);
+    }
+  }
+
+  public FileStatus[] listStatus(final Path path) throws IOException {
+    LOG.debug("listStatus filesystem: {} path: {}",
+            client.getFileSystem(),
+           path);
+
+    String relativePath = path.isRoot() ? AbfsHttpConstants.EMPTY_STRING : getRelativePath(path);
+    String continuation = null;
+    ArrayList<FileStatus> fileStatuses = new ArrayList<>();
+
+    do {
+      AbfsRestOperation op = client.listPath(relativePath, false, LIST_MAX_RESULTS, continuation);
+      continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION);
+      ListResultSchema retrievedSchema = op.getResult().getListResultSchema();
+      if (retrievedSchema == null) {
+        throw new AbfsRestOperationException(
+                AzureServiceErrorCode.PATH_NOT_FOUND.getStatusCode(),
+                AzureServiceErrorCode.PATH_NOT_FOUND.getErrorCode(),
+                "listStatusAsync path not found",
+                null, op.getResult());
+      }
+
+      long blockSize = abfsConfiguration.getAzureBlockSize();
+
+      for (ListResultEntrySchema entry : retrievedSchema.paths()) {
+        final String owner = entry.owner() == null ? userGroupInformation.getUserName() : entry.owner();
+        final String group = entry.group() == null ? userGroupInformation.getPrimaryGroupName() : entry.group();
+        final FsPermission fsPermission = entry.permissions() == null
+                ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL)
+                : AbfsPermission.valueOf(entry.permissions());
+        final boolean hasAcl = AbfsPermission.isExtendedAcl(entry.permissions());
+
+        long lastModifiedMillis = 0;
+        long contentLength = entry.contentLength() == null ? 0 : entry.contentLength();
+        boolean isDirectory = entry.isDirectory() == null ? false : entry.isDirectory();
+        if (entry.lastModified() != null && !entry.lastModified().isEmpty()) {
+          lastModifiedMillis = parseLastModifiedTime(entry.lastModified());
+        }
+
+        Path entryPath = new Path(File.separator + entry.name());
+        entryPath = entryPath.makeQualified(this.uri, entryPath);
+
+        fileStatuses.add(
+                new VersionedFileStatus(
+                        owner,
+                        group,
+                        fsPermission,
+                        hasAcl,
+                        contentLength,
+                        isDirectory,
+                        1,
+                        blockSize,
+                        lastModifiedMillis,
+                        entryPath,
+                        entry.eTag()));
+      }
+
+    } while (continuation != null && !continuation.isEmpty());
+
+    return fileStatuses.toArray(new FileStatus[0]);
+  }
+
+  public void setOwner(final Path path, final String owner, final String group) throws
+          AzureBlobFileSystemException {
+    if (!getIsNamespaceEnabled()) {
+      throw new UnsupportedOperationException(
+          "This operation is only valid for storage accounts with the hierarchical namespace enabled.");
+    }
+
+    LOG.debug(
+            "setOwner filesystem: {} path: {} owner: {} group: {}",
+            client.getFileSystem(),
+            path.toString(),
+            owner,
+            group);
+    client.setOwner(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true), owner, group);
+  }
+
+  public void setPermission(final Path path, final FsPermission permission) throws
+          AzureBlobFileSystemException {
+    if (!getIsNamespaceEnabled()) {
+      throw new UnsupportedOperationException(
+          "This operation is only valid for storage accounts with the hierarchical namespace enabled.");
+    }
+
+    LOG.debug(
+            "setPermission filesystem: {} path: {} permission: {}",
+            client.getFileSystem(),
+            path.toString(),
+            permission.toString());
+    client.setPermission(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true),
+            String.format(AbfsHttpConstants.PERMISSION_FORMAT, permission.toOctal()));
+  }
+
+  public void modifyAclEntries(final Path path, final List<AclEntry> aclSpec) throws
+          AzureBlobFileSystemException {
+    if (!getIsNamespaceEnabled()) {
+      throw new UnsupportedOperationException(
+          "This operation is only valid for storage accounts with the hierarchical namespace enabled.");
+    }
+
+    LOG.debug(
+            "modifyAclEntries filesystem: {} path: {} aclSpec: {}",
+            client.getFileSystem(),
+            path.toString(),
+            AclEntry.aclSpecToString(aclSpec));
+
+    final Map<String, String> modifyAclEntries = AbfsAclHelper.deserializeAclSpec(AclEntry.aclSpecToString(aclSpec));
+
+    final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true));
+    final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
+
+    final Map<String, String> aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL));
+
+    for (Map.Entry<String, String> modifyAclEntry : modifyAclEntries.entrySet()) {
+      aclEntries.put(modifyAclEntry.getKey(), modifyAclEntry.getValue());
+    }
+
+    if (!modifyAclEntries.containsKey(AbfsHttpConstants.ACCESS_MASK)) {
+      aclEntries.remove(AbfsHttpConstants.ACCESS_MASK);
+    }
+
+    if (!modifyAclEntries.containsKey(AbfsHttpConstants.DEFAULT_MASK)) {
+      aclEntries.remove(AbfsHttpConstants.DEFAULT_MASK);
+    }
+
+    client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true),
+        AbfsAclHelper.serializeAclSpec(aclEntries), eTag);
+  }
+
+  public void removeAclEntries(final Path path, final List<AclEntry> aclSpec) throws AzureBlobFileSystemException {
+    if (!getIsNamespaceEnabled()) {
+      throw new UnsupportedOperationException(
+          "This operation is only valid for storage accounts with the hierarchical namespace enabled.");
+    }
+
+    LOG.debug(
+            "removeAclEntries filesystem: {} path: {} aclSpec: {}",
+            client.getFileSystem(),
+            path.toString(),
+            AclEntry.aclSpecToString(aclSpec));
+
+    final Map<String, String> removeAclEntries = AbfsAclHelper.deserializeAclSpec(AclEntry.aclSpecToString(aclSpec));
+    final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true));
+    final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
+
+    final Map<String, String> aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL));
+
+    AbfsAclHelper.removeAclEntriesInternal(aclEntries, removeAclEntries);
+
+    client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true),
+            AbfsAclHelper.serializeAclSpec(aclEntries), eTag);
+  }
+
+  public void removeDefaultAcl(final Path path) throws AzureBlobFileSystemException {
+    if (!getIsNamespaceEnabled()) {
+      throw new UnsupportedOperationException(
+          "This operation is only valid for storage accounts with the hierarchical namespace enabled.");
+    }
+
+    LOG.debug(
+            "removeDefaultAcl filesystem: {} path: {}",
+            client.getFileSystem(),
+            path.toString());
+
+    final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true));
+    final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
+    final Map<String, String> aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL));
+    final Map<String, String> defaultAclEntries = new HashMap<>();
+
+    for (Map.Entry<String, String> aclEntry : aclEntries.entrySet()) {
+      if (aclEntry.getKey().startsWith("default:")) {
+        defaultAclEntries.put(aclEntry.getKey(), aclEntry.getValue());
+      }
+    }
+
+    for (Map.Entry<String, String> defaultAclEntry : defaultAclEntries.entrySet()) {
+      aclEntries.remove(defaultAclEntry.getKey());
+    }
+
+    client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true),
+        AbfsAclHelper.serializeAclSpec(aclEntries), eTag);
+  }
+
+  public void removeAcl(final Path path) throws AzureBlobFileSystemException {
+    if (!getIsNamespaceEnabled()) {
+      throw new UnsupportedOperationException(
+          "This operation is only valid for storage accounts with the hierarchical namespace enabled.");
+    }
+
+    LOG.debug(
+            "removeAcl filesystem: {} path: {}",
+            client.getFileSystem(),
+            path.toString());
+    final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true));
+    final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
+
+    final Map<String, String> aclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL));
+    final Map<String, String> newAclEntries = new HashMap<>();
+
+    newAclEntries.put(AbfsHttpConstants.ACCESS_USER, aclEntries.get(AbfsHttpConstants.ACCESS_USER));
+    newAclEntries.put(AbfsHttpConstants.ACCESS_GROUP, aclEntries.get(AbfsHttpConstants.ACCESS_GROUP));
+    newAclEntries.put(AbfsHttpConstants.ACCESS_OTHER, aclEntries.get(AbfsHttpConstants.ACCESS_OTHER));
+
+    client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true),
+        AbfsAclHelper.serializeAclSpec(newAclEntries), eTag);
+  }
+
+  public void setAcl(final Path path, final List<AclEntry> aclSpec) throws AzureBlobFileSystemException {
+    if (!getIsNamespaceEnabled()) {
+      throw new UnsupportedOperationException(
+          "This operation is only valid for storage accounts with the hierarchical namespace enabled.");
+    }
+
+    LOG.debug(
+            "setAcl filesystem: {} path: {} aclspec: {}",
+            client.getFileSystem(),
+            path.toString(),
+            AclEntry.aclSpecToString(aclSpec));
+    final Map<String, String> aclEntries = AbfsAclHelper.deserializeAclSpec(AclEntry.aclSpecToString(aclSpec));
+    final AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true));
+    final String eTag = op.getResult().getResponseHeader(HttpHeaderConfigurations.ETAG);
+
+    final Map<String, String> getAclEntries = AbfsAclHelper.deserializeAclSpec(op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL));
+    for (Map.Entry<String, String> ace : getAclEntries.entrySet()) {
+      if (ace.getKey().startsWith("default:") && (ace.getKey() != AbfsHttpConstants.DEFAULT_MASK)
+              && !aclEntries.containsKey(ace.getKey())) {
+        aclEntries.put(ace.getKey(), ace.getValue());
+      }
+    }
+
+    client.setAcl(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true),
+        AbfsAclHelper.serializeAclSpec(aclEntries), eTag);
+  }
+
+  public AclStatus getAclStatus(final Path path) throws IOException {
+    if (!getIsNamespaceEnabled()) {
+      throw new UnsupportedOperationException(
+          "This operation is only valid for storage accounts with the hierarchical namespace enabled.");
+    }
+
+    LOG.debug(
+            "getAclStatus filesystem: {} path: {}",
+            client.getFileSystem(),
+            path.toString());
+    AbfsRestOperation op = client.getAclStatus(AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path, true));
+    AbfsHttpOperation result = op.getResult();
+
+    final String owner = result.getResponseHeader(HttpHeaderConfigurations.X_MS_OWNER);
+    final String group = result.getResponseHeader(HttpHeaderConfigurations.X_MS_GROUP);
+    final String permissions = result.getResponseHeader(HttpHeaderConfigurations.X_MS_PERMISSIONS);
+    final String aclSpecString = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ACL);
+
+    final List<AclEntry> processedAclEntries = AclEntry.parseAclSpec(AbfsAclHelper.processAclString(aclSpecString), true);
+    final FsPermission fsPermission = permissions == null ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL)
+            : AbfsPermission.valueOf(permissions);
+
+    final AclStatus.Builder aclStatusBuilder = new AclStatus.Builder();
+    aclStatusBuilder.owner(owner == null ? userGroupInformation.getUserName() : owner);
+    aclStatusBuilder.group(group == null ? userGroupInformation.getPrimaryGroupName() : group);
+
+    aclStatusBuilder.setPermission(fsPermission);
+    aclStatusBuilder.stickyBit(fsPermission.getStickyBit());
+    aclStatusBuilder.addEntries(processedAclEntries);
+    return aclStatusBuilder.build();
+  }
+
+  public boolean isAtomicRenameKey(String key) {
+    return isKeyForDirectorySet(key, azureAtomicRenameDirSet);
+  }
+
+  private void initializeClient(URI uri, String fileSystemName, String accountName, boolean isSecure) throws AzureBlobFileSystemException {
+    if (this.client != null) {
+      return;
+    }
+
+    final URIBuilder uriBuilder = getURIBuilder(accountName, isSecure);
+
+    final String url = uriBuilder.toString() + AbfsHttpConstants.FORWARD_SLASH + fileSystemName;
+
+    URL baseUrl;
+    try {
+      baseUrl = new URL(url);
+    } catch (MalformedURLException e) {
+      throw new InvalidUriException(uri.toString());
+    }
+
+    SharedKeyCredentials creds = null;
+    AccessTokenProvider tokenProvider = null;
+
+    if (abfsConfiguration.getAuthType(accountName) == AuthType.SharedKey) {
+      int dotIndex = accountName.indexOf(AbfsHttpConstants.DOT);
+      if (dotIndex <= 0) {
+        throw new InvalidUriException(
+                uri.toString() + " - account name is not fully qualified.");
+      }
+      creds = new SharedKeyCredentials(accountName.substring(0, dotIndex),
+            abfsConfiguration.getStorageAccountKey());
+    } else {
+      tokenProvider = abfsConfiguration.getTokenProvider();
+    }
+
+    this.client =  new AbfsClient(baseUrl, creds, abfsConfiguration, new ExponentialRetryPolicy(), tokenProvider);
+  }
+
+  private String getOctalNotation(FsPermission fsPermission) {
+    Preconditions.checkNotNull(fsPermission, "fsPermission");
+    return String.format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal());
+  }
+
+  private String getRelativePath(final Path path) {
+    return getRelativePath(path, false);
+  }
+
+  private String getRelativePath(final Path path, final boolean allowRootPath) {
+    Preconditions.checkNotNull(path, "path");
+    final String relativePath = path.toUri().getPath();
+
+    if (relativePath.length() == 0 || (relativePath.length() == 1 && relativePath.charAt(0) == Path.SEPARATOR_CHAR)) {
+      return allowRootPath ? AbfsHttpConstants.ROOT_PATH : AbfsHttpConstants.EMPTY_STRING;
+    }
+
+    if (relativePath.charAt(0) == Path.SEPARATOR_CHAR) {
+      return relativePath.substring(1);
+    }
+
+    return relativePath;
+  }
+
+  private long parseContentLength(final String contentLength) {
+    if (contentLength == null) {
+      return -1;
+    }
+
+    return Long.parseLong(contentLength);
+  }
+
+  private boolean parseIsDirectory(final String resourceType) {
+    return resourceType != null
+        && resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY);
+  }
+
+  private long parseLastModifiedTime(final String lastModifiedTime) {
+    long parsedTime = 0;
+    try {
+      Date utcDate = new SimpleDateFormat(DATE_TIME_PATTERN).parse(lastModifiedTime);
+      parsedTime = utcDate.getTime();
+    } catch (ParseException e) {
+      LOG.error("Failed to parse the date {}", lastModifiedTime);
+    } finally {
+      return parsedTime;
+    }
+  }
+
+  private String convertXmsPropertiesToCommaSeparatedString(final Hashtable<String, String> properties) throws
+          CharacterCodingException {
+    StringBuilder commaSeparatedProperties = new StringBuilder();
+
+    final CharsetEncoder encoder = Charset.forName(XMS_PROPERTIES_ENCODING).newEncoder();
+
+    for (Map.Entry<String, String> propertyEntry : properties.entrySet()) {
+      String key = propertyEntry.getKey();
+      String value = propertyEntry.getValue();
+
+      Boolean canEncodeValue = encoder.canEncode(value);
+      if (!canEncodeValue) {
+        throw new CharacterCodingException();
+      }
+
+      String encodedPropertyValue = Base64.encode(encoder.encode(CharBuffer.wrap(value)).array());
+      commaSeparatedProperties.append(key)
+              .append(AbfsHttpConstants.EQUAL)
+              .append(encodedPropertyValue);
+
+      commaSeparatedProperties.append(AbfsHttpConstants.COMMA);
+    }
+
+    if (commaSeparatedProperties.length() != 0) {
+      commaSeparatedProperties.deleteCharAt(commaSeparatedProperties.length() - 1);
+    }
+
+    return commaSeparatedProperties.toString();
+  }
+
+  private Hashtable<String, String> parseCommaSeparatedXmsProperties(String xMsProperties) throws
+          InvalidFileSystemPropertyException, InvalidAbfsRestOperationException {
+    Hashtable<String, String> properties = new Hashtable<>();
+
+    final CharsetDecoder decoder = Charset.forName(XMS_PROPERTIES_ENCODING).newDecoder();
+
+    if (xMsProperties != null && !xMsProperties.isEmpty()) {
+      String[] userProperties = xMsProperties.split(AbfsHttpConstants.COMMA);
+
+      if (userProperties.length == 0) {
+        return properties;
+      }
+
+      for (String property : userProperties) {
+        if (property.isEmpty()) {
+          throw new InvalidFileSystemPropertyException(xMsProperties);
+        }
+
+        String[] nameValue = property.split(AbfsHttpConstants.EQUAL, 2);
+        if (nameValue.length != 2) {
+          throw new InvalidFileSystemPropertyException(xMsProperties);
+        }
+
+        byte[] decodedValue = Base64.decode(nameValue[1]);
+
+        final String value;
+        try {
+          value = decoder.decode(ByteBuffer.wrap(decodedValue)).toString();
+        } catch (CharacterCodingException ex) {
+          throw new InvalidAbfsRestOperationException(ex);
+        }
+        properties.put(nameValue[0], value);
+      }
+    }
+
+    return properties;
+  }
+
+  private boolean isKeyForDirectorySet(String key, Set<String> dirSet) {
+    for (String dir : dirSet) {
+      if (dir.isEmpty() || key.startsWith(dir + AbfsHttpConstants.FORWARD_SLASH)) {
+        return true;
+      }
+
+      try {
+        URI uri = new URI(dir);
+        if (null == uri.getAuthority()) {
+          if (key.startsWith(dir + "/")){
+            return true;
+          }
+        }
+      } catch (URISyntaxException e) {
+        LOG.info("URI syntax error creating URI for {}", dir);
+      }
+    }
+
+    return false;
+  }
+
+  private static class VersionedFileStatus extends FileStatus {
+    private final String version;
+
+    VersionedFileStatus(
+            final String owner, final String group, final FsPermission fsPermission, final boolean hasAcl,
+            final long length, final boolean isdir, final int blockReplication,
+            final long blocksize, final long modificationTime, final Path path,
+            String version) {
+      super(length, isdir, blockReplication, blocksize, modificationTime, 0,
+              fsPermission,
+              owner,
+              group,
+              null,
+              path,
+              hasAcl, false, false);
+
+      this.version = version;
+    }
+
+    /** Compare if this object is equal to another object.
+     * @param   obj the object to be compared.
+     * @return  true if two file status has the same path name; false if not.
+     */
+    @Override
+    public boolean equals(Object obj) {
+      if (!(obj instanceof FileStatus)) {
+        return false;
+      }
+
+      FileStatus other = (FileStatus) obj;
+
+      if (!other.equals(this)) {// compare the path
+        return false;
+      }
+
+      if (other instanceof VersionedFileStatus) {
+        return this.version.equals(((VersionedFileStatus) other).version);
+      }
+
+      return true;
+    }
+
+    /**
+     * Returns a hash code value for the object, which is defined as
+     * the hash code of the path name.
+     *
+     * @return  a hash code value for the path name and version
+     */
+    @Override
+    public int hashCode() {
+      int hash = getPath().hashCode();
+      hash = 89 * hash + (this.version != null ? this.version.hashCode() : 0);
+      return hash;
+    }
+
+    /**
+     * Returns the version of this FileStatus
+     *
+     * @return  a string value for the FileStatus version
+     */
+    public String getVersion() {
+      return this.version;
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder(
+          "VersionedFileStatus{");
+      sb.append(super.toString());
+      sb.append("; version='").append(version).append('\'');
+      sb.append('}');
+      return sb.toString();
+    }
+  }
+
+  @VisibleForTesting
+  AbfsClient getClient() {
+    return this.client;
+  }
+}

+ 39 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/SecureAzureBlobFileSystem.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.azurebfs;
+
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+
+/**
+ * A secure {@link org.apache.hadoop.fs.FileSystem} for reading and writing files stored on <a
+ * href="http://store.azure.com/">Windows Azure</a>
+ */
+@InterfaceStability.Evolving
+public class SecureAzureBlobFileSystem extends AzureBlobFileSystem {
+  @Override
+  public boolean isSecure() {
+    return true;
+  }
+
+  @Override
+  public String getScheme() {
+    return FileSystemUriSchemes.ABFS_SECURE_SCHEME;
+  }
+}

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

@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.constants;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Responsible to keep all constant keys used in abfs rest client here.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class AbfsHttpConstants {
+  // Abfs Http client constants
+  public static final String FILESYSTEM = "filesystem";
+  public static final String FILE = "file";
+  public static final String DIRECTORY = "directory";
+  public static final String APPEND_ACTION = "append";
+  public static final String FLUSH_ACTION = "flush";
+  public static final String SET_PROPERTIES_ACTION = "setProperties";
+  public static final String SET_ACCESS_CONTROL = "setAccessControl";
+  public static final String GET_ACCESS_CONTROL = "getAccessControl";
+  public static final String DEFAULT_TIMEOUT = "90";
+
+  public static final String JAVA_VERSION = "java.version";
+  public static final String OS_NAME = "os.name";
+  public static final String OS_VERSION = "os.version";
+
+  public static final String CLIENT_VERSION = "Azure Blob FS/1.0";
+
+  // Abfs Http Verb
+  public static final String HTTP_METHOD_DELETE = "DELETE";
+  public static final String HTTP_METHOD_GET = "GET";
+  public static final String HTTP_METHOD_HEAD = "HEAD";
+  public static final String HTTP_METHOD_PATCH = "PATCH";
+  public static final String HTTP_METHOD_POST = "POST";
+  public static final String HTTP_METHOD_PUT = "PUT";
+
+  // Abfs generic constants
+  public static final String SINGLE_WHITE_SPACE = " ";
+  public static final String EMPTY_STRING = "";
+  public static final String FORWARD_SLASH = "/";
+  public static final String DOT = ".";
+  public static final String PLUS = "+";
+  public static final String STAR = "*";
+  public static final String COMMA = ",";
+  public static final String COLON = ":";
+  public static final String EQUAL = "=";
+  public static final String QUESTION_MARK = "?";
+  public static final String AND_MARK = "&";
+  public static final String SEMICOLON = ";";
+  public static final String HTTP_HEADER_PREFIX = "x-ms-";
+
+  public static final String PLUS_ENCODE = "%20";
+  public static final String FORWARD_SLASH_ENCODE = "%2F";
+  public static final String AZURE_DISTRIBUTED_FILE_SYSTEM_AUTHORITY_DELIMITER = "@";
+  public static final String UTF_8 = "utf-8";
+  public static final String GMT_TIMEZONE = "GMT";
+  public static final String APPLICATION_JSON = "application/json";
+  public static final String APPLICATION_OCTET_STREAM = "application/octet-stream";
+
+  public static final String ROOT_PATH = "/";
+  public static final String ACCESS_MASK = "mask:";
+  public static final String ACCESS_USER = "user:";
+  public static final String ACCESS_GROUP = "group:";
+  public static final String ACCESS_OTHER = "other:";
+  public static final String DEFAULT_MASK = "default:mask:";
+  public static final String DEFAULT_USER = "default:user:";
+  public static final String DEFAULT_GROUP = "default:group:";
+  public static final String DEFAULT_OTHER = "default:other:";
+  public static final String DEFAULT_SCOPE = "default:";
+  public static final String PERMISSION_FORMAT = "%04d";
+
+  private AbfsHttpConstants() {}
+}

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

@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.constants;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Responsible to keep all the Azure Blob File System configurations keys in Hadoop configuration file.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ConfigurationKeys {
+  public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME = "fs.azure.account.key";
+  public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME_REGX = "fs\\.azure\\.account\\.key\\.(.*)";
+  public static final String FS_AZURE_SECURE_MODE = "fs.azure.secure.mode";
+
+  // Retry strategy defined by the user
+  public static final String AZURE_MIN_BACKOFF_INTERVAL = "fs.azure.io.retry.min.backoff.interval";
+  public static final String AZURE_MAX_BACKOFF_INTERVAL = "fs.azure.io.retry.max.backoff.interval";
+  public static final String AZURE_BACKOFF_INTERVAL = "fs.azure.io.retry.backoff.interval";
+  public static final String AZURE_MAX_IO_RETRIES = "fs.azure.io.retry.max.retries";
+
+  // Read and write buffer sizes defined by the user
+  public static final String AZURE_WRITE_BUFFER_SIZE = "fs.azure.write.request.size";
+  public static final String AZURE_READ_BUFFER_SIZE = "fs.azure.read.request.size";
+  public static final String AZURE_BLOCK_SIZE_PROPERTY_NAME = "fs.azure.block.size";
+  public static final String AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME = "fs.azure.block.location.impersonatedhost";
+  public static final String AZURE_CONCURRENT_CONNECTION_VALUE_OUT = "fs.azure.concurrentRequestCount.out";
+  public static final String AZURE_CONCURRENT_CONNECTION_VALUE_IN = "fs.azure.concurrentRequestCount.in";
+  public static final String AZURE_TOLERATE_CONCURRENT_APPEND = "fs.azure.io.read.tolerate.concurrent.append";
+  public static final String AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION = "fs.azure.createRemoteFileSystemDuringInitialization";
+  public static final String AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION = "fs.azure.skipUserGroupMetadataDuringInitialization";
+  public static final String FS_AZURE_ENABLE_AUTOTHROTTLING = "fs.azure.enable.autothrottling";
+  public static final String FS_AZURE_ATOMIC_RENAME_KEY = "fs.azure.atomic.rename.key";
+  public static final String FS_AZURE_READ_AHEAD_QUEUE_DEPTH = "fs.azure.readaheadqueue.depth";
+  public static final String FS_AZURE_ENABLE_FLUSH = "fs.azure.enable.flush";
+  public static final String FS_AZURE_USER_AGENT_PREFIX_KEY = "fs.azure.user.agent.prefix";
+  public static final String FS_AZURE_SSL_CHANNEL_MODE_KEY = "fs.azure.ssl.channel.mode";
+
+  public static final String AZURE_KEY_ACCOUNT_KEYPROVIDER = "fs.azure.account.keyprovider";
+  public static final String AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT = "fs.azure.shellkeyprovider.script";
+
+  /** End point of ABFS account: {@value}. */
+  public static final String AZURE_ABFS_ENDPOINT = "fs.azure.abfs.endpoint";
+  /** Key for auth type properties: {@value}. */
+  public static final String FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME = "fs.azure.account.auth.type";
+  /** Key for oauth token provider type: {@value}. */
+  public static final String FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME = "fs.azure.account.oauth.provider.type";
+  /** Key for oauth AAD client id: {@value}. */
+  public static final String FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID = "fs.azure.account.oauth2.client.id";
+  /** Key for oauth AAD client secret: {@value}. */
+  public static final String FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET = "fs.azure.account.oauth2.client.secret";
+  /** Key for oauth AAD client endpoint: {@value}. */
+  public static final String FS_AZURE_ACCOUNT_OAUTH_CLIENT_ENDPOINT = "fs.azure.account.oauth2.client.endpoint";
+  /** Key for oauth msi tenant id: {@value}. */
+  public static final String FS_AZURE_ACCOUNT_OAUTH_MSI_TENANT = "fs.azure.account.oauth2.msi.tenant";
+  /** Key for oauth user name: {@value}. */
+  public static final String FS_AZURE_ACCOUNT_OAUTH_USER_NAME = "fs.azure.account.oauth2.user.name";
+  /** Key for oauth user password: {@value}. */
+  public static final String FS_AZURE_ACCOUNT_OAUTH_USER_PASSWORD = "fs.azure.account.oauth2.user.password";
+  /** Key for oauth refresh token: {@value}. */
+  public static final String FS_AZURE_ACCOUNT_OAUTH_REFRESH_TOKEN = "fs.azure.account.oauth2.refresh.token";
+
+  public static String accountProperty(String property, String account) {
+    return property + "." + account;
+  }
+
+  public static final String FS_AZURE_ENABLE_DELEGATION_TOKEN = "fs.azure.enable.delegation.token";
+  public static final String FS_AZURE_DELEGATION_TOKEN_PROVIDER_TYPE = "fs.azure.delegation.token.provider.type";
+
+  private ConfigurationKeys() {}
+}

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

@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.constants;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
+
+/**
+ * Responsible to keep all the Azure Blob File System related configurations.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class FileSystemConfigurations {
+  public static final String USER_HOME_DIRECTORY_PREFIX = "/user";
+
+  // Retry parameter defaults.
+  public static final int DEFAULT_MIN_BACKOFF_INTERVAL = 3 * 1000;  // 3s
+  public static final int DEFAULT_MAX_BACKOFF_INTERVAL = 30 * 1000;  // 30s
+  public static final int DEFAULT_BACKOFF_INTERVAL = 3 * 1000;  // 3s
+  public static final int DEFAULT_MAX_RETRY_ATTEMPTS = 30;
+
+  private static final int ONE_KB = 1024;
+  private static final int ONE_MB = ONE_KB * ONE_KB;
+
+  // Default upload and download buffer size
+  public static final int DEFAULT_WRITE_BUFFER_SIZE = 8 * ONE_MB;  // 8 MB
+  public static final int DEFAULT_READ_BUFFER_SIZE = 4 * ONE_MB;  // 4 MB
+  public static final int MIN_BUFFER_SIZE = 16 * ONE_KB;  // 16 KB
+  public static final int MAX_BUFFER_SIZE = 100 * ONE_MB;  // 100 MB
+  public static final long MAX_AZURE_BLOCK_SIZE = 512 * 1024 * 1024L;
+  public static final String AZURE_BLOCK_LOCATION_HOST_DEFAULT = "localhost";
+
+  public static final int MAX_CONCURRENT_READ_THREADS = 12;
+  public static final int MAX_CONCURRENT_WRITE_THREADS = 8;
+  public static final boolean DEFAULT_READ_TOLERATE_CONCURRENT_APPEND = false;
+  public static final boolean DEFAULT_AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION = false;
+  public static final boolean DEFAULT_AZURE_SKIP_USER_GROUP_METADATA_DURING_INITIALIZATION = false;
+
+  public static final String DEFAULT_FS_AZURE_ATOMIC_RENAME_DIRECTORIES = "/hbase";
+
+  public static final int DEFAULT_READ_AHEAD_QUEUE_DEPTH = -1;
+  public static final boolean DEFAULT_ENABLE_FLUSH = true;
+  public static final boolean DEFAULT_ENABLE_AUTOTHROTTLING = true;
+
+  public static final SSLSocketFactoryEx.SSLChannelMode DEFAULT_FS_AZURE_SSL_CHANNEL_MODE
+      = SSLSocketFactoryEx.SSLChannelMode.Default;
+
+  public static final boolean DEFAULT_ENABLE_DELEGATION_TOKEN = false;
+  private FileSystemConfigurations() {}
+}

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

@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.constants;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Responsible to keep all Azure Blob File System valid URI schemes.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class FileSystemUriSchemes {
+  public static final String ABFS_SCHEME = "abfs";
+  public static final String ABFS_SECURE_SCHEME = "abfss";
+  public static final String ABFS_DNS_PREFIX = "dfs";
+
+  public static final String HTTP_SCHEME = "http";
+  public static final String HTTPS_SCHEME = "https";
+
+  public static final String WASB_SCHEME = "wasb";
+  public static final String WASB_SECURE_SCHEME = "wasbs";
+  public static final String WASB_DNS_PREFIX = "blob";
+
+  private FileSystemUriSchemes() {}
+}

+ 63 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.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.constants;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Responsible to keep all abfs http headers here.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class HttpHeaderConfigurations {
+  public static final String ACCEPT = "Accept";
+  public static final String ACCEPT_CHARSET = "Accept-Charset";
+  public static final String AUTHORIZATION = "Authorization";
+  public static final String IF_MODIFIED_SINCE = "If-Modified-Since";
+  public static final String IF_UNMODIFIED_SINCE = "If-Unmodified-Since";
+  public static final String IF_MATCH = "If-Match";
+  public static final String IF_NONE_MATCH = "If-None-Match";
+  public static final String CONTENT_LENGTH = "Content-Length";
+  public static final String CONTENT_ENCODING = "Content-Encoding";
+  public static final String CONTENT_LANGUAGE = "Content-Language";
+  public static final String CONTENT_MD5 = "Content-MD5";
+  public static final String CONTENT_TYPE = "Content-Type";
+  public static final String RANGE = "Range";
+  public static final String TRANSFER_ENCODING = "Transfer-Encoding";
+  public static final String USER_AGENT = "User-Agent";
+  public static final String X_HTTP_METHOD_OVERRIDE = "X-HTTP-Method-Override";
+  public static final String X_MS_CLIENT_REQUEST_ID = "x-ms-client-request-id";
+  public static final String X_MS_DATE = "x-ms-date";
+  public static final String X_MS_REQUEST_ID = "x-ms-request-id";
+  public static final String X_MS_VERSION = "x-ms-version";
+  public static final String X_MS_RESOURCE_TYPE = "x-ms-resource-type";
+  public static final String X_MS_CONTINUATION = "x-ms-continuation";
+  public static final String ETAG = "ETag";
+  public static final String X_MS_PROPERTIES = "x-ms-properties";
+  public static final String X_MS_RENAME_SOURCE = "x-ms-rename-source";
+  public static final String LAST_MODIFIED = "Last-Modified";
+  public static final String X_MS_OWNER = "x-ms-owner";
+  public static final String X_MS_GROUP = "x-ms-group";
+  public static final String X_MS_ACL = "x-ms-acl";
+  public static final String X_MS_PERMISSIONS = "x-ms-permissions";
+  public static final String X_MS_UMASK = "x-ms-umask";
+  public static final String X_MS_NAMESPACE_ENABLED = "x-ms-namespace-enabled";
+
+  private HttpHeaderConfigurations() {}
+}

+ 40 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpQueryParams.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.constants;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Responsible to keep all Http Query params here.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class HttpQueryParams {
+  public static final String QUERY_PARAM_RESOURCE = "resource";
+  public static final String QUERY_PARAM_DIRECTORY = "directory";
+  public static final String QUERY_PARAM_CONTINUATION = "continuation";
+  public static final String QUERY_PARAM_RECURSIVE = "recursive";
+  public static final String QUERY_PARAM_MAXRESULTS = "maxResults";
+  public static final String QUERY_PARAM_ACTION = "action";
+  public static final String QUERY_PARAM_POSITION = "position";
+  public static final String QUERY_PARAM_TIMEOUT = "timeout";
+  public static final String QUERY_PARAM_RETAIN_UNCOMMITTED_DATA = "retainUncommittedData";
+
+  private HttpQueryParams() {}
+}

+ 22 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/package-info.java

@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.constants;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 104 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/ConfigurationValidationAnnotations.java

@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.contracts.annotations;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Definitions of Annotations for all types of the validators.
+ */
+@InterfaceStability.Evolving
+public class ConfigurationValidationAnnotations {
+  /**
+   * Describes the requirements when validating the annotated int field.
+   */
+  @Target({ ElementType.FIELD })
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface IntegerConfigurationValidatorAnnotation {
+    String ConfigurationKey();
+
+    int MaxValue() default Integer.MAX_VALUE;
+
+    int MinValue() default Integer.MIN_VALUE;
+
+    int DefaultValue();
+
+    boolean ThrowIfInvalid() default false;
+  }
+
+  /**
+   * Describes the requirements when validating the annotated long field.
+   */
+  @Target({ ElementType.FIELD })
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface LongConfigurationValidatorAnnotation {
+    String ConfigurationKey();
+
+    long MaxValue() default Long.MAX_VALUE;
+
+    long MinValue() default Long.MIN_VALUE;
+
+    long DefaultValue();
+
+    boolean ThrowIfInvalid() default false;
+  }
+
+  /**
+   * Describes the requirements when validating the annotated String field.
+   */
+  @Target({ ElementType.FIELD })
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface StringConfigurationValidatorAnnotation {
+    String ConfigurationKey();
+
+    String DefaultValue();
+
+    boolean ThrowIfInvalid() default false;
+  }
+
+  /**
+   * Describes the requirements when validating the annotated String field.
+   */
+  @Target({ ElementType.FIELD })
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface Base64StringConfigurationValidatorAnnotation {
+    String ConfigurationKey();
+
+    String DefaultValue();
+
+    boolean ThrowIfInvalid() default false;
+  }
+
+  /**
+   * Describes the requirements when validating the annotated boolean field.
+   */
+  @Target({ ElementType.FIELD })
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface BooleanConfigurationValidatorAnnotation {
+    String ConfigurationKey();
+
+    boolean DefaultValue();
+
+    boolean ThrowIfInvalid() default false;
+  }
+}

+ 22 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/annotations/package-info.java

@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.contracts.annotations;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 37 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/ConfigurationValidator.java

@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.contracts.diagnostics;
+
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+
+/**
+ * ConfigurationValidator to validate the value of a configuration key
+ * @param <T> the type of the validator and the validated value.
+ */
+@InterfaceStability.Evolving
+public interface ConfigurationValidator<T> {
+  /**
+   * Validates a configuration value.
+   * @param configValue the configuration value to be validated.
+   * @return validated value of type T
+   * @throws InvalidConfigurationValueException if the configuration value is invalid.
+   */
+  T validate(String configValue) throws InvalidConfigurationValueException;
+}

+ 22 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/diagnostics/package-info.java

@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.contracts.diagnostics;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

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

@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.fs.azurebfs.contracts.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
+import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation;
+
+/**
+ * Exception to wrap Azure service error responses.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class AbfsRestOperationException extends AzureBlobFileSystemException {
+  private final int statusCode;
+  private final AzureServiceErrorCode errorCode;
+  private final String errorMessage;
+
+  public AbfsRestOperationException(
+      final int statusCode,
+      final String errorCode,
+      final String errorMessage,
+      final Exception innerException) {
+    super("Status code: " + statusCode + " error code: " + errorCode + " error message: " + errorMessage, innerException);
+
+    this.statusCode = statusCode;
+    this.errorCode = AzureServiceErrorCode.getAzureServiceCode(this.statusCode, errorCode);
+    this.errorMessage = errorMessage;
+  }
+
+  public AbfsRestOperationException(
+      final int statusCode,
+      final String errorCode,
+      final String errorMessage,
+      final Exception innerException,
+      final AbfsHttpOperation abfsHttpOperation) {
+    super(formatMessage(abfsHttpOperation));
+
+    this.statusCode = statusCode;
+    this.errorCode = AzureServiceErrorCode.getAzureServiceCode(this.statusCode, errorCode);
+    this.errorMessage = errorMessage;
+  }
+
+  public int getStatusCode() {
+    return this.statusCode;
+  }
+
+  public AzureServiceErrorCode getErrorCode() {
+    return this.errorCode;
+  }
+
+  public String getErrorMessage() {
+    return this.errorMessage;
+  }
+
+  private static String formatMessage(final AbfsHttpOperation abfsHttpOperation) {
+    return String.format(
+        "%1$s %2$s%nStatusCode=%3$s%nStatusDescription=%4$s%nErrorCode=%5$s%nErrorMessage=%6$s",
+        abfsHttpOperation.getMethod(),
+        abfsHttpOperation.getUrl().toString(),
+        abfsHttpOperation.getStatusCode(),
+        abfsHttpOperation.getStatusDescription(),
+        abfsHttpOperation.getStorageErrorCode(),
+        abfsHttpOperation.getStorageErrorMessage());
+  }
+}

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

@@ -0,0 +1,56 @@
+/**
+ * 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 org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Base exception for any Azure Blob File System driver exceptions. All the exceptions must inherit this class.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class AzureBlobFileSystemException extends IOException {
+  public AzureBlobFileSystemException(final String message) {
+    super(message);
+  }
+
+  public AzureBlobFileSystemException(final String message, final Exception innerException) {
+    super(message, innerException);
+  }
+
+  @Override
+  public String toString() {
+    if (this.getMessage() == null && this.getCause() == null) {
+      return "AzureBlobFileSystemException";
+    }
+
+    if (this.getCause() == null) {
+      return this.getMessage();
+    }
+
+    if (this.getMessage() == null) {
+      return this.getCause().toString();
+    }
+
+    return this.getMessage() + this.getCause().toString();
+  }
+}

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

@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.azurebfs.contracts.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when a searched for element is not found
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class ConfigurationPropertyNotFoundException extends AzureBlobFileSystemException {
+  public ConfigurationPropertyNotFoundException(String property) {
+    super("Configuration property " + property + " not found.");
+  }
+}

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

@@ -0,0 +1,33 @@
+/**
+ * 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.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when an unhandled exception is occurred during a file system operation.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class FileSystemOperationUnhandledException extends AzureBlobFileSystemException {
+  public FileSystemOperationUnhandledException(Exception innerException) {
+    super("An unhandled file operation exception", innerException);
+  }
+}

+ 40 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/InvalidAbfsRestOperationException.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
+
+/**
+ * Exception to wrap invalid Azure service error responses.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class InvalidAbfsRestOperationException extends AbfsRestOperationException {
+  public InvalidAbfsRestOperationException(
+      final Exception innerException) {
+    super(
+        AzureServiceErrorCode.UNKNOWN.getStatusCode(),
+        AzureServiceErrorCode.UNKNOWN.getErrorCode(),
+        "InvalidAbfsRestOperationException",
+        innerException);
+  }
+}

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

@@ -0,0 +1,33 @@
+/**
+ * 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.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when there is an attempt to perform an invalid operation on an ACL.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class InvalidAclOperationException extends AzureBlobFileSystemException {
+  public InvalidAclOperationException(String message) {
+    super(message);
+  }
+}

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

@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.contracts.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when a configuration value is invalid
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class InvalidConfigurationValueException extends AzureBlobFileSystemException {
+  public InvalidConfigurationValueException(String configKey, Exception innerException) {
+    super("Invalid configuration value detected for " + configKey, innerException);
+  }
+
+  public InvalidConfigurationValueException(String configKey) {
+    super("Invalid configuration value detected for " + configKey);
+  }
+}

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

@@ -0,0 +1,33 @@
+/**
+ * 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.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when a file system property is invalid.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class InvalidFileSystemPropertyException extends AzureBlobFileSystemException {
+  public InvalidFileSystemPropertyException(String property) {
+    super(String.format("%s is invalid.", property));
+  }
+}

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

@@ -0,0 +1,33 @@
+/**
+ * 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.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when URI authority is invalid.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class InvalidUriAuthorityException extends AzureBlobFileSystemException {
+  public InvalidUriAuthorityException(String url) {
+    super(String.format("%s has invalid authority.", url));
+  }
+}

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

@@ -0,0 +1,33 @@
+/**
+ * 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.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when URI is invalid.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class InvalidUriException extends AzureBlobFileSystemException {
+  public InvalidUriException(String url) {
+    super(String.format("Invalid URI %s", url));
+  }
+}

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

@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.hadoop.classification.InterfaceAudience;
+
+/**
+ * Thrown if there is a problem instantiating a KeyProvider or retrieving a key
+ * using a KeyProvider object.
+ */
+@InterfaceAudience.Private
+public class KeyProviderException extends AzureBlobFileSystemException {
+  private static final long serialVersionUID = 1L;
+
+  public KeyProviderException(String message) {
+    super(message);
+  }
+
+  public KeyProviderException(String message, Throwable cause) {
+    super(message);
+  }
+
+  public KeyProviderException(Throwable t) {
+    super(t.getMessage());
+  }
+}

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

@@ -0,0 +1,33 @@
+/**
+ * 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.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when a timeout happens.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class TimeoutException extends AzureBlobFileSystemException {
+  public TimeoutException(String message) {
+    super(message);
+  }
+}

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

@@ -0,0 +1,36 @@
+/**
+ * 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.hadoop.classification.InterfaceAudience;
+
+/**
+ * Thrown if there is a problem instantiating a TokenAccessProvider or retrieving a configuration
+ * using a TokenAccessProvider object.
+ */
+@InterfaceAudience.Private
+public class TokenAccessProviderException extends AzureBlobFileSystemException {
+
+    public TokenAccessProviderException(String message) {
+        super(message);
+    }
+
+    public TokenAccessProviderException(String message, Throwable cause) {
+        super(message);
+    }
+}

+ 22 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/exceptions/package-info.java

@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.contracts.exceptions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 22 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/package-info.java

@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.contracts;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 115 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/AzureServiceErrorCode.java

@@ -0,0 +1,115 @@
+/**
+ * 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.services;
+
+import java.net.HttpURLConnection;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Azure service error codes.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public enum AzureServiceErrorCode {
+  FILE_SYSTEM_ALREADY_EXISTS("FilesystemAlreadyExists", HttpURLConnection.HTTP_CONFLICT, null),
+  PATH_ALREADY_EXISTS("PathAlreadyExists", HttpURLConnection.HTTP_CONFLICT, null),
+  INTERNAL_OPERATION_ABORT("InternalOperationAbortError", HttpURLConnection.HTTP_CONFLICT, null),
+  PATH_CONFLICT("PathConflict", HttpURLConnection.HTTP_CONFLICT, null),
+  FILE_SYSTEM_NOT_FOUND("FilesystemNotFound", HttpURLConnection.HTTP_NOT_FOUND, null),
+  PATH_NOT_FOUND("PathNotFound", HttpURLConnection.HTTP_NOT_FOUND, null),
+  PRE_CONDITION_FAILED("PreconditionFailed", HttpURLConnection.HTTP_PRECON_FAILED, null),
+  SOURCE_PATH_NOT_FOUND("SourcePathNotFound", HttpURLConnection.HTTP_NOT_FOUND, null),
+  INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE("InvalidSourceOrDestinationResourceType", HttpURLConnection.HTTP_CONFLICT, null),
+  RENAME_DESTINATION_PARENT_PATH_NOT_FOUND("RenameDestinationParentPathNotFound", HttpURLConnection.HTTP_NOT_FOUND, null),
+  INVALID_RENAME_SOURCE_PATH("InvalidRenameSourcePath", HttpURLConnection.HTTP_CONFLICT, null),
+  INGRESS_OVER_ACCOUNT_LIMIT(null, HttpURLConnection.HTTP_UNAVAILABLE, "Ingress is over the account limit."),
+  EGRESS_OVER_ACCOUNT_LIMIT(null, HttpURLConnection.HTTP_UNAVAILABLE, "Egress is over the account limit."),
+  INVALID_QUERY_PARAMETER_VALUE("InvalidQueryParameterValue", HttpURLConnection.HTTP_BAD_REQUEST, null),
+  AUTHORIZATION_PERMISSION_MISS_MATCH("AuthorizationPermissionMismatch", HttpURLConnection.HTTP_FORBIDDEN, null),
+  UNKNOWN(null, -1, null);
+
+  private final String errorCode;
+  private final int httpStatusCode;
+  private final String errorMessage;
+  AzureServiceErrorCode(String errorCode, int httpStatusCodes, String errorMessage) {
+    this.errorCode = errorCode;
+    this.httpStatusCode = httpStatusCodes;
+    this.errorMessage = errorMessage;
+  }
+
+  public int getStatusCode() {
+    return this.httpStatusCode;
+  }
+
+  public String getErrorCode() {
+    return this.errorCode;
+  }
+
+  public static List<AzureServiceErrorCode> getAzureServiceCode(int httpStatusCode) {
+    List<AzureServiceErrorCode> errorCodes = new ArrayList<>();
+    if (httpStatusCode == UNKNOWN.httpStatusCode) {
+      errorCodes.add(UNKNOWN);
+      return errorCodes;
+    }
+
+    for (AzureServiceErrorCode azureServiceErrorCode : AzureServiceErrorCode.values()) {
+      if (azureServiceErrorCode.httpStatusCode == httpStatusCode) {
+        errorCodes.add(azureServiceErrorCode);
+      }
+    }
+
+    return errorCodes;
+  }
+
+  public static AzureServiceErrorCode getAzureServiceCode(int httpStatusCode, String errorCode) {
+    if (errorCode == null || errorCode.isEmpty() || httpStatusCode == UNKNOWN.httpStatusCode) {
+      return UNKNOWN;
+    }
+
+    for (AzureServiceErrorCode azureServiceErrorCode : AzureServiceErrorCode.values()) {
+      if (errorCode.equalsIgnoreCase(azureServiceErrorCode.errorCode)
+          && azureServiceErrorCode.httpStatusCode == httpStatusCode) {
+        return azureServiceErrorCode;
+      }
+    }
+
+    return UNKNOWN;
+  }
+
+  public static AzureServiceErrorCode getAzureServiceCode(int httpStatusCode, String errorCode, final String errorMessage) {
+    if (errorCode == null || errorCode.isEmpty() || httpStatusCode == UNKNOWN.httpStatusCode || errorMessage == null || errorMessage.isEmpty()) {
+      return UNKNOWN;
+    }
+
+    for (AzureServiceErrorCode azureServiceErrorCode : AzureServiceErrorCode.values()) {
+      if (azureServiceErrorCode.httpStatusCode == httpStatusCode
+          && errorCode.equalsIgnoreCase(azureServiceErrorCode.errorCode)
+          && errorMessage.equalsIgnoreCase(azureServiceErrorCode.errorMessage)
+      ) {
+        return azureServiceErrorCode;
+      }
+    }
+
+    return UNKNOWN;
+  }
+}

+ 239 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java

@@ -0,0 +1,239 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.services;
+
+import org.codehaus.jackson.annotate.JsonProperty;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * The ListResultEntrySchema model.
+ */
+@InterfaceStability.Evolving
+public class ListResultEntrySchema {
+  /**
+   * The name property.
+   */
+  @JsonProperty(value = "name")
+  private String name;
+
+  /**
+   * The isDirectory property.
+   */
+  @JsonProperty(value = "isDirectory")
+  private Boolean isDirectory;
+
+  /**
+   * The lastModified property.
+   */
+  @JsonProperty(value = "lastModified")
+  private String lastModified;
+
+  /**
+   * The eTag property.
+   */
+  @JsonProperty(value = "etag")
+  private String eTag;
+
+  /**
+   * The contentLength property.
+   */
+  @JsonProperty(value = "contentLength")
+  private Long contentLength;
+
+  /**
+   * The owner property.
+   */
+  @JsonProperty(value = "owner")
+  private String owner;
+
+  /**
+   * The group property.
+   */
+  @JsonProperty(value = "group")
+  private String group;
+
+  /**
+   * The permissions property.
+   */
+  @JsonProperty(value = "permissions")
+  private String permissions;
+
+  /**
+   * Get the name value.
+   *
+   * @return the name value
+   */
+  public String name() {
+    return name;
+  }
+
+  /**
+   * Set the name value.
+   *
+   * @param name the name value to set
+   * @return the ListEntrySchema object itself.
+   */
+  public ListResultEntrySchema withName(String name) {
+    this.name = name;
+    return this;
+  }
+
+  /**
+   * Get the isDirectory value.
+   *
+   * @return the isDirectory value
+   */
+  public Boolean isDirectory() {
+    return isDirectory;
+  }
+
+  /**
+   * Set the isDirectory value.
+   *
+   * @param isDirectory the isDirectory value to set
+   * @return the ListEntrySchema object itself.
+   */
+  public ListResultEntrySchema withIsDirectory(final Boolean isDirectory) {
+    this.isDirectory = isDirectory;
+    return this;
+  }
+
+  /**
+   * Get the lastModified value.
+   *
+   * @return the lastModified value
+   */
+  public String lastModified() {
+    return lastModified;
+  }
+
+  /**
+   * Set the lastModified value.
+   *
+   * @param lastModified the lastModified value to set
+   * @return the ListEntrySchema object itself.
+   */
+  public ListResultEntrySchema withLastModified(String lastModified) {
+    this.lastModified = lastModified;
+    return this;
+  }
+
+  /**
+   * Get the etag value.
+   *
+   * @return the etag value
+   */
+  public String eTag() {
+    return eTag;
+  }
+
+  /**
+   * Set the eTag value.
+   *
+   * @param eTag the eTag value to set
+   * @return the ListEntrySchema object itself.
+   */
+  public ListResultEntrySchema withETag(final String eTag) {
+    this.eTag = eTag;
+    return this;
+  }
+
+  /**
+   * Get the contentLength value.
+   *
+   * @return the contentLength value
+   */
+  public Long contentLength() {
+    return contentLength;
+  }
+
+  /**
+   * Set the contentLength value.
+   *
+   * @param contentLength the contentLength value to set
+   * @return the ListEntrySchema object itself.
+   */
+  public ListResultEntrySchema withContentLength(final Long contentLength) {
+    this.contentLength = contentLength;
+    return this;
+  }
+
+  /**
+   *
+   Get the owner value.
+   *
+   * @return the owner value
+   */
+  public String owner() {
+    return owner;
+  }
+
+  /**
+   * Set the owner value.
+   *
+   * @param owner the owner value to set
+   * @return the ListEntrySchema object itself.
+   */
+  public ListResultEntrySchema withOwner(final String owner) {
+    this.owner = owner;
+    return this;
+  }
+
+  /**
+   * Get the group value.
+   *
+   * @return the group value
+   */
+  public String group() {
+    return group;
+  }
+
+  /**
+   * Set the group value.
+   *
+   * @param group the group value to set
+   * @return the ListEntrySchema object itself.
+   */
+  public ListResultEntrySchema withGroup(final String group) {
+    this.group = group;
+    return this;
+  }
+
+  /**
+   * Get the permissions value.
+   *
+   * @return the permissions value
+   */
+  public String permissions() {
+    return permissions;
+  }
+
+  /**
+   * Set the permissions value.
+   *
+   * @param permissions the permissions value to set
+   * @return the ListEntrySchema object itself.
+   */
+  public ListResultEntrySchema withPermissions(final String permissions) {
+    this.permissions = permissions;
+    return this;
+  }
+
+}

+ 58 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultSchema.java

@@ -0,0 +1,58 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.services;
+
+import java.util.List;
+
+import org.codehaus.jackson.annotate.JsonProperty;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * The ListResultSchema model.
+ */
+@InterfaceStability.Evolving
+public class ListResultSchema {
+  /**
+   * The paths property.
+   */
+  @JsonProperty(value = "paths")
+  private List<ListResultEntrySchema> paths;
+
+  /**
+   * * Get the paths value.
+   *
+   * @return the paths value
+   */
+  public List<ListResultEntrySchema> paths() {
+    return this.paths;
+  }
+
+  /**
+   * Set the paths value.
+   *
+   * @param paths the paths value to set
+   * @return the ListSchema object itself.
+   */
+  public ListResultSchema withPaths(final List<ListResultEntrySchema> paths) {
+    this.paths = paths;
+    return this;
+  }
+
+}

+ 29 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ReadBufferStatus.java

@@ -0,0 +1,29 @@
+/**
+ * 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.services;
+
+/**
+ * The ReadBufferStatus for Rest AbfsClient
+ */
+public enum ReadBufferStatus {
+  NOT_AVAILABLE,  // buffers sitting in readaheadqueue have this stats
+  READING_IN_PROGRESS,  // reading is in progress on this buffer. Buffer should be in inProgressList
+  AVAILABLE,  // data is available in buffer. It should be in completedList
+  READ_FAILED  // read completed, but failed.
+}

+ 22 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/package-info.java

@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.contracts.services;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 50 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/Base64StringConfigurationBasicValidator.java

@@ -0,0 +1,50 @@
+/**
+ * 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.diagnostics;
+
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+
+/**
+* String Base64 configuration value Validator.
+*/
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class Base64StringConfigurationBasicValidator extends ConfigurationBasicValidator<String> implements ConfigurationValidator{
+
+  public Base64StringConfigurationBasicValidator(final String configKey, final String defaultVal, final boolean throwIfInvalid){
+    super(configKey, defaultVal, throwIfInvalid);
+  }
+
+  public String validate(final String configValue) throws InvalidConfigurationValueException {
+    String result = super.validate((configValue));
+    if (result != null) {
+      return result;
+    }
+
+    if (!Base64.validateIsBase64String(configValue)) {
+      throw new InvalidConfigurationValueException(getConfigKey());
+    }
+    return configValue;
+  }
+}

+ 50 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/BooleanConfigurationBasicValidator.java

@@ -0,0 +1,50 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.diagnostics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+
+/**
+ * Boolean configuration value validator.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class BooleanConfigurationBasicValidator extends ConfigurationBasicValidator<Boolean> {
+  private static final String TRUE = "true";
+  private static final String FALSE = "false";
+
+  public BooleanConfigurationBasicValidator(final String configKey, final boolean defaultVal, final boolean throwIfInvalid) {
+    super(configKey, defaultVal, throwIfInvalid);
+  }
+
+  public Boolean validate(final String configValue) throws InvalidConfigurationValueException {
+    Boolean result = super.validate(configValue);
+    if (result != null) {
+      return result;
+    }
+
+    if (configValue.equalsIgnoreCase(TRUE) || configValue.equalsIgnoreCase(FALSE)) {
+      return Boolean.valueOf(configValue);
+    }
+
+    throw new InvalidConfigurationValueException(getConfigKey());
+  }
+}

+ 67 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/ConfigurationBasicValidator.java

@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.diagnostics;
+
+import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+
+/**
+ * ConfigurationBasicValidator covers the base case of missing user defined configuration value
+ * @param <T> the type of the validated value
+ */
+abstract class ConfigurationBasicValidator<T> implements ConfigurationValidator {
+  private final T defaultVal;
+  private final String configKey;
+  private final boolean throwIfInvalid;
+
+  ConfigurationBasicValidator(final String configKey, final T defaultVal, final boolean throwIfInvalid) {
+    this.configKey = configKey;
+    this.defaultVal = defaultVal;
+    this.throwIfInvalid = throwIfInvalid;
+  }
+
+  /**
+   * This method handles the base case where the configValue is null, based on the throwIfInvalid it either throws or returns the defaultVal,
+   * otherwise it returns null indicating that the configValue needs to be validated further.
+   * @param configValue the configuration value set by the user
+   * @return the defaultVal in case the configValue is null and not required to be set, null in case the configValue not null
+   * @throws InvalidConfigurationValueException in case the configValue is null and required to be set
+   */
+  public T validate(final String configValue) throws InvalidConfigurationValueException {
+    if (configValue == null) {
+      if (this.throwIfInvalid) {
+        throw new InvalidConfigurationValueException(this.configKey);
+      }
+      return this.defaultVal;
+    }
+    return null;
+  }
+
+  public T getDefaultVal() {
+    return this.defaultVal;
+  }
+
+  public String getConfigKey() {
+    return this.configKey;
+  }
+
+  public boolean getThrowIfInvalid() {
+    return this.throwIfInvalid;
+  }
+}

+ 68 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/IntegerConfigurationBasicValidator.java

@@ -0,0 +1,68 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.diagnostics;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+
+/**
+ * Integer configuration value Validator.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class IntegerConfigurationBasicValidator extends ConfigurationBasicValidator<Integer> implements ConfigurationValidator {
+  private final int min;
+  private final int max;
+
+  public IntegerConfigurationBasicValidator(final int min, final int max, final int defaultVal, final String configKey, final boolean throwIfInvalid) {
+    super(configKey, defaultVal, throwIfInvalid);
+    this.min = min;
+    this.max = max;
+  }
+
+  public Integer validate(final String configValue) throws InvalidConfigurationValueException {
+    Integer result = super.validate(configValue);
+    if (result != null) {
+      return result;
+    }
+
+    try {
+      result = Integer.parseInt(configValue);
+      // throw an exception if a 'within bounds' value is missing
+      if (getThrowIfInvalid() && (result < this.min || result > this.max)) {
+        throw new InvalidConfigurationValueException(getConfigKey());
+      }
+
+      // set the value to the nearest bound if it's out of bounds
+      if (result < this.min) {
+        return this.min;
+      }
+
+      if (result > this.max) {
+        return this.max;
+      }
+    } catch (NumberFormatException ex) {
+      throw new InvalidConfigurationValueException(getConfigKey(), ex);
+    }
+
+    return result;
+  }
+}

+ 63 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/LongConfigurationBasicValidator.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.diagnostics;
+
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+
+/**
+ * Long configuration value Validator.
+ */
+@InterfaceStability.Evolving
+public class LongConfigurationBasicValidator extends ConfigurationBasicValidator<Long> implements ConfigurationValidator {
+  private final long min;
+  private final long max;
+
+  public LongConfigurationBasicValidator(final long min, final long max, final long defaultVal, final String configKey, final boolean throwIfInvalid) {
+    super(configKey, defaultVal, throwIfInvalid);
+    this.min = min;
+    this.max = max;
+  }
+
+  public Long validate(final String configValue) throws InvalidConfigurationValueException {
+    Long result = super.validate(configValue);
+    if (result != null) {
+      return result;
+    }
+
+    try {
+      result = Long.parseLong(configValue);
+      // throw an exception if a 'within bounds' value is missing
+      if (getThrowIfInvalid() && (result < this.min || result > this.max)) {
+        throw new InvalidConfigurationValueException(getConfigKey());
+      }
+
+      // set the value to the nearest bound if it's out of bounds
+      if (result < this.min) {
+        return this.min;
+      } else if (result > this.max) {
+        return this.max;
+      }
+    } catch (NumberFormatException ex) {
+      throw new InvalidConfigurationValueException(getConfigKey(), ex);
+    }
+
+    return result;
+  }
+}

+ 43 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/StringConfigurationBasicValidator.java

@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.diagnostics;
+
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.contracts.diagnostics.ConfigurationValidator;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+
+/**
+ * String configuration value Validator.
+ */
+@InterfaceStability.Evolving
+public class StringConfigurationBasicValidator extends ConfigurationBasicValidator<String> implements ConfigurationValidator{
+
+  public StringConfigurationBasicValidator(final String configKey, final String defaultVal, final boolean throwIfInvalid){
+    super(configKey, defaultVal, throwIfInvalid);
+  }
+
+  public String validate(final String configValue) throws InvalidConfigurationValueException {
+    String result = super.validate((configValue));
+    if (result != null) {
+      return result;
+    }
+
+    return configValue;
+  }
+}

+ 22 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/diagnostics/package-info.java

@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.diagnostics;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 70 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomDelegationTokenManager.java

@@ -0,0 +1,70 @@
+/**
+ * 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.extensions;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+
+/**
+ * Interface for Managing the Delegation tokens.
+ */
+@InterfaceAudience.LimitedPrivate("authorization-subsystems")
+@InterfaceStability.Unstable
+public interface CustomDelegationTokenManager {
+
+  /**
+   * Initialize with supported configuration. This method is invoked when the
+   * (URI, Configuration)} method is invoked.
+   *
+   * @param configuration Configuration object
+   * @throws IOException if instance can not be configured.
+   */
+  void initialize(Configuration configuration)
+          throws IOException;
+
+  /**
+   * Get Delegation token.
+   * @param renewer delegation token renewer
+   * @return delegation token
+   * @throws IOException when error in getting the delegation token
+   */
+  Token<DelegationTokenIdentifier> getDelegationToken(String renewer)
+          throws IOException;
+
+  /**
+   * Renew the delegation token.
+   * @param token delegation token.
+   * @return renewed time.
+   * @throws IOException when error in renewing the delegation token
+   */
+  long renewDelegationToken(Token<?> token) throws IOException;
+
+  /**
+   * Cancel the delegation token.
+   * @param token delegation token.
+   * @throws IOException when error in cancelling the delegation token.
+   */
+  void cancelDelegationToken(Token<?> token) throws IOException;
+}

+ 75 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/CustomTokenProviderAdaptee.java

@@ -0,0 +1,75 @@
+/**
+ * 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.extensions;
+
+import java.io.IOException;
+import java.util.Date;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+
+
+/**
+ * This interface provides an extensibility model for customizing the acquisition
+ * of Azure Active Directory Access Tokens.   When "fs.azure.account.auth.type" is
+ * set to "Custom", implementors may use the
+ * "fs.azure.account.oauth.provider.type.{accountName}" configuration property
+ * to specify a class with a custom implementation of CustomTokenProviderAdaptee.
+ * This class will be dynamically loaded, initialized, and invoked to provide
+ * AAD Access Tokens and their Expiry.
+ */
+@InterfaceAudience.LimitedPrivate("authorization-subsystems")
+@InterfaceStability.Unstable
+public interface CustomTokenProviderAdaptee {
+
+  /**
+   * Initialize with supported configuration. This method is invoked when the
+   * (URI, Configuration)} method is invoked.
+   *
+   * @param configuration Configuration object
+   * @param accountName Account Name
+   * @throws IOException if instance can not be configured.
+   */
+  void initialize(Configuration configuration, String accountName)
+      throws IOException;
+
+  /**
+   * Obtain the access token that should be added to https connection's header.
+   * Will be called depending upon {@link #getExpiryTime()} expiry time is set,
+   * so implementations should be performant. Implementations are responsible
+   * for any refreshing of the token.
+   *
+   * @return String containing the access token
+   * @throws IOException if there is an error fetching the token
+   */
+  String getAccessToken() throws IOException;
+
+  /**
+   * Obtain expiry time of the token. If implementation is performant enough to
+   * maintain expiry and expect {@link #getAccessToken()} call for every
+   * connection then safe to return current or past time.
+   *
+   * However recommended to use the token expiry time received from Azure Active
+   * Directory.
+   *
+   * @return Date to expire access token retrieved from AAD.
+   */
+  Date getExpiryTime();
+}

+ 32 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/package-info.java

@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * This package is for extension points under ABFS;
+ * There are no stability guarantees as these extension points are
+ * deep inside the ABFS implementation code.
+ *
+ * Note, however: this is how the ABFS client integrates with
+ * authorization services and other aspects of Azure's infrastructure.
+ * Do not change these APIs without good reason or detailed discussion.
+ */
+@InterfaceAudience.LimitedPrivate("authorization-subsystems")
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.azurebfs.extensions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 98 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AccessTokenProvider.java

@@ -0,0 +1,98 @@
+/**
+ * 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.oauth2;
+
+import java.io.IOException;
+import java.util.Date;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Returns an Azure Active Directory token when requested. The provider can
+ * cache the token if it has already retrieved one. If it does, then the
+ * provider is responsible for checking expiry and refreshing as needed.
+ *
+ * In other words, this is is a token cache that fetches tokens when
+ * requested, if the cached token has expired.
+ *
+ */
+public abstract class AccessTokenProvider {
+
+  private AzureADToken token;
+  private static final Logger LOG = LoggerFactory.getLogger(AccessTokenProvider.class);
+
+  /**
+   * returns the {@link AzureADToken} cached (or retrieved) by this instance.
+   *
+   * @return {@link AzureADToken} containing the access token
+   * @throws IOException if there is an error fetching the token
+   */
+  public synchronized AzureADToken getToken() throws IOException {
+    if (isTokenAboutToExpire()) {
+      LOG.debug("AAD Token is missing or expired:"
+              + " Calling refresh-token from abstract base class");
+      token = refreshToken();
+    }
+    return token;
+  }
+
+  /**
+   * the method to fetch the access token. Derived classes should override
+   * this method to actually get the token from Azure Active Directory.
+   *
+   * This method will be called initially, and then once when the token
+   * is about to expire.
+   *
+   *
+   * @return {@link AzureADToken} containing the access token
+   * @throws IOException if there is an error fetching the token
+   */
+  protected abstract AzureADToken refreshToken() throws IOException;
+
+  /**
+   * Checks if the token is about to expire in the next 5 minutes.
+   * The 5 minute allowance is to allow for clock skew and also to
+   * allow for token to be refreshed in that much time.
+   *
+   * @return true if the token is expiring in next 5 minutes
+   */
+  private boolean isTokenAboutToExpire() {
+    if (token == null) {
+      LOG.debug("AADToken: no token. Returning expiring=true");
+      return true;   // no token should have same response as expired token
+    }
+    boolean expiring = false;
+    // allow 5 minutes for clock skew
+    long approximatelyNow = System.currentTimeMillis() + FIVE_MINUTES;
+    if (token.getExpiry().getTime() < approximatelyNow) {
+      expiring = true;
+    }
+    if (expiring) {
+      LOG.debug("AADToken: token expiring: "
+              + token.getExpiry().toString()
+              + " : Five-minute window: "
+              + new Date(approximatelyNow).toString());
+    }
+
+    return expiring;
+  }
+
+  // 5 minutes in milliseconds
+  private static final long FIVE_MINUTES = 300 * 1000;
+}

+ 344 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADAuthenticator.java

@@ -0,0 +1,344 @@
+/**
+ * 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.oauth2;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.Date;
+import java.util.Hashtable;
+import java.util.Map;
+
+import com.google.common.base.Preconditions;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.JsonToken;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.services.ExponentialRetryPolicy;
+
+/**
+ * This class provides convenience methods to obtain AAD tokens.
+ * While convenient, it is not necessary to use these methods to
+ * obtain the tokens. Customers can use any other method
+ * (e.g., using the adal4j client) to obtain tokens.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class AzureADAuthenticator {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AzureADAuthenticator.class);
+  private static final String RESOURCE_NAME = "https://storage.azure.com/";
+  private static final int CONNECT_TIMEOUT = 30 * 1000;
+  private static final int READ_TIMEOUT = 30 * 1000;
+
+  private AzureADAuthenticator() {
+    // no operation
+  }
+
+  /**
+   * gets Azure Active Directory token using the user ID and password of
+   * a service principal (that is, Web App in Azure Active Directory).
+   *
+   * Azure Active Directory allows users to set up a web app as a
+   * service principal. Users can optionally obtain service principal keys
+   * from AAD. This method gets a token using a service principal's client ID
+   * and keys. In addition, it needs the token endpoint associated with the
+   * user's directory.
+   *
+   *
+   * @param authEndpoint the OAuth 2.0 token endpoint associated
+   *                     with the user's directory (obtain from
+   *                     Active Directory configuration)
+   * @param clientId     the client ID (GUID) of the client web app
+   *                     btained from Azure Active Directory configuration
+   * @param clientSecret the secret key of the client web app
+   * @return {@link AzureADToken} obtained using the creds
+   * @throws IOException throws IOException if there is a failure in connecting to Azure AD
+   */
+  public static AzureADToken getTokenUsingClientCreds(String authEndpoint,
+                                                      String clientId, String clientSecret)
+          throws IOException {
+    Preconditions.checkNotNull(authEndpoint, "authEndpoint");
+    Preconditions.checkNotNull(clientId, "clientId");
+    Preconditions.checkNotNull(clientSecret, "clientSecret");
+
+    QueryParams qp = new QueryParams();
+    qp.add("resource", RESOURCE_NAME);
+    qp.add("grant_type", "client_credentials");
+    qp.add("client_id", clientId);
+    qp.add("client_secret", clientSecret);
+    LOG.debug("AADToken: starting to fetch token using client creds for client ID " + clientId);
+
+    return getTokenCall(authEndpoint, qp.serialize(), null, null);
+  }
+
+  /**
+   * Gets AAD token from the local virtual machine's VM extension. This only works on
+   * an Azure VM with MSI extension
+   * enabled.
+   *
+   * @param tenantGuid  (optional) The guid of the AAD tenant. Can be {@code null}.
+   * @param clientId    (optional) The clientId guid of the MSI service
+   *                    principal to use. Can be {@code null}.
+   * @param bypassCache {@code boolean} specifying whether a cached token is acceptable or a fresh token
+   *                    request should me made to AAD
+   * @return {@link AzureADToken} obtained using the creds
+   * @throws IOException throws IOException if there is a failure in obtaining the token
+   */
+  public static AzureADToken getTokenFromMsi(String tenantGuid, String clientId,
+                                             boolean bypassCache) throws IOException {
+    Preconditions.checkNotNull(tenantGuid, "tenantGuid");
+    Preconditions.checkNotNull(clientId, "clientId");
+
+    String authEndpoint = "http://169.254.169.254/metadata/identity/oauth2/token";
+
+    QueryParams qp = new QueryParams();
+    qp.add("api-version", "2018-02-01");
+    qp.add("resource", RESOURCE_NAME);
+
+
+    if (tenantGuid.length() > 0) {
+      String authority = "https://login.microsoftonline.com/" + tenantGuid;
+      qp.add("authority", authority);
+    }
+
+    if (clientId.length() > 0) {
+      qp.add("client_id", clientId);
+    }
+
+    if (bypassCache) {
+      qp.add("bypass_cache", "true");
+    }
+
+    Hashtable<String, String> headers = new Hashtable<>();
+    headers.put("Metadata", "true");
+
+    LOG.debug("AADToken: starting to fetch token using MSI");
+    return getTokenCall(authEndpoint, qp.serialize(), headers, "GET");
+  }
+
+  /**
+   * Gets Azure Active Directory token using refresh token.
+   *
+   * @param clientId the client ID (GUID) of the client web app obtained from Azure Active Directory configuration
+   * @param refreshToken the refresh token
+   * @return {@link AzureADToken} obtained using the refresh token
+   * @throws IOException throws IOException if there is a failure in connecting to Azure AD
+   */
+  public static AzureADToken getTokenUsingRefreshToken(String clientId,
+                                                       String refreshToken) throws IOException {
+    String authEndpoint = "https://login.microsoftonline.com/Common/oauth2/token";
+    QueryParams qp = new QueryParams();
+    qp.add("grant_type", "refresh_token");
+    qp.add("refresh_token", refreshToken);
+    if (clientId != null) {
+      qp.add("client_id", clientId);
+    }
+    LOG.debug("AADToken: starting to fetch token using refresh token for client ID " + clientId);
+    return getTokenCall(authEndpoint, qp.serialize(), null, null);
+  }
+
+  private static class HttpException extends IOException {
+    private int httpErrorCode;
+    private String requestId;
+
+    public int getHttpErrorCode() {
+      return this.httpErrorCode;
+    }
+
+    public String getRequestId() {
+      return this.requestId;
+    }
+
+    HttpException(int httpErrorCode, String requestId, String message) {
+      super(message);
+      this.httpErrorCode = httpErrorCode;
+      this.requestId = requestId;
+    }
+  }
+
+  private static AzureADToken getTokenCall(String authEndpoint, String body,
+                                           Hashtable<String, String> headers, String httpMethod)
+          throws IOException {
+    AzureADToken token = null;
+    ExponentialRetryPolicy retryPolicy
+            = new ExponentialRetryPolicy(3, 0, 1000, 2);
+
+    int httperror = 0;
+    String requestId;
+    String httpExceptionMessage = null;
+    IOException ex = null;
+    boolean succeeded = false;
+    int retryCount = 0;
+    do {
+      httperror = 0;
+      requestId = "";
+      ex = null;
+      try {
+        token = getTokenSingleCall(authEndpoint, body, headers, httpMethod);
+      } catch (HttpException e) {
+        httperror = e.httpErrorCode;
+        requestId = e.requestId;
+        httpExceptionMessage = e.getMessage();
+      } catch (IOException e) {
+        ex = e;
+      }
+      succeeded = ((httperror == 0) && (ex == null));
+      retryCount++;
+    } while (!succeeded && retryPolicy.shouldRetry(retryCount, httperror));
+    if (!succeeded) {
+      if (ex != null) {
+        throw ex;
+      }
+      if (httperror != 0) {
+        throw new IOException(httpExceptionMessage);
+      }
+    }
+    return token;
+  }
+
+  private static AzureADToken getTokenSingleCall(
+          String authEndpoint, String payload, Hashtable<String, String> headers, String httpMethod)
+          throws IOException {
+
+    AzureADToken token = null;
+    HttpURLConnection conn = null;
+    String urlString = authEndpoint;
+
+    httpMethod = (httpMethod == null) ? "POST" : httpMethod;
+    if (httpMethod.equals("GET")) {
+      urlString = urlString + "?" + payload;
+    }
+
+    try {
+      URL url = new URL(urlString);
+      conn = (HttpURLConnection) url.openConnection();
+      conn.setRequestMethod(httpMethod);
+      conn.setReadTimeout(READ_TIMEOUT);
+      conn.setConnectTimeout(CONNECT_TIMEOUT);
+
+      if (headers != null && headers.size() > 0) {
+        for (Map.Entry<String, String> entry : headers.entrySet()) {
+          conn.setRequestProperty(entry.getKey(), entry.getValue());
+        }
+      }
+      conn.setRequestProperty("Connection", "close");
+
+      if (httpMethod.equals("POST")) {
+        conn.setDoOutput(true);
+        conn.getOutputStream().write(payload.getBytes("UTF-8"));
+      }
+
+      int httpResponseCode = conn.getResponseCode();
+      String requestId = conn.getHeaderField("x-ms-request-id");
+      String responseContentType = conn.getHeaderField("Content-Type");
+      long responseContentLength = conn.getHeaderFieldLong("Content-Length", 0);
+
+      requestId = requestId == null ? "" : requestId;
+      if (httpResponseCode == HttpURLConnection.HTTP_OK
+              && responseContentType.startsWith("application/json") && responseContentLength > 0) {
+        InputStream httpResponseStream = conn.getInputStream();
+        token = parseTokenFromStream(httpResponseStream);
+      } else {
+        String responseBody = consumeInputStream(conn.getInputStream(), 1024);
+        String proxies = "none";
+        String httpProxy = System.getProperty("http.proxy");
+        String httpsProxy = System.getProperty("https.proxy");
+        if (httpProxy != null || httpsProxy != null) {
+          proxies = "http:" + httpProxy + "; https:" + httpsProxy;
+        }
+        String logMessage =
+                "AADToken: HTTP connection failed for getting token from AzureAD. Http response: "
+                        + httpResponseCode + " " + conn.getResponseMessage()
+                        + " Content-Type: " + responseContentType
+                        + " Content-Length: " + responseContentLength
+                        + " Request ID: " + requestId.toString()
+                        + " Proxies: " + proxies
+                        + " First 1K of Body: " + responseBody;
+        LOG.debug(logMessage);
+        throw new HttpException(httpResponseCode, requestId, logMessage);
+      }
+    } finally {
+      if (conn != null) {
+        conn.disconnect();
+      }
+    }
+    return token;
+  }
+
+  private static AzureADToken parseTokenFromStream(InputStream httpResponseStream) throws IOException {
+    AzureADToken token = new AzureADToken();
+    try {
+      int expiryPeriod = 0;
+
+      JsonFactory jf = new JsonFactory();
+      JsonParser jp = jf.createJsonParser(httpResponseStream);
+      String fieldName, fieldValue;
+      jp.nextToken();
+      while (jp.hasCurrentToken()) {
+        if (jp.getCurrentToken() == JsonToken.FIELD_NAME) {
+          fieldName = jp.getCurrentName();
+          jp.nextToken();  // field value
+          fieldValue = jp.getText();
+
+          if (fieldName.equals("access_token")) {
+            token.setAccessToken(fieldValue);
+          }
+          if (fieldName.equals("expires_in")) {
+            expiryPeriod = Integer.parseInt(fieldValue);
+          }
+        }
+        jp.nextToken();
+      }
+      jp.close();
+      long expiry = System.currentTimeMillis();
+      expiry = expiry + expiryPeriod * 1000L; // convert expiryPeriod to milliseconds and add
+      token.setExpiry(new Date(expiry));
+      LOG.debug("AADToken: fetched token with expiry " + token.getExpiry().toString());
+    } catch (Exception ex) {
+      LOG.debug("AADToken: got exception when parsing json token " + ex.toString());
+      throw ex;
+    } finally {
+      httpResponseStream.close();
+    }
+    return token;
+  }
+
+  private static String consumeInputStream(InputStream inStream, int length) throws IOException {
+    byte[] b = new byte[length];
+    int totalBytesRead = 0;
+    int bytesRead = 0;
+
+    do {
+      bytesRead = inStream.read(b, totalBytesRead, length - totalBytesRead);
+      if (bytesRead > 0) {
+        totalBytesRead += bytesRead;
+      }
+    } while (bytesRead >= 0 && totalBytesRead < length);
+
+    return new String(b, 0, totalBytesRead, StandardCharsets.UTF_8);
+  }
+}

+ 47 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/AzureADToken.java

@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.oauth2;
+
+import java.util.Date;
+
+
+/**
+ * Object representing the AAD access token to use when making HTTP requests to Azure Data Lake Storage.
+ */
+public class AzureADToken {
+  private String accessToken;
+  private Date expiry;
+
+  public String getAccessToken() {
+    return this.accessToken;
+  }
+
+  public void setAccessToken(String accessToken) {
+    this.accessToken = accessToken;
+  }
+
+  public Date getExpiry() {
+    return new Date(this.expiry.getTime());
+  }
+
+  public void setExpiry(Date expiry) {
+    this.expiry = new Date(expiry.getTime());
+  }
+
+}

+ 62 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/ClientCredsTokenProvider.java

@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.oauth2;
+
+import java.io.IOException;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Provides tokens based on client credentials.
+ */
+public class ClientCredsTokenProvider extends AccessTokenProvider {
+
+  private final String authEndpoint;
+
+  private final String clientId;
+
+  private final String clientSecret;
+
+  private static final Logger LOG = LoggerFactory.getLogger(AccessTokenProvider.class);
+
+
+  public ClientCredsTokenProvider(final String authEndpoint,
+                                  final String clientId, final String clientSecret) {
+
+    Preconditions.checkNotNull(authEndpoint, "authEndpoint");
+    Preconditions.checkNotNull(clientId, "clientId");
+    Preconditions.checkNotNull(clientSecret, "clientSecret");
+
+    this.authEndpoint = authEndpoint;
+    this.clientId = clientId;
+    this.clientSecret = clientSecret;
+  }
+
+
+  @Override
+  protected AzureADToken refreshToken() throws IOException {
+    LOG.debug("AADToken: refreshing client-credential based token");
+    return AzureADAuthenticator.getTokenUsingClientCreds(authEndpoint, clientId, clientSecret);
+  }
+
+
+}

+ 58 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/CustomTokenProviderAdapter.java

@@ -0,0 +1,58 @@
+/**
+ * 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.oauth2;
+
+
+import java.io.IOException;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azurebfs.extensions.CustomTokenProviderAdaptee;
+
+/**
+ * Provides tokens based on custom implementation, following the Adapter Design
+ * Pattern.
+ */
+public final class CustomTokenProviderAdapter extends AccessTokenProvider {
+
+  private CustomTokenProviderAdaptee adaptee;
+  private static final Logger LOG = LoggerFactory.getLogger(AccessTokenProvider.class);
+
+  /**
+   * Constructs a token provider based on the custom token provider.
+   *
+   * @param adaptee the custom token provider
+   */
+  public CustomTokenProviderAdapter(CustomTokenProviderAdaptee adaptee) {
+    Preconditions.checkNotNull(adaptee, "adaptee");
+    this.adaptee = adaptee;
+  }
+
+  protected AzureADToken refreshToken() throws IOException {
+    LOG.debug("AADToken: refreshing custom based token");
+
+    AzureADToken azureADToken = new AzureADToken();
+    azureADToken.setAccessToken(adaptee.getAccessToken());
+    azureADToken.setExpiry(adaptee.getExpiryTime());
+
+    return azureADToken;
+  }
+}

+ 48 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/MsiTokenProvider.java

@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.oauth2;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Provides tokens based on Azure VM's Managed Service Identity.
+ */
+public class MsiTokenProvider extends AccessTokenProvider {
+
+  private final String tenantGuid;
+
+  private final String clientId;
+
+  private static final Logger LOG = LoggerFactory.getLogger(AccessTokenProvider.class);
+
+  public MsiTokenProvider(final String tenantGuid, final String clientId) {
+    this.tenantGuid = tenantGuid;
+    this.clientId = clientId;
+  }
+
+  @Override
+  protected AzureADToken refreshToken() throws IOException {
+    LOG.debug("AADToken: refreshing token from MSI");
+    AzureADToken token = AzureADAuthenticator.getTokenFromMsi(tenantGuid, clientId, false);
+    return token;
+  }
+}

+ 69 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/QueryParams.java

@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.oauth2;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLEncoder;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Utilities class http query parameters.
+ */
+public class QueryParams {
+  private Map<String, String> params = new HashMap<>();
+  private String apiVersion = null;
+  private String separator = "";
+  private String serializedString = null;
+
+  public void add(String name, String value) {
+    params.put(name, value);
+    serializedString = null;
+  }
+
+  public void setApiVersion(String apiVersion) {
+    this.apiVersion = apiVersion;
+    serializedString = null;
+  }
+
+  public String serialize() {
+    if (serializedString == null) {
+      StringBuilder sb = new StringBuilder();
+      for (Map.Entry<String, String> entry : params.entrySet()) {
+        String name = entry.getKey();
+        try {
+          sb.append(separator);
+          sb.append(URLEncoder.encode(name, "UTF-8"));
+          sb.append('=');
+          sb.append(URLEncoder.encode(entry.getValue(), "UTF-8"));
+          separator = "&";
+        } catch (UnsupportedEncodingException ex) {
+        }
+      }
+
+      if (apiVersion != null) {
+        sb.append(separator);
+        sb.append("api-version=");
+        sb.append(apiVersion);
+        separator = "&";
+      }
+      serializedString = sb.toString();
+    }
+    return serializedString;
+  }
+}

+ 57 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/RefreshTokenBasedTokenProvider.java

@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.oauth2;
+
+import java.io.IOException;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Provides tokens based on refresh token.
+ */
+public class RefreshTokenBasedTokenProvider extends AccessTokenProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(AccessTokenProvider.class);
+
+  private final String clientId;
+
+  private final String refreshToken;
+
+  /**
+   * Constructs a token provider based on the refresh token provided.
+   *
+   * @param clientId the client ID (GUID) of the client web app obtained from Azure Active Directory configuration
+   * @param refreshToken the refresh token
+   */
+  public RefreshTokenBasedTokenProvider(String clientId, String refreshToken) {
+    Preconditions.checkNotNull(clientId, "clientId");
+    Preconditions.checkNotNull(refreshToken, "refreshToken");
+    this.clientId = clientId;
+    this.refreshToken = refreshToken;
+  }
+
+
+  @Override
+  protected AzureADToken refreshToken() throws IOException {
+    LOG.debug("AADToken: refreshing refresh-token based token");
+    return AzureADAuthenticator.getTokenUsingRefreshToken(clientId, refreshToken);
+  }
+}

+ 56 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/UserPasswordTokenProvider.java

@@ -0,0 +1,56 @@
+/**
+ * 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.oauth2;
+
+import java.io.IOException;
+
+import com.google.common.base.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Provides tokens based on username and password.
+ */
+public class UserPasswordTokenProvider extends AccessTokenProvider {
+
+  private final String authEndpoint;
+
+  private final String username;
+
+  private final String password;
+
+  private static final Logger LOG = LoggerFactory.getLogger(AccessTokenProvider.class);
+
+  public UserPasswordTokenProvider(final String authEndpoint,
+                                   final String username, final String password) {
+    Preconditions.checkNotNull(authEndpoint, "authEndpoint");
+    Preconditions.checkNotNull(username, "username");
+    Preconditions.checkNotNull(password, "password");
+
+    this.authEndpoint = authEndpoint;
+    this.username = username;
+    this.password = password;
+  }
+
+  @Override
+  protected AzureADToken refreshToken() throws IOException {
+    LOG.debug("AADToken: refreshing user-password based token");
+    return AzureADAuthenticator.getTokenUsingClientCreds(authEndpoint, username, password);
+  }
+}

+ 18 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/oauth2/package-info.java

@@ -0,0 +1,18 @@
+/**
+ * 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.oauth2;

+ 31 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/package.html

@@ -0,0 +1,31 @@
+<html>
+
+<!--
+   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.
+-->
+
+<body>
+
+<p>
+A distributed implementation of {@link
+org.apache.hadoop.fs.FileSystem} for reading and writing files on
+<a href="http://store.azure.com">Azure Storage</a>.
+This implementation stores files on Azure in their native form for
+interoperability with other Azure tools.
+</p>
+
+</body>
+</html>

+ 49 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsDelegationTokenIdentifier.java

@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.fs.azurebfs.security;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier;
+
+/**
+ * Delegation token Identifier for ABFS delegation tokens.
+ */
+public class AbfsDelegationTokenIdentifier extends DelegationTokenIdentifier {
+  public static final Text TOKEN_KIND = new Text("ABFS delegation");
+
+  public AbfsDelegationTokenIdentifier(){
+    super(TOKEN_KIND);
+  }
+
+  public AbfsDelegationTokenIdentifier(Text kind) {
+    super(kind);
+  }
+
+  public AbfsDelegationTokenIdentifier(Text kind, Text owner, Text renewer,
+      Text realUser) {
+    super(kind, owner, renewer, realUser);
+  }
+
+  @Override
+  public Text getKind() {
+    return TOKEN_KIND;
+  }
+
+}

+ 88 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsDelegationTokenManager.java

@@ -0,0 +1,88 @@
+/**
+ * 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.security;
+
+import java.io.IOException;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.extensions.CustomDelegationTokenManager;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * Class for delegation token Manager.
+ */
+public class AbfsDelegationTokenManager {
+
+  private CustomDelegationTokenManager tokenManager;
+  private static final Logger LOG =
+          LoggerFactory.getLogger(AbfsDelegationTokenManager.class);
+
+  public AbfsDelegationTokenManager(final Configuration conf) throws IOException {
+
+    Preconditions.checkNotNull(conf, "conf");
+
+    Class<? extends CustomDelegationTokenManager> customDelegationTokenMgrClass =
+            conf.getClass(ConfigurationKeys.FS_AZURE_DELEGATION_TOKEN_PROVIDER_TYPE, null,
+                    CustomDelegationTokenManager.class);
+
+    if (customDelegationTokenMgrClass == null) {
+      throw new IllegalArgumentException(
+              "The value for \"fs.azure.delegation.token.provider.type\" is not defined.");
+    }
+
+    CustomDelegationTokenManager customTokenMgr = (CustomDelegationTokenManager) ReflectionUtils
+            .newInstance(customDelegationTokenMgrClass, conf);
+    if (customTokenMgr == null) {
+      throw new IllegalArgumentException(String.format("Failed to initialize %s.", customDelegationTokenMgrClass));
+    }
+
+    customTokenMgr.initialize(conf);
+
+    tokenManager = customTokenMgr;
+  }
+
+  public Token<DelegationTokenIdentifier> getDelegationToken(
+      String renewer) throws IOException {
+
+    Token<DelegationTokenIdentifier> token = tokenManager.getDelegationToken(renewer);
+
+    token.setKind(AbfsDelegationTokenIdentifier.TOKEN_KIND);
+    return token;
+  }
+
+  public long renewDelegationToken(Token<?> token)
+      throws IOException {
+
+    return tokenManager.renewDelegationToken(token);
+  }
+
+  public void cancelDelegationToken(Token<?> token)
+          throws IOException {
+
+    tokenManager.cancelDelegationToken(token);
+  }
+}

+ 96 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/AbfsTokenRenewer.java

@@ -0,0 +1,96 @@
+/**
+ * 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.security;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenRenewer;
+
+/**
+ * Token Renewer for renewing ABFS delegation tokens with remote service.
+ */
+public class AbfsTokenRenewer extends TokenRenewer {
+  public static final Logger LOG =
+          LoggerFactory.getLogger(AbfsTokenRenewer.class);
+
+  /**
+   * Checks if this particular object handles the Kind of token passed.
+   *
+   * @param kind the kind of the token
+   * @return true if it handles passed token kind false otherwise.
+   */
+  @Override
+  public boolean handleKind(Text kind) {
+    return AbfsDelegationTokenIdentifier.TOKEN_KIND.equals(kind);
+  }
+
+  /**
+   * Checks if passed token is managed.
+   *
+   * @param token the token being checked
+   * @return true if it is managed.
+   * @throws IOException thrown when evaluating if token is managed.
+   */
+  @Override
+  public boolean isManaged(Token<?> token) throws IOException {
+    return true;
+  }
+
+  /**
+   * Renew the delegation token.
+   *
+   * @param token token to renew.
+   * @param conf  configuration object.
+   * @return extended expiry time of the token.
+   * @throws IOException          thrown when trying get current user.
+   * @throws InterruptedException thrown when thread is interrupted
+   */
+  @Override
+  public long renew(final Token<?> token, Configuration conf)
+          throws IOException, InterruptedException {
+    LOG.debug("Renewing the delegation token");
+    return getInstance(conf).renewDelegationToken(token);
+  }
+
+  /**
+   * Cancel the delegation token.
+   *
+   * @param token token to cancel.
+   * @param conf  configuration object.
+   * @throws IOException          thrown when trying get current user.
+   * @throws InterruptedException thrown when thread is interrupted.
+   */
+  @Override
+  public void cancel(final Token<?> token, Configuration conf)
+          throws IOException, InterruptedException {
+    LOG.debug("Cancelling the delegation token");
+    getInstance(conf).cancelDelegationToken(token);
+  }
+
+  private AbfsDelegationTokenManager getInstance(Configuration conf)
+          throws IOException {
+    return new AbfsDelegationTokenManager(conf);
+  }
+}

+ 23 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/package-info.java

@@ -0,0 +1,23 @@
+/**
+ * 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.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.azurebfs.security;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

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

@@ -0,0 +1,202 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidAclOperationException;
+import org.apache.hadoop.fs.permission.FsAction;
+
+/**
+ * AbfsAclHelper provides convenience methods to implement modifyAclEntries / removeAclEntries / removeAcl / removeDefaultAcl
+ * from setAcl and getAcl.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class AbfsAclHelper {
+
+  private AbfsAclHelper() {
+    // not called
+  }
+
+  public static Map<String, String> deserializeAclSpec(final String aclSpecString) {
+    final Map<String, String> aclEntries  = new HashMap<>();
+    final String[] aclArray = aclSpecString.split(AbfsHttpConstants.COMMA);
+    for (String acl : aclArray) {
+      int idx = acl.lastIndexOf(AbfsHttpConstants.COLON);
+      aclEntries.put(acl.substring(0, idx), acl.substring(idx + 1));
+    }
+    return aclEntries;
+  }
+
+  public static String serializeAclSpec(final Map<String, String> aclEntries) {
+    final StringBuilder sb = new StringBuilder();
+    for (Map.Entry<String, String> aclEntry : aclEntries.entrySet()) {
+      sb.append(aclEntry.getKey() + AbfsHttpConstants.COLON + aclEntry.getValue() + AbfsHttpConstants.COMMA);
+    }
+    if (sb.length() > 0) {
+      sb.setLength(sb.length() - 1);
+    }
+    return sb.toString();
+  }
+
+  public static String processAclString(final String aclSpecString) {
+    final List<String> aclEntries = Arrays.asList(aclSpecString.split(AbfsHttpConstants.COMMA));
+    final StringBuilder sb = new StringBuilder();
+
+    boolean containsMask = false;
+    for (int i = aclEntries.size() - 1; i >= 0; i--) {
+      String ace = aclEntries.get(i);
+      if (ace.startsWith(AbfsHttpConstants.ACCESS_OTHER)|| ace.startsWith(AbfsHttpConstants.ACCESS_USER + AbfsHttpConstants.COLON)) {
+        // skip
+      } else if (ace.startsWith(AbfsHttpConstants.ACCESS_MASK)) {
+        containsMask = true;
+        // skip
+      } else if (ace.startsWith(AbfsHttpConstants.ACCESS_GROUP + AbfsHttpConstants.COLON) && !containsMask) {
+        // skip
+      } else {
+        sb.insert(0, ace + AbfsHttpConstants.COMMA);
+      }
+    }
+
+    return sb.length() == 0 ? AbfsHttpConstants.EMPTY_STRING : sb.substring(0, sb.length() - 1);
+  }
+
+  public static void removeAclEntriesInternal(Map<String, String> aclEntries, Map<String, String> toRemoveEntries)
+      throws AzureBlobFileSystemException {
+    boolean accessAclTouched = false;
+    boolean defaultAclTouched = false;
+
+    final Set<String> removeIndicationSet = new HashSet<>();
+
+    for (String entryKey : toRemoveEntries.keySet()) {
+      final boolean isDefaultAcl = isDefaultAce(entryKey);
+      if (removeNamedAceAndUpdateSet(entryKey, isDefaultAcl, removeIndicationSet, aclEntries)) {
+        if (isDefaultAcl) {
+          defaultAclTouched = true;
+        } else {
+          accessAclTouched = true;
+        }
+      }
+    }
+    if (accessAclTouched) {
+      if (removeIndicationSet.contains(AbfsHttpConstants.ACCESS_MASK)) {
+        aclEntries.remove(AbfsHttpConstants.ACCESS_MASK);
+      }
+      recalculateMask(aclEntries, false);
+    }
+    if (defaultAclTouched) {
+      if (removeIndicationSet.contains(AbfsHttpConstants.DEFAULT_MASK)) {
+        aclEntries.remove(AbfsHttpConstants.DEFAULT_MASK);
+      }
+      if (removeIndicationSet.contains(AbfsHttpConstants.DEFAULT_USER)) {
+        aclEntries.put(AbfsHttpConstants.DEFAULT_USER, aclEntries.get(AbfsHttpConstants.ACCESS_USER));
+      }
+      if (removeIndicationSet.contains(AbfsHttpConstants.DEFAULT_GROUP)) {
+        aclEntries.put(AbfsHttpConstants.DEFAULT_GROUP, aclEntries.get(AbfsHttpConstants.ACCESS_GROUP));
+      }
+      if (removeIndicationSet.contains(AbfsHttpConstants.DEFAULT_OTHER)) {
+        aclEntries.put(AbfsHttpConstants.DEFAULT_OTHER, aclEntries.get(AbfsHttpConstants.ACCESS_OTHER));
+      }
+      recalculateMask(aclEntries, true);
+    }
+  }
+
+  private static boolean removeNamedAceAndUpdateSet(String entry, boolean isDefaultAcl, Set<String> removeIndicationSet,
+                                                    Map<String, String> aclEntries)
+      throws AzureBlobFileSystemException {
+    final int startIndex = isDefaultAcl ? 1 : 0;
+    final String[] entryParts = entry.split(AbfsHttpConstants.COLON);
+    final String tag = isDefaultAcl ? AbfsHttpConstants.DEFAULT_SCOPE + entryParts[startIndex] + AbfsHttpConstants.COLON
+        : entryParts[startIndex] + AbfsHttpConstants.COLON;
+
+    if ((entry.equals(AbfsHttpConstants.ACCESS_USER) || entry.equals(AbfsHttpConstants.ACCESS_GROUP)
+        || entry.equals(AbfsHttpConstants.ACCESS_OTHER))
+        && !isNamedAce(entry)) {
+      throw new InvalidAclOperationException("Cannot remove user, group or other entry from access ACL.");
+    }
+
+    boolean touched = false;
+    if (!isNamedAce(entry)) {
+      removeIndicationSet.add(tag); // this must not be a access user, group or other
+      touched = true;
+    } else {
+      if (aclEntries.remove(entry) != null) {
+        touched = true;
+      }
+    }
+    return touched;
+  }
+
+  private static void recalculateMask(Map<String, String> aclEntries, boolean isDefaultMask) {
+    FsAction umask = FsAction.NONE;
+    if (!isExtendAcl(aclEntries, isDefaultMask)) {
+      return;
+    }
+
+    for (Map.Entry<String, String> aclEntry : aclEntries.entrySet()) {
+      if (isDefaultMask) {
+        if ((isDefaultAce(aclEntry.getKey()) && isNamedAce(aclEntry.getKey()))
+            || aclEntry.getKey().equals(AbfsHttpConstants.DEFAULT_GROUP)) {
+          umask = umask.or(FsAction.getFsAction(aclEntry.getValue()));
+        }
+      } else {
+        if ((!isDefaultAce(aclEntry.getKey()) && isNamedAce(aclEntry.getKey()))
+            || aclEntry.getKey().equals(AbfsHttpConstants.ACCESS_GROUP)) {
+          umask = umask.or(FsAction.getFsAction(aclEntry.getValue()));
+        }
+      }
+    }
+
+    aclEntries.put(isDefaultMask ? AbfsHttpConstants.DEFAULT_MASK : AbfsHttpConstants.ACCESS_MASK, umask.SYMBOL);
+  }
+
+  private static boolean isExtendAcl(Map<String, String> aclEntries, boolean checkDefault) {
+    for (String entryKey : aclEntries.keySet()) {
+      if (checkDefault && !(entryKey.equals(AbfsHttpConstants.DEFAULT_USER)
+          || entryKey.equals(AbfsHttpConstants.DEFAULT_GROUP)
+          || entryKey.equals(AbfsHttpConstants.DEFAULT_OTHER) || !isDefaultAce(entryKey))) {
+        return true;
+      }
+      if (!checkDefault && !(entryKey.equals(AbfsHttpConstants.ACCESS_USER)
+          || entryKey.equals(AbfsHttpConstants.ACCESS_GROUP)
+          || entryKey.equals(AbfsHttpConstants.ACCESS_OTHER) || isDefaultAce(entryKey))) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private static boolean isDefaultAce(String entry) {
+    return entry.startsWith(AbfsHttpConstants.DEFAULT_SCOPE);
+  }
+
+  private static boolean isNamedAce(String entry) {
+    return entry.charAt(entry.length() - 1) != AbfsHttpConstants.COLON.charAt(0);
+  }
+}

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

@@ -0,0 +1,581 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.UnsupportedEncodingException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidUriException;
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider;
+
+import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*;
+import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.*;
+
+/**
+ * AbfsClient.
+ */
+public class AbfsClient {
+  public static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class);
+  private final URL baseUrl;
+  private final SharedKeyCredentials sharedKeyCredentials;
+  private final String xMsVersion = "2018-06-17";
+  private final ExponentialRetryPolicy retryPolicy;
+  private final String filesystem;
+  private final AbfsConfiguration abfsConfiguration;
+  private final String userAgent;
+
+  private final AccessTokenProvider tokenProvider;
+
+
+  public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials,
+                    final AbfsConfiguration abfsConfiguration,
+                    final ExponentialRetryPolicy exponentialRetryPolicy,
+                    final AccessTokenProvider tokenProvider) {
+    this.baseUrl = baseUrl;
+    this.sharedKeyCredentials = sharedKeyCredentials;
+    String baseUrlString = baseUrl.toString();
+    this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(FORWARD_SLASH) + 1);
+    this.abfsConfiguration = abfsConfiguration;
+    this.retryPolicy = exponentialRetryPolicy;
+
+    String sslProviderName = null;
+
+    if (this.baseUrl.toString().startsWith(HTTPS_SCHEME)) {
+      try {
+        SSLSocketFactoryEx.initializeDefaultFactory(this.abfsConfiguration.getPreferredSSLFactoryOption());
+        sslProviderName = SSLSocketFactoryEx.getDefaultFactory().getProviderName();
+      } catch (IOException e) {
+        // Suppress exception. Failure to init SSLSocketFactoryEx would have only performance impact.
+      }
+    }
+
+    this.userAgent = initializeUserAgent(abfsConfiguration, sslProviderName);
+    this.tokenProvider = tokenProvider;
+  }
+
+  public String getFileSystem() {
+    return filesystem;
+  }
+
+  ExponentialRetryPolicy getRetryPolicy() {
+    return retryPolicy;
+  }
+
+  SharedKeyCredentials getSharedKeyCredentials() {
+    return sharedKeyCredentials;
+  }
+
+  List<AbfsHttpHeader> createDefaultHeaders() {
+    final List<AbfsHttpHeader> requestHeaders = new ArrayList<AbfsHttpHeader>();
+    requestHeaders.add(new AbfsHttpHeader(X_MS_VERSION, xMsVersion));
+    requestHeaders.add(new AbfsHttpHeader(ACCEPT, APPLICATION_JSON
+            + COMMA + SINGLE_WHITE_SPACE + APPLICATION_OCTET_STREAM));
+    requestHeaders.add(new AbfsHttpHeader(ACCEPT_CHARSET,
+            UTF_8));
+    requestHeaders.add(new AbfsHttpHeader(CONTENT_TYPE, EMPTY_STRING));
+    requestHeaders.add(new AbfsHttpHeader(USER_AGENT, userAgent));
+    return requestHeaders;
+  }
+
+  AbfsUriQueryBuilder createDefaultUriQueryBuilder() {
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_TIMEOUT, DEFAULT_TIMEOUT);
+    return abfsUriQueryBuilder;
+  }
+
+  public AbfsRestOperation createFilesystem() throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
+
+    final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            AbfsRestOperationType.CreateFileSystem,
+            this,
+            HTTP_METHOD_PUT,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation setFilesystemProperties(final String properties) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    // JDK7 does not support PATCH, so to workaround the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
+            HTTP_METHOD_PATCH));
+
+    requestHeaders.add(new AbfsHttpHeader(X_MS_PROPERTIES,
+            properties));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
+
+    final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            AbfsRestOperationType.SetFileSystemProperties,
+            this,
+            HTTP_METHOD_PUT,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation listPath(final String relativePath, final boolean recursive, final int listMaxResults,
+                                    final String continuation) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_DIRECTORY, relativePath == null ? AbfsHttpConstants.EMPTY_STRING
+        : relativePath);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RECURSIVE, String.valueOf(recursive));
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_MAXRESULTS, String.valueOf(listMaxResults));
+
+    final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            AbfsRestOperationType.ListPaths,
+            this,
+            HTTP_METHOD_GET,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation getFilesystemProperties() throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
+
+    final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            AbfsRestOperationType.GetFileSystemProperties,
+            this,
+            HTTP_METHOD_HEAD,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation deleteFilesystem() throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM);
+
+    final URL url = createRequestUrl(abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            AbfsRestOperationType.DeleteFileSystem,
+            this,
+            HTTP_METHOD_DELETE,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite,
+                                      final String permission, final String umask) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    if (!overwrite) {
+      requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, AbfsHttpConstants.STAR));
+    }
+
+    if (permission != null && !permission.isEmpty()) {
+      requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PERMISSIONS, permission));
+    }
+
+    if (umask != null && !umask.isEmpty()) {
+      requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_UMASK, umask));
+    }
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, isFile ? FILE : DIRECTORY);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            AbfsRestOperationType.CreatePath,
+            this,
+            HTTP_METHOD_PUT,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation renamePath(final String source, final String destination, final String continuation)
+          throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final String encodedRenameSource = urlEncode(FORWARD_SLASH + this.getFileSystem() + source);
+    requestHeaders.add(new AbfsHttpHeader(X_MS_RENAME_SOURCE, encodedRenameSource));
+    requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, STAR));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation);
+
+    final URL url = createRequestUrl(destination, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            AbfsRestOperationType.RenamePath,
+            this,
+            HTTP_METHOD_PUT,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation append(final String path, final long position, final byte[] buffer, final int offset,
+                                  final int length) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    // JDK7 does not support PATCH, so to workaround the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
+            HTTP_METHOD_PATCH));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, APPEND_ACTION);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(position));
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+        AbfsRestOperationType.Append,
+            this,
+            HTTP_METHOD_PUT,
+            url,
+            requestHeaders, buffer, offset, length);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData)
+      throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    // JDK7 does not support PATCH, so to workaround the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
+            HTTP_METHOD_PATCH));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, FLUSH_ACTION);
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_POSITION, Long.toString(position));
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RETAIN_UNCOMMITTED_DATA, String.valueOf(retainUncommittedData));
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            AbfsRestOperationType.Flush,
+            this,
+            HTTP_METHOD_PUT,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation setPathProperties(final String path, final String properties)
+      throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    // JDK7 does not support PATCH, so to workaround the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
+            HTTP_METHOD_PATCH));
+
+    requestHeaders.add(new AbfsHttpHeader(X_MS_PROPERTIES, properties));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_ACTION, SET_PROPERTIES_ACTION);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            AbfsRestOperationType.SetPathProperties,
+            this,
+            HTTP_METHOD_PUT,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation getPathProperties(final String path) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            AbfsRestOperationType.GetPathProperties,
+            this,
+            HTTP_METHOD_HEAD,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset,
+                                final int bufferLength, final String eTag) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    requestHeaders.add(new AbfsHttpHeader(RANGE,
+            String.format("bytes=%d-%d", position, position + bufferLength - 1)));
+    requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+
+    final AbfsRestOperation op = new AbfsRestOperation(
+            AbfsRestOperationType.ReadFile,
+            this,
+            HTTP_METHOD_GET,
+            url,
+            requestHeaders,
+            buffer,
+            bufferOffset,
+            bufferLength);
+    op.execute();
+
+    return op;
+  }
+
+  public AbfsRestOperation deletePath(final String path, final boolean recursive, final String continuation)
+          throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_RECURSIVE, String.valueOf(recursive));
+    abfsUriQueryBuilder.addQuery(QUERY_PARAM_CONTINUATION, continuation);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+            AbfsRestOperationType.DeletePath,
+            this,
+            HTTP_METHOD_DELETE,
+            url,
+            requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation setOwner(final String path, final String owner, final String group)
+      throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    // JDK7 does not support PATCH, so to workaround the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
+            HTTP_METHOD_PATCH));
+
+    if (owner != null && !owner.isEmpty()) {
+      requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_OWNER, owner));
+    }
+    if (group != null && !group.isEmpty()) {
+      requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_GROUP, group));
+    }
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_ACCESS_CONTROL);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+        AbfsRestOperationType.SetOwner,
+        this,
+        AbfsHttpConstants.HTTP_METHOD_PUT,
+        url,
+        requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation setPermission(final String path, final String permission)
+      throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    // JDK7 does not support PATCH, so to workaround the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
+            HTTP_METHOD_PATCH));
+
+    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PERMISSIONS, permission));
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_ACCESS_CONTROL);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+        AbfsRestOperationType.SetPermissions,
+        this,
+        AbfsHttpConstants.HTTP_METHOD_PUT,
+        url,
+        requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation setAcl(final String path, final String aclSpecString) throws AzureBlobFileSystemException {
+    return setAcl(path, aclSpecString, AbfsHttpConstants.EMPTY_STRING);
+  }
+
+  public AbfsRestOperation setAcl(final String path, final String aclSpecString, final String eTag)
+      throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+    // JDK7 does not support PATCH, so to workaround the issue we will use
+    // PUT and specify the real method in the X-Http-Method-Override header.
+    requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE,
+            HTTP_METHOD_PATCH));
+
+    requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_ACL, aclSpecString));
+
+    if (eTag != null && !eTag.isEmpty()) {
+      requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.IF_MATCH, eTag));
+    }
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.SET_ACCESS_CONTROL);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+        AbfsRestOperationType.SetAcl,
+        this,
+        AbfsHttpConstants.HTTP_METHOD_PUT,
+        url,
+        requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  public AbfsRestOperation getAclStatus(final String path) throws AzureBlobFileSystemException {
+    final List<AbfsHttpHeader> requestHeaders = createDefaultHeaders();
+
+    final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder();
+    abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.GET_ACCESS_CONTROL);
+
+    final URL url = createRequestUrl(path, abfsUriQueryBuilder.toString());
+    final AbfsRestOperation op = new AbfsRestOperation(
+        AbfsRestOperationType.GetAcl,
+        this,
+        AbfsHttpConstants.HTTP_METHOD_HEAD,
+        url,
+        requestHeaders);
+    op.execute();
+    return op;
+  }
+
+  private URL createRequestUrl(final String query) throws AzureBlobFileSystemException {
+    return createRequestUrl(EMPTY_STRING, query);
+  }
+
+  private URL createRequestUrl(final String path, final String query)
+          throws AzureBlobFileSystemException {
+    final String base = baseUrl.toString();
+    String encodedPath = path;
+    try {
+      encodedPath = urlEncode(path);
+    } catch (AzureBlobFileSystemException ex) {
+      LOG.debug("Unexpected error.", ex);
+      throw new InvalidUriException(path);
+    }
+
+    final StringBuilder sb = new StringBuilder();
+    sb.append(base);
+    sb.append(encodedPath);
+    sb.append(query);
+
+    final URL url;
+    try {
+      url = new URL(sb.toString());
+    } catch (MalformedURLException ex) {
+      throw new InvalidUriException(sb.toString());
+    }
+    return url;
+  }
+
+  public static String urlEncode(final String value) throws AzureBlobFileSystemException {
+    String encodedString;
+    try {
+      encodedString =  URLEncoder.encode(value, UTF_8)
+          .replace(PLUS, PLUS_ENCODE)
+          .replace(FORWARD_SLASH_ENCODE, FORWARD_SLASH);
+    } catch (UnsupportedEncodingException ex) {
+        throw new InvalidUriException(value);
+    }
+
+    return encodedString;
+  }
+
+  public synchronized String getAccessToken() throws IOException {
+    if (tokenProvider != null) {
+      return "Bearer " + tokenProvider.getToken().getAccessToken();
+    } else {
+      return null;
+    }
+  }
+
+  @VisibleForTesting
+  String initializeUserAgent(final AbfsConfiguration abfsConfiguration,
+                             final String sslProviderName) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("(JavaJRE ");
+    sb.append(System.getProperty(JAVA_VERSION));
+    sb.append("; ");
+    sb.append(
+        System.getProperty(OS_NAME).replaceAll(SINGLE_WHITE_SPACE, EMPTY_STRING));
+    sb.append(" ");
+    sb.append(System.getProperty(OS_VERSION));
+    if (sslProviderName != null && !sslProviderName.isEmpty()) {
+      sb.append("; ");
+      sb.append(sslProviderName);
+    }
+    sb.append(")");
+    final String userAgentComment = sb.toString();
+    String customUserAgentId = abfsConfiguration.getCustomUserAgentPrefix();
+    if (customUserAgentId != null && !customUserAgentId.isEmpty()) {
+      return String.format(Locale.ROOT, CLIENT_VERSION + " %s %s",
+          userAgentComment, customUserAgentId);
+    }
+    return String.format(CLIENT_VERSION + " %s", userAgentComment);
+  }
+
+  @VisibleForTesting
+  URL getBaseUrl() {
+    return baseUrl;
+  }
+}

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

@@ -0,0 +1,272 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class AbfsClientThrottlingAnalyzer {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbfsClientThrottlingAnalyzer.class);
+  private static final int DEFAULT_ANALYSIS_PERIOD_MS = 10 * 1000;
+  private static final int MIN_ANALYSIS_PERIOD_MS = 1000;
+  private static final int MAX_ANALYSIS_PERIOD_MS = 30000;
+  private static final double MIN_ACCEPTABLE_ERROR_PERCENTAGE = .1;
+  private static final double MAX_EQUILIBRIUM_ERROR_PERCENTAGE = 1;
+  private static final double RAPID_SLEEP_DECREASE_FACTOR = .75;
+  private static final double RAPID_SLEEP_DECREASE_TRANSITION_PERIOD_MS = 150
+      * 1000;
+  private static final double SLEEP_DECREASE_FACTOR = .975;
+  private static final double SLEEP_INCREASE_FACTOR = 1.05;
+  private int analysisPeriodMs;
+
+  private volatile int sleepDuration = 0;
+  private long consecutiveNoErrorCount = 0;
+  private String name = null;
+  private Timer timer = null;
+  private AtomicReference<AbfsOperationMetrics> blobMetrics = null;
+
+  private AbfsClientThrottlingAnalyzer() {
+    // hide default constructor
+  }
+
+  /**
+   * Creates an instance of the <code>AbfsClientThrottlingAnalyzer</code> class with
+   * the specified name.
+   *
+   * @param name a name used to identify this instance.
+   * @throws IllegalArgumentException if name is null or empty.
+   */
+  AbfsClientThrottlingAnalyzer(String name) throws IllegalArgumentException {
+    this(name, DEFAULT_ANALYSIS_PERIOD_MS);
+  }
+
+  /**
+   * Creates an instance of the <code>AbfsClientThrottlingAnalyzer</code> class with
+   * the specified name and period.
+   *
+   * @param name   A name used to identify this instance.
+   * @param period The frequency, in milliseconds, at which metrics are
+   *               analyzed.
+   * @throws IllegalArgumentException If name is null or empty.
+   *                                  If period is less than 1000 or greater than 30000 milliseconds.
+   */
+  AbfsClientThrottlingAnalyzer(String name, int period)
+      throws IllegalArgumentException {
+    Preconditions.checkArgument(
+        StringUtils.isNotEmpty(name),
+        "The argument 'name' cannot be null or empty.");
+    Preconditions.checkArgument(
+        period >= MIN_ANALYSIS_PERIOD_MS && period <= MAX_ANALYSIS_PERIOD_MS,
+        "The argument 'period' must be between 1000 and 30000.");
+    this.name = name;
+    this.analysisPeriodMs = period;
+    this.blobMetrics = new AtomicReference<AbfsOperationMetrics>(
+        new AbfsOperationMetrics(System.currentTimeMillis()));
+    this.timer = new Timer(
+        String.format("abfs-timer-client-throttling-analyzer-%s", name), true);
+    this.timer.schedule(new TimerTaskImpl(),
+        analysisPeriodMs,
+        analysisPeriodMs);
+  }
+
+  /**
+   * Updates metrics with results from the current storage operation.
+   *
+   * @param count             The count of bytes transferred.
+   * @param isFailedOperation True if the operation failed; otherwise false.
+   */
+  public void addBytesTransferred(long count, boolean isFailedOperation) {
+    AbfsOperationMetrics metrics = blobMetrics.get();
+    if (isFailedOperation) {
+      metrics.bytesFailed.addAndGet(count);
+      metrics.operationsFailed.incrementAndGet();
+    } else {
+      metrics.bytesSuccessful.addAndGet(count);
+      metrics.operationsSuccessful.incrementAndGet();
+    }
+  }
+
+  /**
+   * Suspends the current storage operation, as necessary, to reduce throughput.
+   */
+  public void suspendIfNecessary() {
+    int duration = sleepDuration;
+    if (duration > 0) {
+      try {
+        Thread.sleep(duration);
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+      }
+    }
+  }
+
+  @VisibleForTesting
+  int getSleepDuration() {
+    return sleepDuration;
+  }
+
+  private int analyzeMetricsAndUpdateSleepDuration(AbfsOperationMetrics metrics,
+                                                   int sleepDuration) {
+    final double percentageConversionFactor = 100;
+    double bytesFailed = metrics.bytesFailed.get();
+    double bytesSuccessful = metrics.bytesSuccessful.get();
+    double operationsFailed = metrics.operationsFailed.get();
+    double operationsSuccessful = metrics.operationsSuccessful.get();
+    double errorPercentage = (bytesFailed <= 0)
+        ? 0
+        : (percentageConversionFactor
+        * bytesFailed
+        / (bytesFailed + bytesSuccessful));
+    long periodMs = metrics.endTime - metrics.startTime;
+
+    double newSleepDuration;
+
+    if (errorPercentage < MIN_ACCEPTABLE_ERROR_PERCENTAGE) {
+      ++consecutiveNoErrorCount;
+      // Decrease sleepDuration in order to increase throughput.
+      double reductionFactor =
+          (consecutiveNoErrorCount * analysisPeriodMs
+              >= RAPID_SLEEP_DECREASE_TRANSITION_PERIOD_MS)
+              ? RAPID_SLEEP_DECREASE_FACTOR
+              : SLEEP_DECREASE_FACTOR;
+
+      newSleepDuration = sleepDuration * reductionFactor;
+    } else if (errorPercentage < MAX_EQUILIBRIUM_ERROR_PERCENTAGE) {
+      // Do not modify sleepDuration in order to stabilize throughput.
+      newSleepDuration = sleepDuration;
+    } else {
+      // Increase sleepDuration in order to minimize error rate.
+      consecutiveNoErrorCount = 0;
+
+      // Increase sleep duration in order to reduce throughput and error rate.
+      // First, calculate target throughput: bytesSuccessful / periodMs.
+      // Next, calculate time required to send *all* data (assuming next period
+      // is similar to previous) at the target throughput: (bytesSuccessful
+      // + bytesFailed) * periodMs / bytesSuccessful. Next, subtract periodMs to
+      // get the total additional delay needed.
+      double additionalDelayNeeded = 5 * analysisPeriodMs;
+      if (bytesSuccessful > 0) {
+        additionalDelayNeeded = (bytesSuccessful + bytesFailed)
+            * periodMs
+            / bytesSuccessful
+            - periodMs;
+      }
+
+      // amortize the additional delay needed across the estimated number of
+      // requests during the next period
+      newSleepDuration = additionalDelayNeeded
+          / (operationsFailed + operationsSuccessful);
+
+      final double maxSleepDuration = analysisPeriodMs;
+      final double minSleepDuration = sleepDuration * SLEEP_INCREASE_FACTOR;
+
+      // Add 1 ms to avoid rounding down and to decrease proximity to the server
+      // side ingress/egress limit.  Ensure that the new sleep duration is
+      // larger than the current one to more quickly reduce the number of
+      // errors.  Don't allow the sleep duration to grow unbounded, after a
+      // certain point throttling won't help, for example, if there are far too
+      // many tasks/containers/nodes no amount of throttling will help.
+      newSleepDuration = Math.max(newSleepDuration, minSleepDuration) + 1;
+      newSleepDuration = Math.min(newSleepDuration, maxSleepDuration);
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(String.format(
+          "%5.5s, %10d, %10d, %10d, %10d, %6.2f, %5d, %5d, %5d",
+          name,
+          (int) bytesFailed,
+          (int) bytesSuccessful,
+          (int) operationsFailed,
+          (int) operationsSuccessful,
+          errorPercentage,
+          periodMs,
+          (int) sleepDuration,
+          (int) newSleepDuration));
+    }
+
+    return (int) newSleepDuration;
+  }
+
+  /**
+   * Timer callback implementation for periodically analyzing metrics.
+   */
+  class TimerTaskImpl extends TimerTask {
+    private AtomicInteger doingWork = new AtomicInteger(0);
+
+    /**
+     * Periodically analyzes a snapshot of the blob storage metrics and updates
+     * the sleepDuration in order to appropriately throttle storage operations.
+     */
+    @Override
+    public void run() {
+      boolean doWork = false;
+      try {
+        doWork = doingWork.compareAndSet(0, 1);
+
+        // prevent concurrent execution of this task
+        if (!doWork) {
+          return;
+        }
+
+        long now = System.currentTimeMillis();
+        if (now - blobMetrics.get().startTime >= analysisPeriodMs) {
+          AbfsOperationMetrics oldMetrics = blobMetrics.getAndSet(
+              new AbfsOperationMetrics(now));
+          oldMetrics.endTime = now;
+          sleepDuration = analyzeMetricsAndUpdateSleepDuration(oldMetrics,
+              sleepDuration);
+        }
+      } finally {
+        if (doWork) {
+          doingWork.set(0);
+        }
+      }
+    }
+  }
+
+  /**
+   * Stores Abfs operation metrics during each analysis period.
+   */
+  static class AbfsOperationMetrics {
+    private AtomicLong bytesFailed;
+    private AtomicLong bytesSuccessful;
+    private AtomicLong operationsFailed;
+    private AtomicLong operationsSuccessful;
+    private long endTime;
+    private long startTime;
+
+    AbfsOperationMetrics(long startTime) {
+      this.startTime = startTime;
+      this.bytesFailed = new AtomicLong();
+      this.bytesSuccessful = new AtomicLong();
+      this.operationsFailed = new AtomicLong();
+      this.operationsSuccessful = new AtomicLong();
+    }
+  }
+}

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

@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.net.HttpURLConnection;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+
+/**
+ * Throttles Azure Blob File System read and write operations to achieve maximum
+ * throughput by minimizing errors.  The errors occur when the account ingress
+ * or egress limits are exceeded and the server-side throttles requests.
+ * Server-side throttling causes the retry policy to be used, but the retry
+ * policy sleeps for long periods of time causing the total ingress or egress
+ * throughput to be as much as 35% lower than optimal.  The retry policy is also
+ * after the fact, in that it applies after a request fails.  On the other hand,
+ * the client-side throttling implemented here happens before requests are made
+ * and sleeps just enough to minimize errors, allowing optimal ingress and/or
+ * egress throughput.
+ */
+public final class AbfsClientThrottlingIntercept {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbfsClientThrottlingIntercept.class);
+  private static final String RANGE_PREFIX = "bytes=";
+  private static AbfsClientThrottlingIntercept singleton = null;
+  private AbfsClientThrottlingAnalyzer readThrottler = null;
+  private AbfsClientThrottlingAnalyzer writeThrottler = null;
+  private static boolean isAutoThrottlingEnabled = false;
+
+  // Hide default constructor
+  private AbfsClientThrottlingIntercept() {
+    readThrottler = new AbfsClientThrottlingAnalyzer("read");
+    writeThrottler = new AbfsClientThrottlingAnalyzer("write");
+  }
+
+  public static synchronized void initializeSingleton(boolean enableAutoThrottling) {
+    if (!enableAutoThrottling) {
+      return;
+    }
+    if (singleton == null) {
+      singleton = new AbfsClientThrottlingIntercept();
+      isAutoThrottlingEnabled = true;
+      LOG.debug("Client-side throttling is enabled for the ABFS file system.");
+    }
+  }
+
+  static void updateMetrics(AbfsRestOperationType operationType,
+                            AbfsHttpOperation abfsHttpOperation) {
+    if (!isAutoThrottlingEnabled || abfsHttpOperation == null) {
+      return;
+    }
+
+    int status = abfsHttpOperation.getStatusCode();
+    long contentLength = 0;
+    // If the socket is terminated prior to receiving a response, the HTTP
+    // status may be 0 or -1.  A status less than 200 or greater than or equal
+    // to 500 is considered an error.
+    boolean isFailedOperation = (status < HttpURLConnection.HTTP_OK
+        || status >= HttpURLConnection.HTTP_INTERNAL_ERROR);
+
+    switch (operationType) {
+      case Append:
+        contentLength = abfsHttpOperation.getBytesSent();
+        if (contentLength > 0) {
+          singleton.writeThrottler.addBytesTransferred(contentLength,
+              isFailedOperation);
+        }
+        break;
+      case ReadFile:
+        String range = abfsHttpOperation.getConnection().getRequestProperty(HttpHeaderConfigurations.RANGE);
+        contentLength = getContentLengthIfKnown(range);
+        if (contentLength > 0) {
+          singleton.readThrottler.addBytesTransferred(contentLength,
+              isFailedOperation);
+        }
+        break;
+      default:
+        break;
+    }
+  }
+
+  /**
+   * Called before the request is sent.  Client-side throttling
+   * uses this to suspend the request, if necessary, to minimize errors and
+   * maximize throughput.
+   */
+  static void sendingRequest(AbfsRestOperationType operationType) {
+    if (!isAutoThrottlingEnabled) {
+      return;
+    }
+
+    switch (operationType) {
+      case ReadFile:
+        singleton.readThrottler.suspendIfNecessary();
+        break;
+      case Append:
+        singleton.writeThrottler.suspendIfNecessary();
+        break;
+      default:
+        break;
+    }
+  }
+
+  private static long getContentLengthIfKnown(String range) {
+    long contentLength = 0;
+    // Format is "bytes=%d-%d"
+    if (range != null && range.startsWith(RANGE_PREFIX)) {
+      String[] offsets = range.substring(RANGE_PREFIX.length()).split("-");
+      if (offsets.length == 2) {
+        contentLength = Long.parseLong(offsets[1]) - Long.parseLong(offsets[0])
+                + 1;
+      }
+    }
+    return contentLength;
+  }
+}

+ 40 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+
+/**
+ * The Http Request / Response Headers for Rest AbfsClient.
+ */
+public class AbfsHttpHeader {
+  private final String name;
+  private final String value;
+
+  public AbfsHttpHeader(final String name, final String value) {
+    this.name = name;
+    this.value = value;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public String getValue() {
+    return value;
+  }
+}

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

@@ -0,0 +1,446 @@
+/**
+ * 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.URL;
+import java.util.List;
+import java.util.UUID;
+
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+
+import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.JsonToken;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema;
+
+/**
+ * Represents an HTTP operation.
+ */
+public class AbfsHttpOperation {
+  private static final Logger LOG = LoggerFactory.getLogger(AbfsHttpOperation.class);
+
+  private static final int CONNECT_TIMEOUT = 30 * 1000;
+  private static final int READ_TIMEOUT = 30 * 1000;
+
+  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 HttpURLConnection connection;
+  private int statusCode;
+  private String statusDescription;
+  private String storageErrorCode = "";
+  private String storageErrorMessage  = "";
+  private String clientRequestId = "";
+  private String requestId  = "";
+  private ListResultSchema listResultSchema = null;
+
+  // metrics
+  private int bytesSent;
+  private long bytesReceived;
+
+  // optional trace enabled metrics
+  private final boolean isTraceEnabled;
+  private long connectionTimeMs;
+  private long sendRequestTimeMs;
+  private long recvResponseTimeMs;
+
+  protected  HttpURLConnection getConnection() {
+    return connection;
+  }
+
+  public String getMethod() {
+    return method;
+  }
+
+  public URL getUrl() {
+    return url;
+  }
+
+  public int getStatusCode() {
+    return statusCode;
+  }
+
+  public String getStatusDescription() {
+    return statusDescription;
+  }
+
+  public String getStorageErrorCode() {
+    return storageErrorCode;
+  }
+
+  public String getStorageErrorMessage() {
+    return storageErrorMessage;
+  }
+
+  public String getClientRequestId() {
+    return clientRequestId;
+  }
+
+  public String getRequestId() {
+    return requestId;
+  }
+
+  public int getBytesSent() {
+    return bytesSent;
+  }
+
+  public long getBytesReceived() {
+    return bytesReceived;
+  }
+
+  public ListResultSchema getListResultSchema() {
+    return listResultSchema;
+  }
+
+  public String getResponseHeader(String httpHeader) {
+    return connection.getHeaderField(httpHeader);
+  }
+
+  // Returns a trace message for the request
+  @Override
+  public String toString() {
+    final String urlStr = url.toString();
+    final StringBuilder sb = new StringBuilder();
+    sb.append(statusCode);
+    sb.append(",");
+    sb.append(storageErrorCode);
+    sb.append(",cid=");
+    sb.append(clientRequestId);
+    sb.append(",rid=");
+    sb.append(requestId);
+    if (isTraceEnabled) {
+      sb.append(",connMs=");
+      sb.append(connectionTimeMs);
+      sb.append(",sendMs=");
+      sb.append(sendRequestTimeMs);
+      sb.append(",recvMs=");
+      sb.append(recvResponseTimeMs);
+    }
+    sb.append(",sent=");
+    sb.append(bytesSent);
+    sb.append(",recv=");
+    sb.append(bytesReceived);
+    sb.append(",");
+    sb.append(method);
+    sb.append(",");
+    sb.append(urlStr);
+    return sb.toString();
+  }
+
+  /**
+   * 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
+   *
+   * @throws IOException if an error occurs.
+   */
+  public AbfsHttpOperation(final URL url, final String method, final List<AbfsHttpHeader> requestHeaders)
+      throws IOException {
+    this.isTraceEnabled = LOG.isTraceEnabled();
+    this.url = url;
+    this.method = method;
+    this.clientRequestId = UUID.randomUUID().toString();
+
+    this.connection = openConnection();
+    if (this.connection instanceof HttpsURLConnection) {
+      HttpsURLConnection secureConn = (HttpsURLConnection) this.connection;
+      SSLSocketFactory sslSocketFactory = SSLSocketFactoryEx.getDefaultFactory();
+      if (sslSocketFactory != null) {
+        secureConn.setSSLSocketFactory(sslSocketFactory);
+      }
+    }
+
+    this.connection.setConnectTimeout(CONNECT_TIMEOUT);
+    this.connection.setReadTimeout(READ_TIMEOUT);
+
+    this.connection.setRequestMethod(method);
+
+    for (AbfsHttpHeader header : requestHeaders) {
+      this.connection.setRequestProperty(header.getName(), header.getValue());
+    }
+
+    this.connection.setRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID, clientRequestId);
+  }
+
+   /**
+   * 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.
+   *
+   * @param buffer the request entity body.
+   * @param offset an offset into the buffer where the data beings.
+   * @param length the length of the data in the buffer.
+   *
+   * @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;
+    if (this.isTraceEnabled) {
+      startTime = System.nanoTime();
+    }
+    try (OutputStream outputStream = this.connection.getOutputStream()) {
+      // update bytes sent before they are sent so we may observe
+      // attempted sends as well as successful sends via the
+      // accompanying statusCode
+      this.bytesSent = length;
+      outputStream.write(buffer, offset, length);
+    } finally {
+      if (this.isTraceEnabled) {
+        this.sendRequestTimeMs = elapsedTimeMs(startTime);
+      }
+    }
+  }
+
+  /**
+   * Gets and processes the HTTP response.
+   *
+   * @param buffer a buffer to hold the response entity body
+   * @param offset an offset in the buffer where the data will being.
+   * @param length the number of bytes to be written to the buffer.
+   *
+   * @throws IOException if an error occurs.
+   */
+  public void processResponse(final byte[] buffer, final int offset, final int length) throws IOException {
+
+    // get the response
+    long startTime = 0;
+    if (this.isTraceEnabled) {
+      startTime = System.nanoTime();
+    }
+
+    this.statusCode = this.connection.getResponseCode();
+
+    if (this.isTraceEnabled) {
+      this.recvResponseTimeMs = elapsedTimeMs(startTime);
+    }
+
+    this.statusDescription = this.connection.getResponseMessage();
+
+    this.requestId = this.connection.getHeaderField(HttpHeaderConfigurations.X_MS_REQUEST_ID);
+    if (this.requestId == null) {
+      this.requestId = AbfsHttpConstants.EMPTY_STRING;
+    }
+
+    if (AbfsHttpConstants.HTTP_METHOD_HEAD.equals(this.method)) {
+      // If it is HEAD, and it is ERROR
+      return;
+    }
+
+    if (this.isTraceEnabled) {
+      startTime = System.nanoTime();
+    }
+
+    if (statusCode >= HttpURLConnection.HTTP_BAD_REQUEST) {
+      processStorageErrorResponse();
+      if (this.isTraceEnabled) {
+        this.recvResponseTimeMs += elapsedTimeMs(startTime);
+      }
+      this.bytesReceived = this.connection.getHeaderFieldLong(HttpHeaderConfigurations.CONTENT_LENGTH, 0);
+    } else {
+      // consume the input stream to release resources
+      int totalBytesRead = 0;
+
+      try (InputStream stream = this.connection.getInputStream()) {
+        if (isNullInputStream(stream)) {
+          return;
+        }
+        boolean endOfStream = false;
+
+        // this is a list operation and need to retrieve the data
+        // need a better solution
+        if (AbfsHttpConstants.HTTP_METHOD_GET.equals(this.method) && buffer == null) {
+          parseListFilesResponse(stream);
+        } else {
+          if (buffer != null) {
+            while (totalBytesRead < length) {
+              int bytesRead = stream.read(buffer, offset + totalBytesRead, length - totalBytesRead);
+              if (bytesRead == -1) {
+                endOfStream = true;
+                break;
+              }
+              totalBytesRead += bytesRead;
+            }
+          }
+          if (!endOfStream && stream.read() != -1) {
+            // read and discard
+            int bytesRead = 0;
+            byte[] b = new byte[CLEAN_UP_BUFFER_SIZE];
+            while ((bytesRead = stream.read(b)) >= 0) {
+              totalBytesRead += bytesRead;
+            }
+          }
+        }
+      } catch (IOException ex) {
+        LOG.error("UnexpectedError: ", ex);
+        throw ex;
+      } finally {
+        if (this.isTraceEnabled) {
+          this.recvResponseTimeMs += elapsedTimeMs(startTime);
+        }
+        this.bytesReceived = totalBytesRead;
+      }
+    }
+  }
+
+
+  /**
+   * Open the HTTP connection.
+   *
+   * @throws IOException if an error occurs.
+   */
+  private HttpURLConnection openConnection() throws IOException {
+    if (!isTraceEnabled) {
+      return (HttpURLConnection) url.openConnection();
+    }
+    long start = System.nanoTime();
+    try {
+      return (HttpURLConnection) url.openConnection();
+    } finally {
+      connectionTimeMs = elapsedTimeMs(start);
+    }
+  }
+
+  /**
+   * When the request fails, this function is used to parse the responseAbfsHttpClient.LOG.debug("ExpectedError: ", ex);
+   * and extract the storageErrorCode and storageErrorMessage.  Any errors
+   * encountered while attempting to process the error response are logged,
+   * but otherwise ignored.
+   *
+   * For storage errors, the response body *usually* has the following format:
+   *
+   * {
+   *   "error":
+   *   {
+   *     "code": "string",
+   *     "message": "string"
+   *   }
+   * }
+   *
+   */
+  private void processStorageErrorResponse() {
+    try (InputStream stream = connection.getErrorStream()) {
+      if (stream == null) {
+        return;
+      }
+      JsonFactory jf = new JsonFactory();
+      try (JsonParser jp = jf.createJsonParser(stream)) {
+        String fieldName, fieldValue;
+        jp.nextToken();  // START_OBJECT - {
+        jp.nextToken();  // FIELD_NAME - "error":
+        jp.nextToken();  // START_OBJECT - {
+        jp.nextToken();
+        while (jp.hasCurrentToken()) {
+          if (jp.getCurrentToken() == JsonToken.FIELD_NAME) {
+            fieldName = jp.getCurrentName();
+            jp.nextToken();
+            fieldValue = jp.getText();
+            switch (fieldName) {
+              case "code":
+                storageErrorCode = fieldValue;
+                break;
+              case "message":
+                storageErrorMessage = fieldValue;
+                break;
+              default:
+                break;
+            }
+          }
+          jp.nextToken();
+        }
+      }
+    } catch (IOException ex) {
+      // 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);
+    }
+  }
+
+  /**
+   * Returns the elapsed time in milliseconds.
+   */
+  private long elapsedTimeMs(final long startTime) {
+    return (System.nanoTime() - startTime) / ONE_MILLION;
+  }
+
+  /**
+   * Parse the list file response
+   *
+   * @param stream InputStream contains the list results.
+   * @throws IOException
+   */
+  private void parseListFilesResponse(final InputStream stream) throws IOException {
+    if (stream == null) {
+      return;
+    }
+
+    if (listResultSchema != null) {
+      // already parse the response
+      return;
+    }
+
+    try {
+      final ObjectMapper objectMapper = new ObjectMapper();
+      this.listResultSchema = objectMapper.readValue(stream, ListResultSchema.class);
+    } catch (IOException ex) {
+      LOG.error("Unable to deserialize list results", ex);
+      throw ex;
+    }
+  }
+
+  /**
+   * Check null stream, this is to pass findbugs's redundant check for NULL
+   * @param stream InputStream
+   */
+  private boolean isNullInputStream(InputStream stream) {
+    return stream == null ? true : false;
+  }
+}

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

@@ -0,0 +1,381 @@
+/**
+ * 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.EOFException;
+import java.io.IOException;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileSystem.Statistics;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+
+/**
+ * The AbfsInputStream for AbfsClient.
+ */
+public class AbfsInputStream extends FSInputStream {
+  private final AbfsClient client;
+  private final Statistics statistics;
+  private final String path;
+  private final long contentLength;
+  private final int bufferSize; // default buffer size
+  private final int readAheadQueueDepth;         // initialized in constructor
+  private final String eTag;                  // eTag of the path when InputStream are created
+  private final boolean tolerateOobAppends; // whether tolerate Oob Appends
+  private final boolean readAheadEnabled; // whether enable readAhead;
+
+  private byte[] buffer = null;            // will be initialized on first use
+
+  private long fCursor = 0;  // cursor of buffer within file - offset of next byte to read from remote server
+  private long fCursorAfterLastRead = -1;
+  private int bCursor = 0;   // cursor of read within buffer - offset of next byte to be returned from buffer
+  private int limit = 0;     // offset of next byte to be read into buffer from service (i.e., upper marker+1
+  //                                                      of valid bytes in buffer)
+  private boolean closed = false;
+
+  public AbfsInputStream(
+      final AbfsClient client,
+      final Statistics statistics,
+      final String path,
+      final long contentLength,
+      final int bufferSize,
+      final int readAheadQueueDepth,
+      final String eTag) {
+    this.client = client;
+    this.statistics = statistics;
+    this.path = path;
+    this.contentLength = contentLength;
+    this.bufferSize = bufferSize;
+    this.readAheadQueueDepth = (readAheadQueueDepth >= 0) ? readAheadQueueDepth : Runtime.getRuntime().availableProcessors();
+    this.eTag = eTag;
+    this.tolerateOobAppends = false;
+    this.readAheadEnabled = true;
+  }
+
+  public String getPath() {
+    return path;
+  }
+
+  @Override
+  public int read() throws IOException {
+    byte[] b = new byte[1];
+    int numberOfBytesRead = read(b, 0, 1);
+    if (numberOfBytesRead < 0) {
+      return -1;
+    } else {
+      return (b[0] & 0xFF);
+    }
+  }
+
+  @Override
+  public synchronized int read(final byte[] b, final int off, final int len) throws IOException {
+    int currentOff = off;
+    int currentLen = len;
+    int lastReadBytes;
+    int totalReadBytes = 0;
+    do {
+      lastReadBytes = readOneBlock(b, currentOff, currentLen);
+      if (lastReadBytes > 0) {
+        currentOff += lastReadBytes;
+        currentLen -= lastReadBytes;
+        totalReadBytes += lastReadBytes;
+      }
+      if (currentLen <= 0 || currentLen > b.length - currentOff) {
+        break;
+      }
+    } while (lastReadBytes > 0);
+    return totalReadBytes > 0 ? totalReadBytes : lastReadBytes;
+  }
+
+  private int readOneBlock(final byte[] b, final int off, final int len) throws IOException {
+    if (closed) {
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+    }
+
+    Preconditions.checkNotNull(b);
+
+    if (len == 0) {
+      return 0;
+    }
+
+    if (this.available() == 0) {
+      return -1;
+    }
+
+    if (off < 0 || len < 0 || len > b.length - off) {
+      throw new IndexOutOfBoundsException();
+    }
+
+    //If buffer is empty, then fill the buffer.
+    if (bCursor == limit) {
+      //If EOF, then return -1
+      if (fCursor >= contentLength) {
+        return -1;
+      }
+
+      long bytesRead = 0;
+      //reset buffer to initial state - i.e., throw away existing data
+      bCursor = 0;
+      limit = 0;
+      if (buffer == null) {
+        buffer = new byte[bufferSize];
+      }
+
+      // Enable readAhead when reading sequentially
+      if (-1 == fCursorAfterLastRead || fCursorAfterLastRead == fCursor || b.length >= bufferSize) {
+        bytesRead = readInternal(fCursor, buffer, 0, bufferSize, false);
+      } else {
+        bytesRead = readInternal(fCursor, buffer, 0, b.length, true);
+      }
+
+      if (bytesRead == -1) {
+        return -1;
+      }
+
+      limit += bytesRead;
+      fCursor += bytesRead;
+      fCursorAfterLastRead = fCursor;
+    }
+
+    //If there is anything in the buffer, then return lesser of (requested bytes) and (bytes in buffer)
+    //(bytes returned may be less than requested)
+    int bytesRemaining = limit - bCursor;
+    int bytesToRead = Math.min(len, bytesRemaining);
+    System.arraycopy(buffer, bCursor, b, off, bytesToRead);
+    bCursor += bytesToRead;
+    if (statistics != null) {
+      statistics.incrementBytesRead(bytesToRead);
+    }
+    return bytesToRead;
+  }
+
+
+  private int readInternal(final long position, final byte[] b, final int offset, final int length,
+                           final boolean bypassReadAhead) throws IOException {
+    if (readAheadEnabled && !bypassReadAhead) {
+      // try reading from read-ahead
+      if (offset != 0) {
+        throw new IllegalArgumentException("readahead buffers cannot have non-zero buffer offsets");
+      }
+      int receivedBytes;
+
+      // queue read-aheads
+      int numReadAheads = this.readAheadQueueDepth;
+      long nextSize;
+      long nextOffset = position;
+      while (numReadAheads > 0 && nextOffset < contentLength) {
+        nextSize = Math.min((long) bufferSize, contentLength - nextOffset);
+        ReadBufferManager.getBufferManager().queueReadAhead(this, nextOffset, (int) nextSize);
+        nextOffset = nextOffset + nextSize;
+        numReadAheads--;
+      }
+
+      // try reading from buffers first
+      receivedBytes = ReadBufferManager.getBufferManager().getBlock(this, position, length, b);
+      if (receivedBytes > 0) {
+        return receivedBytes;
+      }
+
+      // got nothing from read-ahead, do our own read now
+      receivedBytes = readRemote(position, b, offset, length);
+      return receivedBytes;
+    } else {
+      return readRemote(position, b, offset, length);
+    }
+  }
+
+  int readRemote(long position, byte[] b, int offset, int length) throws IOException {
+    if (position < 0) {
+      throw new IllegalArgumentException("attempting to read from negative offset");
+    }
+    if (position >= contentLength) {
+      return -1;  // Hadoop prefers -1 to EOFException
+    }
+    if (b == null) {
+      throw new IllegalArgumentException("null byte array passed in to read() method");
+    }
+    if (offset >= b.length) {
+      throw new IllegalArgumentException("offset greater than length of array");
+    }
+    if (length < 0) {
+      throw new IllegalArgumentException("requested read length is less than zero");
+    }
+    if (length > (b.length - offset)) {
+      throw new IllegalArgumentException("requested read length is more than will fit after requested offset in buffer");
+    }
+    final AbfsRestOperation op;
+    try {
+      op = client.read(path, position, b, offset, length, tolerateOobAppends ? "*" : eTag);
+    } catch (AzureBlobFileSystemException ex) {
+      throw new IOException(ex);
+    }
+    long bytesRead = op.getResult().getBytesReceived();
+    if (bytesRead > Integer.MAX_VALUE) {
+      throw new IOException("Unexpected Content-Length");
+    }
+    return (int) bytesRead;
+  }
+
+  /**
+   * Seek to given position in stream.
+   * @param n position to seek to
+   * @throws IOException if there is an error
+   * @throws EOFException if attempting to seek past end of file
+   */
+  @Override
+  public synchronized void seek(long n) throws IOException {
+    if (closed) {
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+    }
+    if (n < 0) {
+      throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK);
+    }
+    if (n > contentLength) {
+      throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
+    }
+
+    if (n>=fCursor-limit && n<=fCursor) { // within buffer
+      bCursor = (int) (n-(fCursor-limit));
+      return;
+    }
+
+    // next read will read from here
+    fCursor = n;
+
+    //invalidate buffer
+    limit = 0;
+    bCursor = 0;
+  }
+
+  @Override
+  public synchronized long skip(long n) throws IOException {
+    if (closed) {
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+    }
+    long currentPos = getPos();
+    if (currentPos == contentLength) {
+      if (n > 0) {
+        throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
+      }
+    }
+    long newPos = currentPos + n;
+    if (newPos < 0) {
+      newPos = 0;
+      n = newPos - currentPos;
+    }
+    if (newPos > contentLength) {
+      newPos = contentLength;
+      n = newPos - currentPos;
+    }
+    seek(newPos);
+    return n;
+  }
+
+  /**
+   * Return the size of the remaining available bytes
+   * if the size is less than or equal to {@link Integer#MAX_VALUE},
+   * otherwise, return {@link Integer#MAX_VALUE}.
+   *
+   * This is to match the behavior of DFSInputStream.available(),
+   * which some clients may rely on (HBase write-ahead log reading in
+   * particular).
+   */
+  @Override
+  public synchronized int available() throws IOException {
+    if (closed) {
+      throw new IOException(
+          FSExceptionMessages.STREAM_IS_CLOSED);
+    }
+    final long remaining = this.contentLength - this.getPos();
+    return remaining <= Integer.MAX_VALUE
+        ? (int) remaining : Integer.MAX_VALUE;
+  }
+
+  /**
+   * Returns the length of the file that this stream refers to. Note that the length returned is the length
+   * as of the time the Stream was opened. Specifically, if there have been subsequent appends to the file,
+   * they wont be reflected in the returned length.
+   *
+   * @return length of the file.
+   * @throws IOException if the stream is closed
+   */
+  public long length() throws IOException {
+    if (closed) {
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+    }
+    return contentLength;
+  }
+
+  /**
+   * Return the current offset from the start of the file
+   * @throws IOException throws {@link IOException} if there is an error
+   */
+  @Override
+  public synchronized long getPos() throws IOException {
+    if (closed) {
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+    }
+    return fCursor - limit + bCursor;
+  }
+
+  /**
+   * Seeks a different copy of the data.  Returns true if
+   * found a new source, false otherwise.
+   * @throws IOException throws {@link IOException} if there is an error
+   */
+  @Override
+  public boolean seekToNewSource(long l) throws IOException {
+    return false;
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    closed = true;
+    buffer = null; // de-reference the buffer so it can be GC'ed sooner
+  }
+
+  /**
+   * Not supported by this stream. Throws {@link UnsupportedOperationException}
+   * @param readlimit ignored
+   */
+  @Override
+  public synchronized void mark(int readlimit) {
+    throw new UnsupportedOperationException("mark()/reset() not supported on this stream");
+  }
+
+  /**
+   * Not supported by this stream. Throws {@link UnsupportedOperationException}
+   */
+  @Override
+  public synchronized void reset() throws IOException {
+    throw new UnsupportedOperationException("mark()/reset() not supported on this stream");
+  }
+
+  /**
+   * gets whether mark and reset are supported by {@code ADLFileInputStream}. Always returns false.
+   *
+   * @return always {@code false}
+   */
+  @Override
+  public boolean markSupported() {
+    return false;
+  }
+}

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

@@ -0,0 +1,378 @@
+/**
+ * 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.InterruptedIOException;
+import java.io.OutputStream;
+import java.util.Locale;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.StreamCapabilities;
+import org.apache.hadoop.fs.Syncable;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+
+/**
+ * The BlobFsOutputStream for Rest AbfsClient.
+ */
+public class AbfsOutputStream extends OutputStream implements Syncable, StreamCapabilities {
+  private final AbfsClient client;
+  private final String path;
+  private long position;
+  private boolean closed;
+  private boolean supportFlush;
+  private volatile IOException lastError;
+
+  private long lastFlushOffset;
+  private long lastTotalAppendOffset = 0;
+
+  private final int bufferSize;
+  private byte[] buffer;
+  private int bufferIndex;
+  private final int maxConcurrentRequestCount;
+
+  private ConcurrentLinkedDeque<WriteOperation> writeOperations;
+  private final ThreadPoolExecutor threadExecutor;
+  private final ExecutorCompletionService<Void> completionService;
+
+  public AbfsOutputStream(
+      final AbfsClient client,
+      final String path,
+      final long position,
+      final int bufferSize,
+      final boolean supportFlush) {
+    this.client = client;
+    this.path = path;
+    this.position = position;
+    this.closed = false;
+    this.supportFlush = supportFlush;
+    this.lastError = null;
+    this.lastFlushOffset = 0;
+    this.bufferSize = bufferSize;
+    this.buffer = new byte[bufferSize];
+    this.bufferIndex = 0;
+    this.writeOperations = new ConcurrentLinkedDeque<>();
+
+    this.maxConcurrentRequestCount = 4 * Runtime.getRuntime().availableProcessors();
+
+    this.threadExecutor
+        = new ThreadPoolExecutor(maxConcurrentRequestCount,
+        maxConcurrentRequestCount,
+        10L,
+        TimeUnit.SECONDS,
+        new LinkedBlockingQueue<>());
+    this.completionService = new ExecutorCompletionService<>(this.threadExecutor);
+  }
+
+  /**
+   * Query the stream for a specific capability.
+   *
+   * @param capability string to query the stream support for.
+   * @return true for hsync and hflush.
+   */
+  @Override
+  public boolean hasCapability(String capability) {
+    switch (capability.toLowerCase(Locale.ENGLISH)) {
+      case StreamCapabilities.HSYNC:
+      case StreamCapabilities.HFLUSH:
+        return supportFlush;
+      default:
+        return false;
+    }
+  }
+
+  /**
+   * Writes the specified byte to this output stream. The general contract for
+   * write is that one byte is written to the output stream. The byte to be
+   * written is the eight low-order bits of the argument b. The 24 high-order
+   * bits of b are ignored.
+   *
+   * @param byteVal the byteValue to write.
+   * @throws IOException if an I/O error occurs. In particular, an IOException may be
+   *                     thrown if the output stream has been closed.
+   */
+  @Override
+  public void write(final int byteVal) throws IOException {
+    write(new byte[]{(byte) (byteVal & 0xFF)});
+  }
+
+  /**
+   * Writes length bytes from the specified byte array starting at off to
+   * this output stream.
+   *
+   * @param data   the byte array to write.
+   * @param off the start off in the data.
+   * @param length the number of bytes to write.
+   * @throws IOException if an I/O error occurs. In particular, an IOException may be
+   *                     thrown if the output stream has been closed.
+   */
+  @Override
+  public synchronized void write(final byte[] data, final int off, final int length)
+      throws IOException {
+    maybeThrowLastError();
+
+    Preconditions.checkArgument(data != null, "null data");
+
+    if (off < 0 || length < 0 || length > data.length - off) {
+      throw new IndexOutOfBoundsException();
+    }
+
+    int currentOffset = off;
+    int writableBytes = bufferSize - bufferIndex;
+    int numberOfBytesToWrite = length;
+
+    while (numberOfBytesToWrite > 0) {
+      if (writableBytes <= numberOfBytesToWrite) {
+        System.arraycopy(data, currentOffset, buffer, bufferIndex, writableBytes);
+        bufferIndex += writableBytes;
+        writeCurrentBufferToService();
+        currentOffset += writableBytes;
+        numberOfBytesToWrite = numberOfBytesToWrite - writableBytes;
+      } else {
+        System.arraycopy(data, currentOffset, buffer, bufferIndex, numberOfBytesToWrite);
+        bufferIndex += numberOfBytesToWrite;
+        numberOfBytesToWrite = 0;
+      }
+
+      writableBytes = bufferSize - bufferIndex;
+    }
+  }
+
+  /**
+   * Throw the last error recorded if not null.
+   * After the stream is closed, this is always set to
+   * an exception, so acts as a guard against method invocation once
+   * closed.
+   * @throws IOException if lastError is set
+   */
+  private void maybeThrowLastError() throws IOException {
+    if (lastError != null) {
+      throw lastError;
+    }
+  }
+
+  /**
+   * Flushes this output stream and forces any buffered output bytes to be
+   * written out. If any data remains in the payload it is committed to the
+   * service. Data is queued for writing and forced out to the service
+   * before the call returns.
+   */
+  @Override
+  public void flush() throws IOException {
+    if (supportFlush) {
+      flushInternalAsync();
+    }
+  }
+
+  /** Similar to posix fsync, flush out the data in client's user buffer
+   * all the way to the disk device (but the disk may have it in its cache).
+   * @throws IOException if error occurs
+   */
+  @Override
+  public void hsync() throws IOException {
+    if (supportFlush) {
+      flushInternal();
+    }
+  }
+
+  /** Flush out the data in client's user buffer. After the return of
+   * this call, new readers will see the data.
+   * @throws IOException if any error occurs
+   */
+  @Override
+  public void hflush() throws IOException {
+    if (supportFlush) {
+      flushInternal();
+    }
+  }
+
+  /**
+   * Force all data in the output stream to be written to Azure storage.
+   * Wait to return until this is complete. Close the access to the stream and
+   * shutdown the upload thread pool.
+   * If the blob was created, its lease will be released.
+   * Any error encountered caught in threads and stored will be rethrown here
+   * after cleanup.
+   */
+  @Override
+  public synchronized void close() throws IOException {
+    if (closed) {
+      return;
+    }
+
+    try {
+      flushInternal();
+      threadExecutor.shutdown();
+    } finally {
+      lastError = new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
+      buffer = null;
+      bufferIndex = 0;
+      closed = true;
+      writeOperations.clear();
+      if (!threadExecutor.isShutdown()) {
+        threadExecutor.shutdownNow();
+      }
+    }
+  }
+
+  private synchronized void flushInternal() throws IOException {
+    maybeThrowLastError();
+    writeCurrentBufferToService();
+    flushWrittenBytesToService();
+  }
+
+  private synchronized void flushInternalAsync() throws IOException {
+    maybeThrowLastError();
+    writeCurrentBufferToService();
+    flushWrittenBytesToServiceAsync();
+  }
+
+  private synchronized void writeCurrentBufferToService() throws IOException {
+    if (bufferIndex == 0) {
+      return;
+    }
+
+    final byte[] bytes = buffer;
+    final int bytesLength = bufferIndex;
+
+    buffer = new byte[bufferSize];
+    bufferIndex = 0;
+    final long offset = position;
+    position += bytesLength;
+
+    if (threadExecutor.getQueue().size() >= maxConcurrentRequestCount * 2) {
+      waitForTaskToComplete();
+    }
+
+    final Future<Void> job = completionService.submit(new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        client.append(path, offset, bytes, 0,
+            bytesLength);
+        return null;
+      }
+    });
+
+    writeOperations.add(new WriteOperation(job, offset, bytesLength));
+
+    // Try to shrink the queue
+    shrinkWriteOperationQueue();
+  }
+
+  private synchronized void flushWrittenBytesToService() throws IOException {
+    for (WriteOperation writeOperation : writeOperations) {
+      try {
+        writeOperation.task.get();
+      } catch (Exception ex) {
+        if (ex.getCause() instanceof AzureBlobFileSystemException) {
+          ex = (AzureBlobFileSystemException) ex.getCause();
+        }
+        lastError = new IOException(ex);
+        throw lastError;
+      }
+    }
+    flushWrittenBytesToServiceInternal(position, false);
+  }
+
+  private synchronized void flushWrittenBytesToServiceAsync() throws IOException {
+    shrinkWriteOperationQueue();
+
+    if (this.lastTotalAppendOffset > this.lastFlushOffset) {
+      this.flushWrittenBytesToServiceInternal(this.lastTotalAppendOffset, true);
+    }
+  }
+
+  private synchronized void flushWrittenBytesToServiceInternal(final long offset,
+      final boolean retainUncommitedData) throws IOException {
+    try {
+      client.flush(path, offset, retainUncommitedData);
+    } catch (AzureBlobFileSystemException ex) {
+      throw new IOException(ex);
+    }
+    this.lastFlushOffset = offset;
+  }
+
+  /**
+   * Try to remove the completed write operations from the beginning of write
+   * operation FIFO queue.
+   */
+  private synchronized void shrinkWriteOperationQueue() throws IOException {
+    try {
+      while (writeOperations.peek() != null && writeOperations.peek().task.isDone()) {
+        writeOperations.peek().task.get();
+        lastTotalAppendOffset += writeOperations.peek().length;
+        writeOperations.remove();
+      }
+    } catch (Exception e) {
+      if (e.getCause() instanceof AzureBlobFileSystemException) {
+        lastError = (AzureBlobFileSystemException) e.getCause();
+      } else {
+        lastError = new IOException(e);
+      }
+      throw lastError;
+    }
+  }
+
+  private void waitForTaskToComplete() throws IOException {
+    boolean completed;
+    for (completed = false; completionService.poll() != null; completed = true) {
+      // keep polling until there is no data
+    }
+
+    if (!completed) {
+      try {
+        completionService.take();
+      } catch (InterruptedException e) {
+        lastError = (IOException) new InterruptedIOException(e.toString()).initCause(e);
+        throw lastError;
+      }
+    }
+  }
+
+  private static class WriteOperation {
+    private final Future<Void> task;
+    private final long startOffset;
+    private final long length;
+
+    WriteOperation(final Future<Void> task, final long startOffset, final long length) {
+      Preconditions.checkNotNull(task, "task");
+      Preconditions.checkArgument(startOffset >= 0, "startOffset");
+      Preconditions.checkArgument(length >= 0, "length");
+
+      this.task = task;
+      this.startOffset = startOffset;
+      this.length = length;
+    }
+  }
+
+  @VisibleForTesting
+  public synchronized void waitForPendingUploads() throws IOException {
+    waitForTaskToComplete();
+  }
+}

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

@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * The AbfsPermission for AbfsClient.
+ */
+public class AbfsPermission extends FsPermission {
+  private static final int STICKY_BIT_OCTAL_VALUE = 01000;
+  private final boolean aclBit;
+
+  public AbfsPermission(Short aShort, boolean aclBitStatus) {
+    super(aShort);
+    this.aclBit = aclBitStatus;
+  }
+
+  public AbfsPermission(FsAction u, FsAction g, FsAction o) {
+    super(u, g, o, false);
+    this.aclBit = false;
+  }
+
+  /**
+   * Returns true if there is also an ACL (access control list).
+   *
+   * @return boolean true if there is also an ACL (access control list).
+   * @deprecated Get acl bit from the {@link org.apache.hadoop.fs.FileStatus}
+   * object.
+   */
+  public boolean getAclBit() {
+    return aclBit;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof FsPermission) {
+      FsPermission that = (FsPermission) obj;
+      return this.getUserAction() == that.getUserAction()
+          && this.getGroupAction() == that.getGroupAction()
+          && this.getOtherAction() == that.getOtherAction()
+          && this.getStickyBit() == that.getStickyBit();
+    }
+    return false;
+  }
+
+  /**
+   * Create a AbfsPermission from a abfs symbolic permission string
+   * @param abfsSymbolicPermission e.g. "rw-rw-rw-+" / "rw-rw-rw-"
+   * @return a permission object for the provided string representation
+   */
+  public static AbfsPermission valueOf(final String abfsSymbolicPermission) {
+    if (abfsSymbolicPermission == null) {
+      return null;
+    }
+
+    final boolean isExtendedAcl = abfsSymbolicPermission.charAt(abfsSymbolicPermission.length() - 1) == '+';
+
+    final String abfsRawSymbolicPermission = isExtendedAcl ? abfsSymbolicPermission.substring(0, abfsSymbolicPermission.length() - 1)
+        : abfsSymbolicPermission;
+
+    int n = 0;
+    for (int i = 0; i < abfsRawSymbolicPermission.length(); i++) {
+      n = n << 1;
+      char c = abfsRawSymbolicPermission.charAt(i);
+      n += (c == '-' || c == 'T' || c == 'S') ? 0: 1;
+    }
+
+    // Add sticky bit value if set
+    if (abfsRawSymbolicPermission.charAt(abfsRawSymbolicPermission.length() - 1) == 't'
+        || abfsRawSymbolicPermission.charAt(abfsRawSymbolicPermission.length() - 1) == 'T') {
+      n += STICKY_BIT_OCTAL_VALUE;
+    }
+
+    return new AbfsPermission((short) n, isExtendedAcl);
+  }
+
+  /**
+   * Check whether abfs symbolic permission string is a extended Acl
+   * @param abfsSymbolicPermission e.g. "rw-rw-rw-+" / "rw-rw-rw-"
+   * @return true if the permission string indicates the existence of an
+   * extended ACL; otherwise false.
+   */
+  public static boolean isExtendedAcl(final String abfsSymbolicPermission) {
+    if (abfsSymbolicPermission == null) {
+      return false;
+    }
+
+    return abfsSymbolicPermission.charAt(abfsSymbolicPermission.length() - 1) == '+';
+  }
+
+  @Override
+  public int hashCode() {
+    return toShort();
+  }
+}

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

@@ -0,0 +1,193 @@
+/**
+ * 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.HttpURLConnection;
+import java.net.URL;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+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;
+
+/**
+ * The AbfsRestOperation for Rest AbfsClient.
+ */
+public class AbfsRestOperation {
+  // The type of the REST operation (Append, ReadFile, etc)
+  private final AbfsRestOperationType operationType;
+  // Blob FS client, which has the credentials, retry policy, and logs.
+  private final AbfsClient client;
+  // the HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE)
+  private final String method;
+  // full URL including query parameters
+  private final URL url;
+  // all the custom HTTP request headers provided by the caller
+  private final List<AbfsHttpHeader> requestHeaders;
+
+  // This is a simple operation class, where all the upload methods have a
+  // request body and all the download methods have a response body.
+  private final boolean hasRequestBody;
+
+  private static final Logger LOG = LoggerFactory.getLogger(AbfsClient.class);
+
+  // For uploads, this is the request entity body.  For downloads,
+  // this will hold the response entity body.
+  private byte[] buffer;
+  private int bufferOffset;
+  private int bufferLength;
+
+  private AbfsHttpOperation result;
+
+  public AbfsHttpOperation getResult() {
+    return result;
+  }
+
+  /**
+   * Initializes a new REST operation.
+   *
+   * @param client The Blob FS client.
+   * @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE).
+   * @param url The full URL including query string parameters.
+   * @param requestHeaders The HTTP request headers.
+   */
+  AbfsRestOperation(final AbfsRestOperationType operationType,
+                    final AbfsClient client,
+                    final String method,
+                    final URL url,
+                    final List<AbfsHttpHeader> requestHeaders) {
+    this.operationType = operationType;
+    this.client = client;
+    this.method = method;
+    this.url = url;
+    this.requestHeaders = requestHeaders;
+    this.hasRequestBody = (AbfsHttpConstants.HTTP_METHOD_PUT.equals(method)
+            || AbfsHttpConstants.HTTP_METHOD_PATCH.equals(method));
+  }
+
+  /**
+   * Initializes a new REST operation.
+   *
+   * @param operationType The type of the REST operation (Append, ReadFile, etc).
+   * @param client The Blob FS client.
+   * @param method The HTTP method (PUT, PATCH, POST, GET, HEAD, or DELETE).
+   * @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.
+   * @param bufferOffset An offset into the buffer where the data beings.
+   * @param bufferLength The length of the data in the buffer.
+   */
+  AbfsRestOperation(AbfsRestOperationType operationType,
+                    AbfsClient client,
+                    String method,
+                    URL url,
+                    List<AbfsHttpHeader> requestHeaders,
+                    byte[] buffer,
+                    int bufferOffset,
+                    int bufferLength) {
+    this(operationType, client, method, url, requestHeaders);
+    this.buffer = buffer;
+    this.bufferOffset = bufferOffset;
+    this.bufferLength = bufferLength;
+  }
+
+  /**
+   * Executes the REST operation with retry, by issuing one or more
+   * HTTP operations.
+   */
+  void execute() throws AzureBlobFileSystemException {
+    int retryCount = 0;
+    while (!executeHttpOperation(retryCount++)) {
+      try {
+        Thread.sleep(client.getRetryPolicy().getRetryInterval(retryCount));
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+      }
+    }
+
+    if (result.getStatusCode() >= HttpURLConnection.HTTP_BAD_REQUEST) {
+      throw new AbfsRestOperationException(result.getStatusCode(), result.getStorageErrorCode(),
+          result.getStorageErrorMessage(), null, result);
+    }
+  }
+
+  /**
+   * Executes a single HTTP operation to complete the REST operation.  If it
+   * fails, there may be a retry.  The retryCount is incremented with each
+   * attempt.
+   */
+  private boolean executeHttpOperation(final int retryCount) throws AzureBlobFileSystemException {
+    AbfsHttpOperation httpOperation = null;
+    try {
+      // initialize the HTTP request and open the connection
+      httpOperation = new AbfsHttpOperation(url, method, requestHeaders);
+
+      // sign the HTTP request
+      if (client.getAccessToken() == null) {
+        // sign the HTTP request
+        client.getSharedKeyCredentials().signRequest(
+                httpOperation.getConnection(),
+                hasRequestBody ? bufferLength : 0);
+      } else {
+        httpOperation.getConnection().setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION,
+                client.getAccessToken());
+      }
+
+      AbfsClientThrottlingIntercept.sendingRequest(operationType);
+
+      if (hasRequestBody) {
+        // HttpUrlConnection requires
+        httpOperation.sendRequest(buffer, bufferOffset, bufferLength);
+      }
+
+      httpOperation.processResponse(buffer, bufferOffset, bufferLength);
+    } catch (IOException ex) {
+      if (LOG.isDebugEnabled()) {
+        if (httpOperation != null) {
+          LOG.debug("HttpRequestFailure: " + httpOperation.toString(), ex);
+        } else {
+          LOG.debug("HttpRequestFailure: " + method + "," + url, ex);
+        }
+      }
+      if (!client.getRetryPolicy().shouldRetry(retryCount, -1)) {
+        throw new InvalidAbfsRestOperationException(ex);
+      }
+      return false;
+    } finally {
+      AbfsClientThrottlingIntercept.updateMetrics(operationType, httpOperation);
+    }
+
+    LOG.debug("HttpRequest: " + httpOperation.toString());
+
+    if (client.getRetryPolicy().shouldRetry(retryCount, httpOperation.getStatusCode())) {
+      return false;
+    }
+
+    result = httpOperation;
+
+    return true;
+  }
+}

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

@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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;
+
+/**
+ * The REST operation type (Read, Append, Other ).
+ */
+public enum AbfsRestOperationType {
+    CreateFileSystem,
+    GetFileSystemProperties,
+    SetFileSystemProperties,
+    ListPaths,
+    DeleteFileSystem,
+    CreatePath,
+    RenamePath,
+    GetAcl,
+    GetPathProperties,
+    SetAcl,
+    SetOwner,
+    SetPathProperties,
+    SetPermissions,
+    Append,
+    Flush,
+    ReadFile,
+    DeletePath
+}

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

@@ -0,0 +1,64 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
+
+/**
+ * The UrlQueryBuilder for Rest AbfsClient.
+ */
+public class AbfsUriQueryBuilder {
+  private Map<String, String> parameters;
+
+  public AbfsUriQueryBuilder() {
+    this.parameters = new HashMap<>();
+  }
+
+  public void addQuery(final String name, final String value) {
+    if (value != null && !value.isEmpty()) {
+      this.parameters.put(name, value);
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    boolean first = true;
+
+    for (Map.Entry<String, String> entry : parameters.entrySet()) {
+      if (first) {
+        sb.append(AbfsHttpConstants.QUESTION_MARK);
+        first = false;
+      } else {
+        sb.append(AbfsHttpConstants.AND_MARK);
+      }
+      try {
+        sb.append(entry.getKey()).append(AbfsHttpConstants.EQUAL).append(AbfsClient.urlEncode(entry.getValue()));
+      }
+      catch (AzureBlobFileSystemException ex) {
+        throw new IllegalArgumentException("Query string param is not encode-able: " + entry.getKey() + "=" + entry.getValue());
+      }
+    }
+    return sb.toString();
+  }
+}

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

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

+ 144 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ExponentialRetryPolicy.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.util.Random;
+import java.net.HttpURLConnection;
+
+/**
+ * Retry policy used by AbfsClient.
+ * */
+public class ExponentialRetryPolicy {
+  /**
+   * Represents the default number of retry attempts.
+   */
+  private static final int DEFAULT_CLIENT_RETRY_COUNT = 30;
+
+  /**
+   * Represents the default amount of time used when calculating a random delta in the exponential
+   * delay between retries.
+   */
+  private static final int DEFAULT_CLIENT_BACKOFF = 1000 * 3;
+
+  /**
+   * Represents the default maximum amount of time used when calculating the exponential
+   * delay between retries.
+   */
+  private static final int DEFAULT_MAX_BACKOFF = 1000 * 30;
+
+  /**
+   * Represents the default minimum amount of time used when calculating the exponential
+   * delay between retries.
+   */
+  private static final int DEFAULT_MIN_BACKOFF = 1000 * 3;
+
+  /**
+   *  The minimum random ratio used for delay interval calculation.
+   */
+  private static final double MIN_RANDOM_RATIO = 0.8;
+
+  /**
+   *  The maximum random ratio used for delay interval calculation.
+   */
+  private static final double MAX_RANDOM_RATIO = 1.2;
+
+  /**
+   *  Holds the random number generator used to calculate randomized backoff intervals
+   */
+  private final Random randRef = new Random();
+
+  /**
+   * The value that will be used to calculate a random delta in the exponential delay interval
+   */
+  private final int deltaBackoff;
+
+  /**
+   * The maximum backoff time.
+   */
+  private final int maxBackoff;
+
+  /**
+   * The minimum backoff time.
+   */
+  private final int minBackoff;
+
+  /**
+   * The maximum number of retry attempts.
+   */
+  private final int retryCount;
+
+  /**
+   * Initializes a new instance of the {@link ExponentialRetryPolicy} class.
+   */
+  public ExponentialRetryPolicy() {
+    this(DEFAULT_CLIENT_RETRY_COUNT, DEFAULT_MIN_BACKOFF, DEFAULT_MAX_BACKOFF, DEFAULT_CLIENT_BACKOFF);
+  }
+
+  /**
+   * Initializes a new instance of the {@link ExponentialRetryPolicy} class.
+   *
+   * @param retryCount The maximum number of retry attempts.
+   * @param minBackoff The minimum backoff time.
+   * @param maxBackoff The maximum backoff time.
+   * @param deltaBackoff The value that will be used to calculate a random delta in the exponential delay
+   *                     between retries.
+   */
+  public ExponentialRetryPolicy(final int retryCount, final int minBackoff, final int maxBackoff, final int deltaBackoff) {
+    this.retryCount = retryCount;
+    this.minBackoff = minBackoff;
+    this.maxBackoff = maxBackoff;
+    this.deltaBackoff = deltaBackoff;
+  }
+
+  /**
+   * Returns if a request should be retried based on the retry count, current response,
+   * and the current strategy.
+   *
+   * @param retryCount The current retry attempt count.
+   * @param statusCode The status code of the response, or -1 for socket error.
+   * @return true if the request should be retried; false otherwise.
+   */
+  public boolean shouldRetry(final int retryCount, final int statusCode) {
+    return retryCount < this.retryCount
+        && (statusCode == -1
+        || statusCode == HttpURLConnection.HTTP_CLIENT_TIMEOUT
+        || (statusCode >= HttpURLConnection.HTTP_INTERNAL_ERROR
+            && statusCode != HttpURLConnection.HTTP_NOT_IMPLEMENTED
+            && statusCode != HttpURLConnection.HTTP_VERSION));
+  }
+
+  /**
+   * Returns backoff interval between 80% and 120% of the desired backoff,
+   * multiply by 2^n-1 for exponential.
+   *
+   * @param retryCount The current retry attempt count.
+   * @return backoff Interval time
+   */
+  public long getRetryInterval(final int retryCount) {
+    final long boundedRandDelta = (int) (this.deltaBackoff * MIN_RANDOM_RATIO)
+        + this.randRef.nextInt((int) (this.deltaBackoff * MAX_RANDOM_RATIO)
+        - (int) (this.deltaBackoff * MIN_RANDOM_RATIO));
+
+    final double incrementDelta = (Math.pow(2, retryCount - 1)) * boundedRandDelta;
+
+    final long retryInterval = (int) Math.round(Math.min(this.minBackoff + incrementDelta, maxBackoff));
+
+    return retryInterval;
+  }
+}

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

@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException;
+
+/**
+ * The interface that every Azure file system key provider must implement.
+ */
+public interface KeyProvider {
+  /**
+   * Key providers must implement this method. Given a list of configuration
+   * parameters for the specified Azure storage account, retrieve the plaintext
+   * storage account key.
+   *
+   * @param accountName
+   *          the storage account name
+   * @param conf
+   *          Hadoop configuration parameters
+   * @return the plaintext storage account key
+   * @throws KeyProviderException if an error occurs while attempting to get
+   *         the storage account key.
+   */
+  String getStorageAccountKey(String accountName, Configuration conf)
+      throws KeyProviderException;
+}

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

@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
+
+class ReadBuffer {
+
+  private AbfsInputStream stream;
+  private long offset;                   // offset within the file for the buffer
+  private int length;                    // actual length, set after the buffer is filles
+  private int requestedLength;           // requested length of the read
+  private byte[] buffer;                 // the buffer itself
+  private int bufferindex = -1;          // index in the buffers array in Buffer manager
+  private ReadBufferStatus status;             // status of the buffer
+  private CountDownLatch latch = null;   // signaled when the buffer is done reading, so any client
+  // waiting on this buffer gets unblocked
+
+  // fields to help with eviction logic
+  private long timeStamp = 0;  // tick at which buffer became available to read
+  private boolean isFirstByteConsumed = false;
+  private boolean isLastByteConsumed = false;
+  private boolean isAnyByteConsumed = false;
+
+  public AbfsInputStream getStream() {
+    return stream;
+  }
+
+  public void setStream(AbfsInputStream stream) {
+    this.stream = stream;
+  }
+
+  public long getOffset() {
+    return offset;
+  }
+
+  public void setOffset(long offset) {
+    this.offset = offset;
+  }
+
+  public int getLength() {
+    return length;
+  }
+
+  public void setLength(int length) {
+    this.length = length;
+  }
+
+  public int getRequestedLength() {
+    return requestedLength;
+  }
+
+  public void setRequestedLength(int requestedLength) {
+    this.requestedLength = requestedLength;
+  }
+
+  public byte[] getBuffer() {
+    return buffer;
+  }
+
+  public void setBuffer(byte[] buffer) {
+    this.buffer = buffer;
+  }
+
+  public int getBufferindex() {
+    return bufferindex;
+  }
+
+  public void setBufferindex(int bufferindex) {
+    this.bufferindex = bufferindex;
+  }
+
+  public ReadBufferStatus getStatus() {
+    return status;
+  }
+
+  public void setStatus(ReadBufferStatus status) {
+    this.status = status;
+  }
+
+  public CountDownLatch getLatch() {
+    return latch;
+  }
+
+  public void setLatch(CountDownLatch latch) {
+    this.latch = latch;
+  }
+
+  public long getTimeStamp() {
+    return timeStamp;
+  }
+
+  public void setTimeStamp(long timeStamp) {
+    this.timeStamp = timeStamp;
+  }
+
+  public boolean isFirstByteConsumed() {
+    return isFirstByteConsumed;
+  }
+
+  public void setFirstByteConsumed(boolean isFirstByteConsumed) {
+    this.isFirstByteConsumed = isFirstByteConsumed;
+  }
+
+  public boolean isLastByteConsumed() {
+    return isLastByteConsumed;
+  }
+
+  public void setLastByteConsumed(boolean isLastByteConsumed) {
+    this.isLastByteConsumed = isLastByteConsumed;
+  }
+
+  public boolean isAnyByteConsumed() {
+    return isAnyByteConsumed;
+  }
+
+  public void setAnyByteConsumed(boolean isAnyByteConsumed) {
+    this.isAnyByteConsumed = isAnyByteConsumed;
+  }
+
+}

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

@@ -0,0 +1,395 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.azurebfs.services;
+
+import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.Stack;
+import java.util.concurrent.CountDownLatch;
+
+/**
+ * The Read Buffer Manager for Rest AbfsClient.
+ */
+final class ReadBufferManager {
+  private static final Logger LOGGER = LoggerFactory.getLogger(ReadBufferManager.class);
+
+  private static final int NUM_BUFFERS = 16;
+  private static final int BLOCK_SIZE = 4 * 1024 * 1024;
+  private static final int NUM_THREADS = 8;
+  private static final int THRESHOLD_AGE_MILLISECONDS = 3000; // have to see if 3 seconds is a good threshold
+
+  private Thread[] threads = new Thread[NUM_THREADS];
+  private byte[][] buffers;    // array of byte[] buffers, to hold the data that is read
+  private Stack<Integer> freeList = new Stack<>();   // indices in buffers[] array that are available
+
+  private Queue<ReadBuffer> readAheadQueue = new LinkedList<>(); // queue of requests that are not picked up by any worker thread yet
+  private LinkedList<ReadBuffer> inProgressList = new LinkedList<>(); // requests being processed by worker threads
+  private LinkedList<ReadBuffer> completedReadList = new LinkedList<>(); // buffers available for reading
+  private static final ReadBufferManager BUFFER_MANAGER; // singleton, initialized in static initialization block
+
+  static {
+    BUFFER_MANAGER = new ReadBufferManager();
+    BUFFER_MANAGER.init();
+  }
+
+  static ReadBufferManager getBufferManager() {
+    return BUFFER_MANAGER;
+  }
+
+  private void init() {
+    buffers = new byte[NUM_BUFFERS][];
+    for (int i = 0; i < NUM_BUFFERS; i++) {
+      buffers[i] = new byte[BLOCK_SIZE];  // same buffers are reused. The byte array never goes back to GC
+      freeList.add(i);
+    }
+    for (int i = 0; i < NUM_THREADS; i++) {
+      Thread t = new Thread(new ReadBufferWorker(i));
+      t.setDaemon(true);
+      threads[i] = t;
+      t.setName("ABFS-prefetch-" + i);
+      t.start();
+    }
+    ReadBufferWorker.UNLEASH_WORKERS.countDown();
+  }
+
+  // hide instance constructor
+  private ReadBufferManager() {
+  }
+
+
+  /*
+   *
+   *  AbfsInputStream-facing methods
+   *
+   */
+
+
+  /**
+   * {@link AbfsInputStream} calls this method to queue read-aheads.
+   *
+   * @param stream          The {@link AbfsInputStream} for which to do the read-ahead
+   * @param requestedOffset The offset in the file which shoukd be read
+   * @param requestedLength The length to read
+   */
+  void queueReadAhead(final AbfsInputStream stream, final long requestedOffset, final int requestedLength) {
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("Start Queueing readAhead for {} offset {} length {}",
+          stream.getPath(), requestedOffset, requestedLength);
+    }
+    ReadBuffer buffer;
+    synchronized (this) {
+      if (isAlreadyQueued(stream, requestedOffset)) {
+        return; // already queued, do not queue again
+      }
+      if (freeList.isEmpty() && !tryEvict()) {
+        return; // no buffers available, cannot queue anything
+      }
+
+      buffer = new ReadBuffer();
+      buffer.setStream(stream);
+      buffer.setOffset(requestedOffset);
+      buffer.setLength(0);
+      buffer.setRequestedLength(requestedLength);
+      buffer.setStatus(ReadBufferStatus.NOT_AVAILABLE);
+      buffer.setLatch(new CountDownLatch(1));
+
+      Integer bufferIndex = freeList.pop();  // will return a value, since we have checked size > 0 already
+
+      buffer.setBuffer(buffers[bufferIndex]);
+      buffer.setBufferindex(bufferIndex);
+      readAheadQueue.add(buffer);
+      notifyAll();
+    }
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("Done q-ing readAhead for file {} offset {} buffer idx {}",
+          stream.getPath(), requestedOffset, buffer.getBufferindex());
+    }
+  }
+
+
+  /**
+   * {@link AbfsInputStream} calls this method read any bytes already available in a buffer (thereby saving a
+   * remote read). This returns the bytes if the data already exists in buffer. If there is a buffer that is reading
+   * the requested offset, then this method blocks until that read completes. If the data is queued in a read-ahead
+   * but not picked up by a worker thread yet, then it cancels that read-ahead and reports cache miss. This is because
+   * depending on worker thread availability, the read-ahead may take a while - the calling thread can do it's own
+   * read to get the data faster (copmared to the read waiting in queue for an indeterminate amount of time).
+   *
+   * @param stream   the file to read bytes for
+   * @param position the offset in the file to do a read for
+   * @param length   the length to read
+   * @param buffer   the buffer to read data into. Note that the buffer will be written into from offset 0.
+   * @return the number of bytes read
+   */
+  int getBlock(final AbfsInputStream stream, final long position, final int length, final byte[] buffer) {
+    // not synchronized, so have to be careful with locking
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("getBlock for file {}  position {}  thread {}",
+          stream.getPath(), position, Thread.currentThread().getName());
+    }
+
+    waitForProcess(stream, position);
+
+    int bytesRead = 0;
+    synchronized (this) {
+      bytesRead = getBlockFromCompletedQueue(stream, position, length, buffer);
+    }
+    if (bytesRead > 0) {
+      if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("Done read from Cache for {} position {} length {}",
+            stream.getPath(), position, bytesRead);
+      }
+      return bytesRead;
+    }
+
+    // otherwise, just say we got nothing - calling thread can do its own read
+    return 0;
+  }
+
+  /*
+   *
+   *  Internal methods
+   *
+   */
+
+  private void waitForProcess(final AbfsInputStream stream, final long position) {
+    ReadBuffer readBuf;
+    synchronized (this) {
+      clearFromReadAheadQueue(stream, position);
+      readBuf = getFromList(inProgressList, stream, position);
+    }
+    if (readBuf != null) {         // if in in-progress queue, then block for it
+      try {
+        if (LOGGER.isTraceEnabled()) {
+          LOGGER.trace("got a relevant read buffer for file {} offset {} buffer idx {}",
+              stream.getPath(), readBuf.getOffset(), readBuf.getBufferindex());
+        }
+        readBuf.getLatch().await();  // blocking wait on the caller stream's thread
+        // Note on correctness: readBuf gets out of inProgressList only in 1 place: after worker thread
+        // is done processing it (in doneReading). There, the latch is set after removing the buffer from
+        // inProgressList. So this latch is safe to be outside the synchronized block.
+        // Putting it in synchronized would result in a deadlock, since this thread would be holding the lock
+        // while waiting, so no one will be able to  change any state. If this becomes more complex in the future,
+        // then the latch cane be removed and replaced with wait/notify whenever inProgressList is touched.
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+      }
+      if (LOGGER.isTraceEnabled()) {
+        LOGGER.trace("latch done for file {} buffer idx {} length {}",
+            stream.getPath(), readBuf.getBufferindex(), readBuf.getLength());
+      }
+    }
+  }
+
+  /**
+   * If any buffer in the completedlist can be reclaimed then reclaim it and return the buffer to free list.
+   * The objective is to find just one buffer - there is no advantage to evicting more than one.
+   *
+   * @return whether the eviction succeeeded - i.e., were we able to free up one buffer
+   */
+  private synchronized boolean tryEvict() {
+    ReadBuffer nodeToEvict = null;
+    if (completedReadList.size() <= 0) {
+      return false;  // there are no evict-able buffers
+    }
+
+    // first, try buffers where all bytes have been consumed (approximated as first and last bytes consumed)
+    for (ReadBuffer buf : completedReadList) {
+      if (buf.isFirstByteConsumed() && buf.isLastByteConsumed()) {
+        nodeToEvict = buf;
+        break;
+      }
+    }
+    if (nodeToEvict != null) {
+      return evict(nodeToEvict);
+    }
+
+    // next, try buffers where any bytes have been consumed (may be a bad idea? have to experiment and see)
+    for (ReadBuffer buf : completedReadList) {
+      if (buf.isAnyByteConsumed()) {
+        nodeToEvict = buf;
+        break;
+      }
+    }
+
+    if (nodeToEvict != null) {
+      return evict(nodeToEvict);
+    }
+
+    // next, try any old nodes that have not been consumed
+    long earliestBirthday = Long.MAX_VALUE;
+    for (ReadBuffer buf : completedReadList) {
+      if (buf.getTimeStamp() < earliestBirthday) {
+        nodeToEvict = buf;
+        earliestBirthday = buf.getTimeStamp();
+      }
+    }
+    if ((currentTimeMillis() - earliestBirthday > THRESHOLD_AGE_MILLISECONDS) && (nodeToEvict != null)) {
+      return evict(nodeToEvict);
+    }
+
+    // nothing can be evicted
+    return false;
+  }
+
+  private boolean evict(final ReadBuffer buf) {
+    freeList.push(buf.getBufferindex());
+    completedReadList.remove(buf);
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("Evicting buffer idx {}; was used for file {} offset {} length {}",
+          buf.getBufferindex(), buf.getStream().getPath(), buf.getOffset(), buf.getLength());
+    }
+    return true;
+  }
+
+  private boolean isAlreadyQueued(final AbfsInputStream stream, final long requestedOffset) {
+    // returns true if any part of the buffer is already queued
+    return (isInList(readAheadQueue, stream, requestedOffset)
+        || isInList(inProgressList, stream, requestedOffset)
+        || isInList(completedReadList, stream, requestedOffset));
+  }
+
+  private boolean isInList(final Collection<ReadBuffer> list, final AbfsInputStream stream, final long requestedOffset) {
+    return (getFromList(list, stream, requestedOffset) != null);
+  }
+
+  private ReadBuffer getFromList(final Collection<ReadBuffer> list, final AbfsInputStream stream, final long requestedOffset) {
+    for (ReadBuffer buffer : list) {
+      if (buffer.getStream() == stream) {
+        if (buffer.getStatus() == ReadBufferStatus.AVAILABLE
+            && requestedOffset >= buffer.getOffset()
+            && requestedOffset < buffer.getOffset() + buffer.getLength()) {
+          return buffer;
+        } else if (requestedOffset >= buffer.getOffset()
+            && requestedOffset < buffer.getOffset() + buffer.getRequestedLength()) {
+          return buffer;
+        }
+      }
+    }
+    return null;
+  }
+
+  private void clearFromReadAheadQueue(final AbfsInputStream stream, final long requestedOffset) {
+    ReadBuffer buffer = getFromList(readAheadQueue, stream, requestedOffset);
+    if (buffer != null) {
+      readAheadQueue.remove(buffer);
+      notifyAll();   // lock is held in calling method
+      freeList.push(buffer.getBufferindex());
+    }
+  }
+
+  private int getBlockFromCompletedQueue(final AbfsInputStream stream, final long position, final int length,
+                                         final byte[] buffer) {
+    ReadBuffer buf = getFromList(completedReadList, stream, position);
+    if (buf == null || position >= buf.getOffset() + buf.getLength()) {
+      return 0;
+    }
+    int cursor = (int) (position - buf.getOffset());
+    int availableLengthInBuffer = buf.getLength() - cursor;
+    int lengthToCopy = Math.min(length, availableLengthInBuffer);
+    System.arraycopy(buf.getBuffer(), cursor, buffer, 0, lengthToCopy);
+    if (cursor == 0) {
+      buf.setFirstByteConsumed(true);
+    }
+    if (cursor + lengthToCopy == buf.getLength()) {
+      buf.setLastByteConsumed(true);
+    }
+    buf.setAnyByteConsumed(true);
+    return lengthToCopy;
+  }
+
+  /*
+   *
+   *  ReadBufferWorker-thread-facing methods
+   *
+   */
+
+  /**
+   * ReadBufferWorker thread calls this to get the next buffer that it should work on.
+   *
+   * @return {@link ReadBuffer}
+   * @throws InterruptedException if thread is interrupted
+   */
+  ReadBuffer getNextBlockToRead() throws InterruptedException {
+    ReadBuffer buffer = null;
+    synchronized (this) {
+      //buffer = readAheadQueue.take();  // blocking method
+      while (readAheadQueue.size() == 0) {
+        wait();
+      }
+      buffer = readAheadQueue.remove();
+      notifyAll();
+      if (buffer == null) {
+        return null;            // should never happen
+      }
+      buffer.setStatus(ReadBufferStatus.READING_IN_PROGRESS);
+      inProgressList.add(buffer);
+    }
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("ReadBufferWorker picked file {} for offset {}",
+          buffer.getStream().getPath(), buffer.getOffset());
+    }
+    return buffer;
+  }
+
+  /**
+   * ReadBufferWorker thread calls this method to post completion.
+   *
+   * @param buffer            the buffer whose read was completed
+   * @param result            the {@link ReadBufferStatus} after the read operation in the worker thread
+   * @param bytesActuallyRead the number of bytes that the worker thread was actually able to read
+   */
+  void doneReading(final ReadBuffer buffer, final ReadBufferStatus result, final int bytesActuallyRead) {
+    if (LOGGER.isTraceEnabled()) {
+      LOGGER.trace("ReadBufferWorker completed file {} for offset {} bytes {}",
+          buffer.getStream().getPath(),  buffer.getOffset(), bytesActuallyRead);
+    }
+    synchronized (this) {
+      inProgressList.remove(buffer);
+      if (result == ReadBufferStatus.AVAILABLE && bytesActuallyRead > 0) {
+        buffer.setStatus(ReadBufferStatus.AVAILABLE);
+        buffer.setTimeStamp(currentTimeMillis());
+        buffer.setLength(bytesActuallyRead);
+        completedReadList.add(buffer);
+      } else {
+        freeList.push(buffer.getBufferindex());
+        // buffer should go out of scope after the end of the calling method in ReadBufferWorker, and eligible for GC
+      }
+    }
+    //outside the synchronized, since anyone receiving a wake-up from the latch must see safe-published results
+    buffer.getLatch().countDown(); // wake up waiting threads (if any)
+  }
+
+  /**
+   * Similar to System.currentTimeMillis, except implemented with System.nanoTime().
+   * System.currentTimeMillis can go backwards when system clock is changed (e.g., with NTP time synchronization),
+   * making it unsuitable for measuring time intervals. nanotime is strictly monotonically increasing per CPU core.
+   * Note: it is not monotonic across Sockets, and even within a CPU, its only the
+   * more recent parts which share a clock across all cores.
+   *
+   * @return current time in milliseconds
+   */
+  private long currentTimeMillis() {
+    return System.nanoTime() / 1000 / 1000;
+  }
+}

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

@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus;
+
+class ReadBufferWorker implements Runnable {
+
+  protected static final CountDownLatch UNLEASH_WORKERS = new CountDownLatch(1);
+  private int id;
+
+  ReadBufferWorker(final int id) {
+    this.id = id;
+  }
+
+  /**
+   * return the ID of ReadBufferWorker.
+   */
+  public int getId() {
+    return this.id;
+  }
+
+  /**
+   * Waits until a buffer becomes available in ReadAheadQueue.
+   * Once a buffer becomes available, reads the file specified in it and then posts results back to buffer manager.
+   * Rinse and repeat. Forever.
+   */
+  public void run() {
+    try {
+      UNLEASH_WORKERS.await();
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+    }
+    ReadBufferManager bufferManager = ReadBufferManager.getBufferManager();
+    ReadBuffer buffer;
+    while (true) {
+      try {
+        buffer = bufferManager.getNextBlockToRead();   // blocks, until a buffer is available for this thread
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        return;
+      }
+      if (buffer != null) {
+        try {
+          // do the actual read, from the file.
+          int bytesRead = buffer.getStream().readRemote(buffer.getOffset(), buffer.getBuffer(), 0, buffer.getRequestedLength());
+          bufferManager.doneReading(buffer, ReadBufferStatus.AVAILABLE, bytesRead);  // post result back to ReadBufferManager
+        } catch (Exception ex) {
+          bufferManager.doneReading(buffer, ReadBufferStatus.READ_FAILED, 0);
+        }
+      }
+    }
+  }
+}

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

@@ -0,0 +1,510 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.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;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TimeZone;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants;
+import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations;
+import org.apache.hadoop.fs.azurebfs.utils.Base64;
+
+/**
+ * Represents the shared key credentials used to access an Azure Storage
+ * account.
+ */
+public class SharedKeyCredentials {
+  private static final int EXPECTED_BLOB_QUEUE_CANONICALIZED_STRING_LENGTH = 300;
+  private static final Pattern CRLF = Pattern.compile("\r\n", Pattern.LITERAL);
+  private static final String HMAC_SHA256 = "HmacSHA256";
+  /**
+   * Stores a reference to the RFC1123 date/time pattern.
+   */
+  private static final String RFC1123_PATTERN = "EEE, dd MMM yyyy HH:mm:ss z";
+
+
+  private String accountName;
+  private byte[] accountKey;
+  private Mac hmacSha256;
+
+  public SharedKeyCredentials(final String accountName,
+                              final String accountKey) {
+    if (accountName == null || accountName.isEmpty()) {
+      throw new IllegalArgumentException("Invalid account name.");
+    }
+    if (accountKey == null || accountKey.isEmpty()) {
+      throw new IllegalArgumentException("Invalid account key.");
+    }
+    this.accountName = accountName;
+    this.accountKey = Base64.decode(accountKey);
+    initializeMac();
+  }
+
+  public void signRequest(HttpURLConnection connection, final long contentLength) throws UnsupportedEncodingException {
+
+    connection.setRequestProperty(HttpHeaderConfigurations.X_MS_DATE, getGMTTime());
+
+    final String stringToSign = canonicalize(connection, accountName, contentLength);
+
+    final String computedBase64Signature = computeHmac256(stringToSign);
+
+    connection.setRequestProperty(HttpHeaderConfigurations.AUTHORIZATION,
+        String.format("%s %s:%s", "SharedKey", accountName, computedBase64Signature));
+  }
+
+  private String computeHmac256(final String stringToSign) {
+    byte[] utf8Bytes;
+    try {
+      utf8Bytes = stringToSign.getBytes(AbfsHttpConstants.UTF_8);
+    } catch (final UnsupportedEncodingException e) {
+      throw new IllegalArgumentException(e);
+    }
+    byte[] hmac;
+    synchronized (this) {
+      hmac = hmacSha256.doFinal(utf8Bytes);
+    }
+    return Base64.encode(hmac);
+  }
+
+  /**
+   * Add x-ms- prefixed headers in a fixed order.
+   *
+   * @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) {
+    // Look for header names that start with
+    // HeaderNames.PrefixForStorageHeader
+    // Then sort them in case-insensitive manner.
+
+    final Map<String, List<String>> headers = conn.getRequestProperties();
+    final ArrayList<String> httpStorageHeaderNameArray = new ArrayList<String>();
+
+    for (final String key : headers.keySet()) {
+      if (key.toLowerCase(Locale.ROOT).startsWith(AbfsHttpConstants.HTTP_HEADER_PREFIX)) {
+        httpStorageHeaderNameArray.add(key.toLowerCase(Locale.ROOT));
+      }
+    }
+
+    Collections.sort(httpStorageHeaderNameArray);
+
+    // Now go through each header's values in the sorted order and append
+    // them to the canonicalized string.
+    for (final String key : httpStorageHeaderNameArray) {
+      final StringBuilder canonicalizedElement = new StringBuilder(key);
+      String delimiter = ":";
+      final ArrayList<String> values = getHeaderValues(headers, key);
+
+      boolean appendCanonicalizedElement = false;
+      // Go through values, unfold them, and then append them to the
+      // canonicalized element string.
+      for (final String value : values) {
+        if (value != null) {
+          appendCanonicalizedElement = true;
+        }
+
+        // Unfolding is simply removal of CRLF.
+        final String unfoldedValue = CRLF.matcher(value)
+            .replaceAll(Matcher.quoteReplacement(""));
+
+        // Append it to the canonicalized element string.
+        canonicalizedElement.append(delimiter);
+        canonicalizedElement.append(unfoldedValue);
+        delimiter = ",";
+      }
+
+      // Now, add this canonicalized element to the canonicalized header
+      // string.
+      if (appendCanonicalizedElement) {
+        appendCanonicalizedElement(canonicalizedString, canonicalizedElement.toString());
+      }
+    }
+  }
+
+  /**
+   * Initialize the HmacSha256 associated with the account key.
+   */
+  private void initializeMac() {
+    // Initializes the HMAC-SHA256 Mac and SecretKey.
+    try {
+      hmacSha256 = Mac.getInstance(HMAC_SHA256);
+      hmacSha256.init(new SecretKeySpec(accountKey, HMAC_SHA256));
+    } catch (final Exception e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
+  /**
+   * Append a string to a string builder with a newline constant.
+   *
+   * @param builder the StringBuilder object
+   * @param element the string to append.
+   */
+  private static void appendCanonicalizedElement(final StringBuilder builder, final String element) {
+    builder.append("\n");
+    builder.append(element);
+  }
+
+  /**
+   * Constructs a canonicalized string from the request's headers that will be used to construct the signature string
+   * for signing a Blob or Queue service request under the Shared Key Full authentication scheme.
+   *
+   * @param address       the request URI
+   * @param accountName   the account name associated with the request
+   * @param method        the verb to be used for the HTTP request.
+   * @param contentType   the content type of the HTTP request.
+   * @param contentLength the length of the content written to the outputstream in bytes, -1 if unknown
+   * @param date          the date/time specification for the HTTP request
+   * @param conn          the HttpURLConnection for the operation.
+   * @return A canonicalized string.
+   */
+  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)
+      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());
+
+    // The next elements are
+    // If any element is missing it may be empty.
+    appendCanonicalizedElement(canonicalizedString,
+        getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_ENCODING, AbfsHttpConstants.EMPTY_STRING));
+    appendCanonicalizedElement(canonicalizedString,
+        getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_LANGUAGE, AbfsHttpConstants.EMPTY_STRING));
+    appendCanonicalizedElement(canonicalizedString,
+        contentLength <= 0 ? "" : String.valueOf(contentLength));
+    appendCanonicalizedElement(canonicalizedString,
+        getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_MD5, AbfsHttpConstants.EMPTY_STRING));
+    appendCanonicalizedElement(canonicalizedString, contentType != null ? contentType : AbfsHttpConstants.EMPTY_STRING);
+
+    final String dateString = getHeaderValue(conn, HttpHeaderConfigurations.X_MS_DATE, AbfsHttpConstants.EMPTY_STRING);
+    // If x-ms-date header exists, Date should be empty string
+    appendCanonicalizedElement(canonicalizedString, dateString.equals(AbfsHttpConstants.EMPTY_STRING) ? date
+        : "");
+
+    appendCanonicalizedElement(canonicalizedString,
+        getHeaderValue(conn, HttpHeaderConfigurations.IF_MODIFIED_SINCE, AbfsHttpConstants.EMPTY_STRING));
+    appendCanonicalizedElement(canonicalizedString,
+        getHeaderValue(conn, HttpHeaderConfigurations.IF_MATCH, AbfsHttpConstants.EMPTY_STRING));
+    appendCanonicalizedElement(canonicalizedString,
+        getHeaderValue(conn, HttpHeaderConfigurations.IF_NONE_MATCH, AbfsHttpConstants.EMPTY_STRING));
+    appendCanonicalizedElement(canonicalizedString,
+        getHeaderValue(conn, HttpHeaderConfigurations.IF_UNMODIFIED_SINCE, AbfsHttpConstants.EMPTY_STRING));
+    appendCanonicalizedElement(canonicalizedString,
+        getHeaderValue(conn, HttpHeaderConfigurations.RANGE, AbfsHttpConstants.EMPTY_STRING));
+
+    addCanonicalizedHeaders(conn, canonicalizedString);
+
+    appendCanonicalizedElement(canonicalizedString, getCanonicalizedResource(address, accountName));
+
+    return canonicalizedString.toString();
+  }
+
+  /**
+   * Gets the canonicalized resource string for a Blob or Queue service request under the Shared Key Lite
+   * authentication scheme.
+   *
+   * @param address     the resource URI.
+   * @param accountName the account name for the request.
+   * @return the canonicalized resource string.
+   */
+  private static String getCanonicalizedResource(final URL address,
+      final String accountName) throws UnsupportedEncodingException {
+    // Resource path
+    final StringBuilder resourcepath = new StringBuilder(AbfsHttpConstants.FORWARD_SLASH);
+    resourcepath.append(accountName);
+
+    // Note that AbsolutePath starts with a '/'.
+    resourcepath.append(address.getPath());
+    final StringBuilder canonicalizedResource = new StringBuilder(resourcepath.toString());
+
+    // query parameters
+    if (address.getQuery() == null || !address.getQuery().contains(AbfsHttpConstants.EQUAL)) {
+      //no query params.
+      return canonicalizedResource.toString();
+    }
+
+    final Map<String, String[]> queryVariables = parseQueryString(address.getQuery());
+
+    final Map<String, String> lowercasedKeyNameValue = new HashMap<>();
+
+    for (final Entry<String, String[]> entry : queryVariables.entrySet()) {
+      // sort the value and organize it as comma separated values
+      final List<String> sortedValues = Arrays.asList(entry.getValue());
+      Collections.sort(sortedValues);
+
+      final StringBuilder stringValue = new StringBuilder();
+
+      for (final String value : sortedValues) {
+        if (stringValue.length() > 0) {
+          stringValue.append(AbfsHttpConstants.COMMA);
+        }
+
+        stringValue.append(value);
+      }
+
+      // key turns out to be null for ?a&b&c&d
+      lowercasedKeyNameValue.put((entry.getKey()) == null ? null
+          : entry.getKey().toLowerCase(Locale.ROOT), stringValue.toString());
+    }
+
+    final ArrayList<String> sortedKeys = new ArrayList<String>(lowercasedKeyNameValue.keySet());
+
+    Collections.sort(sortedKeys);
+
+    for (final String key : sortedKeys) {
+      final StringBuilder queryParamString = new StringBuilder();
+
+      queryParamString.append(key);
+      queryParamString.append(":");
+      queryParamString.append(lowercasedKeyNameValue.get(key));
+
+      appendCanonicalizedElement(canonicalizedResource, queryParamString.toString());
+    }
+
+    return canonicalizedResource.toString();
+  }
+
+  /**
+   * Gets all the values for the given header in the one to many map,
+   * performs a trimStart() on each return value.
+   *
+   * @param headers    a one to many map of key / values representing the header values for the connection.
+   * @param headerName the name of the header to lookup
+   * @return an ArrayList<String> of all trimmed values corresponding to the requested headerName. This may be empty
+   * if the header is not found.
+   */
+  private static ArrayList<String> getHeaderValues(
+      final Map<String, List<String>> headers,
+      final String headerName) {
+
+    final ArrayList<String> arrayOfValues = new ArrayList<String>();
+    List<String> values = null;
+
+    for (final Entry<String, List<String>> entry : headers.entrySet()) {
+      if (entry.getKey().toLowerCase(Locale.ROOT).equals(headerName)) {
+        values = entry.getValue();
+        break;
+      }
+    }
+    if (values != null) {
+      for (final String value : values) {
+        // canonicalization formula requires the string to be left
+        // trimmed.
+        arrayOfValues.add(trimStart(value));
+      }
+    }
+    return arrayOfValues;
+  }
+
+  /**
+   * Parses a query string into a one to many hashmap.
+   *
+   * @param parseString the string to parse
+   * @return a HashMap<String, String[]> of the key values.
+   */
+  private static HashMap<String, String[]> parseQueryString(String parseString) throws UnsupportedEncodingException {
+    final HashMap<String, String[]> retVals = new HashMap<>();
+    if (parseString == null || parseString.isEmpty()) {
+      return retVals;
+    }
+
+    // 1. Remove ? if present
+    final int queryDex = parseString.indexOf(AbfsHttpConstants.QUESTION_MARK);
+    if (queryDex >= 0 && parseString.length() > 0) {
+      parseString = parseString.substring(queryDex + 1);
+    }
+
+    // 2. split name value pairs by splitting on the 'c&' character
+    final String[] valuePairs = parseString.contains(AbfsHttpConstants.AND_MARK)
+            ? parseString.split(AbfsHttpConstants.AND_MARK)
+            : parseString.split(AbfsHttpConstants.SEMICOLON);
+
+    // 3. for each field value pair parse into appropriate map entries
+    for (int m = 0; m < valuePairs.length; m++) {
+      final int equalDex = valuePairs[m].indexOf(AbfsHttpConstants.EQUAL);
+
+      if (equalDex < 0 || equalDex == valuePairs[m].length() - 1) {
+        continue;
+      }
+
+      String key = valuePairs[m].substring(0, equalDex);
+      String value = valuePairs[m].substring(equalDex + 1);
+
+      key = safeDecode(key);
+      value = safeDecode(value);
+
+      // 3.1 add to map
+      String[] values = retVals.get(key);
+
+      if (values == null) {
+        values = new String[]{value};
+        if (!value.equals("")) {
+          retVals.put(key, values);
+        }
+      }
+    }
+
+    return retVals;
+  }
+
+  /**
+   * Performs safe decoding of the specified string, taking care to preserve each <code>+</code> character, rather
+   * than replacing it with a space character.
+   *
+   * @param stringToDecode A <code>String</code> that represents the string to decode.
+   * @return A <code>String</code> that represents the decoded string.
+   * <p>
+   * If a storage service error occurred.
+   */
+  private static String safeDecode(final String stringToDecode) throws UnsupportedEncodingException {
+    if (stringToDecode == null) {
+      return null;
+    }
+
+    if (stringToDecode.length() == 0) {
+      return "";
+    }
+
+    if (stringToDecode.contains(AbfsHttpConstants.PLUS)) {
+      final StringBuilder outBuilder = new StringBuilder();
+
+      int startDex = 0;
+      for (int m = 0; m < stringToDecode.length(); m++) {
+        if (stringToDecode.charAt(m) == '+') {
+          if (m > startDex) {
+            outBuilder.append(URLDecoder.decode(stringToDecode.substring(startDex, m),
+                    AbfsHttpConstants.UTF_8));
+          }
+
+          outBuilder.append(AbfsHttpConstants.PLUS);
+          startDex = m + 1;
+        }
+      }
+
+      if (startDex != stringToDecode.length()) {
+        outBuilder.append(URLDecoder.decode(stringToDecode.substring(startDex, stringToDecode.length()),
+                AbfsHttpConstants.UTF_8));
+      }
+
+      return outBuilder.toString();
+    } else {
+      return URLDecoder.decode(stringToDecode, AbfsHttpConstants.UTF_8);
+    }
+  }
+
+  private static String trimStart(final String value) {
+    int spaceDex = 0;
+    while (spaceDex < value.length() && value.charAt(spaceDex) == ' ') {
+      spaceDex++;
+    }
+
+    return value.substring(spaceDex);
+  }
+
+  private static String getHeaderValue(final HttpURLConnection conn, final String headerName, final String defaultValue) {
+    final String headerValue = conn.getRequestProperty(headerName);
+    return headerValue == null ? defaultValue : headerValue;
+  }
+
+
+  /**
+   * Constructs a canonicalized string for signing a request.
+   *
+   * @param conn          the HttpURLConnection to canonicalize
+   * @param accountName   the account name associated with the request
+   * @param contentLength the length of the content written to the outputstream in bytes,
+   *                      -1 if unknown
+   * @return a canonicalized string.
+   */
+  private String canonicalize(final HttpURLConnection conn,
+                              final String accountName,
+                              final Long contentLength) throws UnsupportedEncodingException {
+
+    if (contentLength < -1) {
+      throw new IllegalArgumentException(
+          "The Content-Length header must be greater than or equal to -1.");
+    }
+
+    String contentType = getHeaderValue(conn, HttpHeaderConfigurations.CONTENT_TYPE, "");
+
+    return canonicalizeHttpRequest(conn.getURL(), accountName,
+        conn.getRequestMethod(), contentType, contentLength, null, conn);
+  }
+
+  /**
+   * Thread local for storing GMT date format.
+   */
+  private static ThreadLocal<DateFormat> rfc1123GmtDateTimeFormatter
+      = new ThreadLocal<DateFormat>() {
+    @Override
+    protected DateFormat initialValue() {
+      final DateFormat formatter = new SimpleDateFormat(RFC1123_PATTERN, Locale.ROOT);
+      formatter.setTimeZone(GMT_ZONE);
+      return formatter;
+    }
+  };
+
+  public static final TimeZone GMT_ZONE = TimeZone.getTimeZone(AbfsHttpConstants.GMT_TIMEZONE);
+
+
+  /**
+   * Returns the current GMT date/time String using the RFC1123 pattern.
+   *
+   * @return A <code>String</code> that represents the current GMT date/time using the RFC1123 pattern.
+   */
+  static String getGMTTime() {
+    return getGMTTime(new Date());
+  }
+
+  /**
+   * Returns the GTM date/time String for the specified value using the RFC1123 pattern.
+   *
+   * @param date
+   *            A <code>Date</code> object that represents the date to convert to GMT date/time in the RFC1123
+   *            pattern.
+   *
+   * @return A <code>String</code> that represents the GMT date/time for the specified value using the RFC1123
+   *         pattern.
+   */
+  static String getGMTTime(final Date date) {
+    return rfc1123GmtDateTimeFormatter.get().format(date);
+  }
+}

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

@@ -0,0 +1,71 @@
+/**
+ * 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.Arrays;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException;
+import org.apache.hadoop.util.Shell;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Shell decryption key provider which invokes an external script that will
+ * perform the key decryption.
+ */
+public class ShellDecryptionKeyProvider extends SimpleKeyProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(ShellDecryptionKeyProvider.class);
+
+  @Override
+  public String getStorageAccountKey(String accountName, Configuration rawConfig)
+      throws KeyProviderException {
+    String envelope = super.getStorageAccountKey(accountName, rawConfig);
+
+    AbfsConfiguration abfsConfig;
+    try {
+      abfsConfig = new AbfsConfiguration(rawConfig, accountName);
+    } catch(IllegalAccessException | IOException e) {
+      throw new KeyProviderException("Unable to get key from credential providers.", e);
+    }
+
+    final String command = abfsConfig.get(ConfigurationKeys.AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT);
+    if (command == null) {
+      throw new KeyProviderException(
+          "Script path is not specified via fs.azure.shellkeyprovider.script");
+    }
+
+    String[] cmd = command.split(" ");
+    String[] cmdWithEnvelope = Arrays.copyOf(cmd, cmd.length + 1);
+    cmdWithEnvelope[cmdWithEnvelope.length - 1] = envelope;
+
+    String decryptedKey = null;
+    try {
+      decryptedKey = Shell.execCommand(cmdWithEnvelope);
+    } catch (IOException ex) {
+      throw new KeyProviderException(ex);
+    }
+
+    // trim any whitespace
+    return decryptedKey.trim();
+  }
+}

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

@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.services;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.KeyProviderException;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Key provider that simply returns the storage account key from the
+ * configuration as plaintext.
+ */
+public class SimpleKeyProvider implements KeyProvider {
+  private static final Logger LOG = LoggerFactory.getLogger(SimpleKeyProvider.class);
+
+  @Override
+  public String getStorageAccountKey(String accountName, Configuration rawConfig)
+      throws KeyProviderException {
+    String key = null;
+
+    try {
+      AbfsConfiguration abfsConfig = new AbfsConfiguration(rawConfig, accountName);
+      key = abfsConfig.getPasswordString(ConfigurationKeys.FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME);
+    } catch(IllegalAccessException | InvalidConfigurationValueException e) {
+      throw new KeyProviderException("Failure to initialize configuration", e);
+    } catch(IOException ioe) {
+      LOG.warn("Unable to get key from credential providers. {}", ioe);
+    }
+
+    return key;
+  }
+}

+ 22 - 0
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/package-info.java

@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+package org.apache.hadoop.fs.azurebfs.services;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

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

@@ -0,0 +1,329 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azurebfs.utils;
+
+/**
+ * Base64
+ */
+public final class Base64 {
+  /**
+   * The Base 64 Characters.
+   */
+  private static final String BASE_64_CHARS = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/";
+
+  /**
+   * Decoded values, -1 is invalid character, -2 is = pad character.
+   */
+  private static final byte[] DECODE_64 = {
+          -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, // 0-15
+
+          -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, /*
+                                                                             * 16- 31
+                                                                             */
+          -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 62, -1, -1, -1, 63, /*
+                                                                             * 32- 47
+                                                                             */
+          52, 53, 54, 55, 56, 57, 58, 59, 60, 61, -1, -1, -1, -2, -1, -1, /*
+                                                                             * 48- 63
+                                                                             */
+          -1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, /* 64-79 */
+          15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, -1, -1, -1, -1, -1, /*
+                                                                             * 80- 95
+                                                                             */
+          -1, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, /*
+                                                                             * 96- 111
+                                                                             */
+          41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, -1, -1, -1, -1, -1 /*
+                                                                            * 112- 127
+                                                                            */
+  };
+
+  /**
+   * Decodes a given Base64 string into its corresponding byte array.
+   *
+   * @param data
+   *            the Base64 string, as a <code>String</code> object, to decode
+   *
+   * @return the corresponding decoded byte array
+   * @throws IllegalArgumentException
+   *             If the string is not a valid base64 encoded string
+   */
+  public static byte[] decode(final String data) {
+    if (data == null) {
+      throw new IllegalArgumentException("The data parameter is not a valid base64-encoded string.");
+    }
+
+    int byteArrayLength = 3 * data.length() / 4;
+
+    if (data.endsWith("==")) {
+      byteArrayLength -= 2;
+    }
+    else if (data.endsWith("=")) {
+      byteArrayLength -= 1;
+    }
+
+    final byte[] retArray = new byte[byteArrayLength];
+    int byteDex = 0;
+    int charDex = 0;
+
+    for (; charDex < data.length(); charDex += 4) {
+      // get 4 chars, convert to 3 bytes
+      final int char1 = DECODE_64[(byte) data.charAt(charDex)];
+      final int char2 = DECODE_64[(byte) data.charAt(charDex + 1)];
+      final int char3 = DECODE_64[(byte) data.charAt(charDex + 2)];
+      final int char4 = DECODE_64[(byte) data.charAt(charDex + 3)];
+
+      if (char1 < 0 || char2 < 0 || char3 == -1 || char4 == -1) {
+        // invalid character(-1), or bad padding (-2)
+        throw new IllegalArgumentException("The data parameter is not a valid base64-encoded string.");
+      }
+
+      int tVal = char1 << 18;
+      tVal += char2 << 12;
+      tVal += (char3 & 0xff) << 6;
+      tVal += char4 & 0xff;
+
+      if (char3 == -2) {
+        // two "==" pad chars, check bits 12-24
+        tVal &= 0x00FFF000;
+        retArray[byteDex++] = (byte) (tVal >> 16 & 0xFF);
+      }
+      else if (char4 == -2) {
+        // one pad char "=" , check bits 6-24.
+        tVal &= 0x00FFFFC0;
+        retArray[byteDex++] = (byte) (tVal >> 16 & 0xFF);
+        retArray[byteDex++] = (byte) (tVal >> 8 & 0xFF);
+
+      }
+      else {
+        // No pads take all 3 bytes, bits 0-24
+        retArray[byteDex++] = (byte) (tVal >> 16 & 0xFF);
+        retArray[byteDex++] = (byte) (tVal >> 8 & 0xFF);
+        retArray[byteDex++] = (byte) (tVal & 0xFF);
+      }
+    }
+    return retArray;
+  }
+
+  /**
+   * Decodes a given Base64 string into its corresponding byte array.
+   *
+   * @param data
+   *            the Base64 string, as a <code>String</code> object, to decode
+   *
+   * @return the corresponding decoded byte array
+   * @throws IllegalArgumentException
+   *             If the string is not a valid base64 encoded string
+   */
+  public static Byte[] decodeAsByteObjectArray(final String data) {
+    int byteArrayLength = 3 * data.length() / 4;
+
+    if (data.endsWith("==")) {
+      byteArrayLength -= 2;
+    }
+    else if (data.endsWith("=")) {
+      byteArrayLength -= 1;
+    }
+
+    final Byte[] retArray = new Byte[byteArrayLength];
+    int byteDex = 0;
+    int charDex = 0;
+
+    for (; charDex < data.length(); charDex += 4) {
+      // get 4 chars, convert to 3 bytes
+      final int char1 = DECODE_64[(byte) data.charAt(charDex)];
+      final int char2 = DECODE_64[(byte) data.charAt(charDex + 1)];
+      final int char3 = DECODE_64[(byte) data.charAt(charDex + 2)];
+      final int char4 = DECODE_64[(byte) data.charAt(charDex + 3)];
+
+      if (char1 < 0 || char2 < 0 || char3 == -1 || char4 == -1) {
+        // invalid character(-1), or bad padding (-2)
+        throw new IllegalArgumentException("The data parameter is not a valid base64-encoded string.");
+      }
+
+      int tVal = char1 << 18;
+      tVal += char2 << 12;
+      tVal += (char3 & 0xff) << 6;
+      tVal += char4 & 0xff;
+
+      if (char3 == -2) {
+        // two "==" pad chars, check bits 12-24
+        tVal &= 0x00FFF000;
+        retArray[byteDex++] = (byte) (tVal >> 16 & 0xFF);
+      }
+      else if (char4 == -2) {
+        // one pad char "=" , check bits 6-24.
+        tVal &= 0x00FFFFC0;
+        retArray[byteDex++] = (byte) (tVal >> 16 & 0xFF);
+        retArray[byteDex++] = (byte) (tVal >> 8 & 0xFF);
+
+      }
+      else {
+        // No pads take all 3 bytes, bits 0-24
+        retArray[byteDex++] = (byte) (tVal >> 16 & 0xFF);
+        retArray[byteDex++] = (byte) (tVal >> 8 & 0xFF);
+        retArray[byteDex++] = (byte) (tVal & 0xFF);
+      }
+    }
+    return retArray;
+  }
+
+  /**
+   * Encodes a byte array as a Base64 string.
+   *
+   * @param data
+   *            the byte array to encode
+   * @return the Base64-encoded string, as a <code>String</code> object
+   */
+  public static String encode(final byte[] data) {
+    final StringBuilder builder = new StringBuilder();
+    final int dataRemainder = data.length % 3;
+
+    int j = 0;
+    int n = 0;
+    for (; j < data.length; j += 3) {
+
+      if (j < data.length - dataRemainder) {
+        n = ((data[j] & 0xFF) << 16) + ((data[j + 1] & 0xFF) << 8) + (data[j + 2] & 0xFF);
+      }
+      else {
+        if (dataRemainder == 1) {
+          n = (data[j] & 0xFF) << 16;
+        }
+        else if (dataRemainder == 2) {
+          n = ((data[j] & 0xFF) << 16) + ((data[j + 1] & 0xFF) << 8);
+        }
+      }
+
+      // Left here for readability
+      // byte char1 = (byte) ((n >>> 18) & 0x3F);
+      // byte char2 = (byte) ((n >>> 12) & 0x3F);
+      // byte char3 = (byte) ((n >>> 6) & 0x3F);
+      // byte char4 = (byte) (n & 0x3F);
+      builder.append(BASE_64_CHARS.charAt((byte) ((n >>> 18) & 0x3F)));
+      builder.append(BASE_64_CHARS.charAt((byte) ((n >>> 12) & 0x3F)));
+      builder.append(BASE_64_CHARS.charAt((byte) ((n >>> 6) & 0x3F)));
+      builder.append(BASE_64_CHARS.charAt((byte) (n & 0x3F)));
+    }
+
+    final int bLength = builder.length();
+
+    // append '=' to pad
+    if (data.length % 3 == 1) {
+      builder.replace(bLength - 2, bLength, "==");
+    }
+    else if (data.length % 3 == 2) {
+      builder.replace(bLength - 1, bLength, "=");
+    }
+
+    return builder.toString();
+  }
+
+  /**
+   * Encodes a byte array as a Base64 string.
+   *
+   * @param data
+   *            the byte array to encode
+   * @return the Base64-encoded string, as a <code>String</code> object
+   */
+  public static String encode(final Byte[] data) {
+    final StringBuilder builder = new StringBuilder();
+    final int dataRemainder = data.length % 3;
+
+    int j = 0;
+    int n = 0;
+    for (; j < data.length; j += 3) {
+
+      if (j < data.length - dataRemainder) {
+        n = ((data[j] & 0xFF) << 16) + ((data[j + 1] & 0xFF) << 8) + (data[j + 2] & 0xFF);
+      }
+      else {
+        if (dataRemainder == 1) {
+          n = (data[j] & 0xFF) << 16;
+        }
+        else if (dataRemainder == 2) {
+          n = ((data[j] & 0xFF) << 16) + ((data[j + 1] & 0xFF) << 8);
+        }
+      }
+
+      // Left here for readability
+      // byte char1 = (byte) ((n >>> 18) & 0x3F);
+      // byte char2 = (byte) ((n >>> 12) & 0x3F);
+      // byte char3 = (byte) ((n >>> 6) & 0x3F);
+      // byte char4 = (byte) (n & 0x3F);
+      builder.append(BASE_64_CHARS.charAt((byte) ((n >>> 18) & 0x3F)));
+      builder.append(BASE_64_CHARS.charAt((byte) ((n >>> 12) & 0x3F)));
+      builder.append(BASE_64_CHARS.charAt((byte) ((n >>> 6) & 0x3F)));
+      builder.append(BASE_64_CHARS.charAt((byte) (n & 0x3F)));
+    }
+
+    final int bLength = builder.length();
+
+    // append '=' to pad
+    if (data.length % 3 == 1) {
+      builder.replace(bLength - 2, bLength, "==");
+    }
+    else if (data.length % 3 == 2) {
+      builder.replace(bLength - 1, bLength, "=");
+    }
+
+    return builder.toString();
+  }
+
+  /**
+   * Determines whether the given string contains only Base64 characters.
+   *
+   * @param data
+   *            the string, as a <code>String</code> object, to validate
+   * @return <code>true</code> if <code>data</code> is a valid Base64 string, otherwise <code>false</code>
+   */
+  public static boolean validateIsBase64String(final String data) {
+
+    if (data == null || data.length() % 4 != 0) {
+      return false;
+    }
+
+    for (int m = 0; m < data.length(); m++) {
+      final byte charByte = (byte) data.charAt(m);
+
+      // pad char detected
+      if (DECODE_64[charByte] == -2) {
+        if (m < data.length() - 2) {
+          return false;
+        }
+        else if (m == data.length() - 2 && DECODE_64[(byte) data.charAt(m + 1)] != -2) {
+          return false;
+        }
+      }
+
+      if (charByte < 0 || DECODE_64[charByte] == -1) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  /**
+   * Private Default Ctor.
+   */
+  private Base64() {
+    // No op
+  }
+}

Một số tệp đã không được hiển thị bởi vì quá nhiều tập tin thay đổi trong này khác