You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ma...@apache.org on 2018/09/23 03:24:24 UTC

[27/45] hadoop git commit: HADOOP-15660. ABFS: Add support for OAuth Contributed by Da Zhou, Rajeev Bansal, and Junhua Gu.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
index b1f1485..e0afeb4 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
 import org.apache.hadoop.fs.azure.AbstractWasbTestWithTimeout;
 import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore;
 import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
@@ -62,7 +63,7 @@ public abstract class AbstractAbfsIntegrationTest extends
   private static final Logger LOG =
       LoggerFactory.getLogger(AbstractAbfsIntegrationTest.class);
 
-  private final boolean isEmulator;
+  private boolean isEmulator;
   private NativeAzureFileSystem wasb;
   private AzureBlobFileSystem abfs;
   private String abfsScheme;
@@ -71,20 +72,18 @@ public abstract class AbstractAbfsIntegrationTest extends
   private String fileSystemName;
   private String accountName;
   private String testUrl;
-
-  protected AbstractAbfsIntegrationTest(final boolean secure) {
-    this(secure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME);
-  }
+  private AuthType authType;
 
   protected AbstractAbfsIntegrationTest() {
-    this(FileSystemUriSchemes.ABFS_SCHEME);
-  }
-
-  private AbstractAbfsIntegrationTest(final String scheme) {
-    abfsScheme = scheme;
     fileSystemName = ABFS_TEST_CONTAINER_PREFIX + UUID.randomUUID().toString();
     configuration = new Configuration();
     configuration.addResource(ABFS_TEST_RESOURCE_XML);
+    this.accountName = this.configuration.get(FS_AZURE_TEST_ACCOUNT_NAME);
+
+    authType = configuration.getEnum(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME
+            + accountName, AuthType.SharedKey);
+    abfsScheme = authType == AuthType.SharedKey ? FileSystemUriSchemes.ABFS_SCHEME
+            : FileSystemUriSchemes.ABFS_SECURE_SCHEME;
 
     String accountName = configuration.get(FS_AZURE_TEST_ACCOUNT_NAME, "");
     assumeTrue("Not set: " + FS_AZURE_TEST_ACCOUNT_NAME,
@@ -94,8 +93,13 @@ public abstract class AbstractAbfsIntegrationTest extends
         accountName, containsString("dfs.core.windows.net"));
     String fullKey = FS_AZURE_TEST_ACCOUNT_KEY_PREFIX
         + accountName;
-    assumeTrue("Not set: " + fullKey,
-        configuration.get(fullKey) != null);
+
+    if (authType == AuthType.SharedKey) {
+      assumeTrue("Not set: " + fullKey, configuration.get(fullKey) != null);
+    } else {
+      String accessTokenProviderKey = FS_AZURE_ACCOUNT_TOKEN_PROVIDER_TYPE_PROPERTY_NAME + accountName;
+      assumeTrue("Not set: " + accessTokenProviderKey, configuration.get(accessTokenProviderKey) != null);
+    }
 
     final String abfsUrl = this.getFileSystemName() + "@" + this.getAccountName();
     URI defaultUri = null;
@@ -110,7 +114,6 @@ public abstract class AbstractAbfsIntegrationTest extends
     configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, defaultUri.toString());
     configuration.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true);
     this.isEmulator = this.configuration.getBoolean(FS_AZURE_EMULATOR_ENABLED, false);
-    this.accountName = this.configuration.get(FS_AZURE_TEST_ACCOUNT_NAME);
   }
 
 
@@ -119,7 +122,7 @@ public abstract class AbstractAbfsIntegrationTest extends
     //Create filesystem first to make sure getWasbFileSystem() can return an existing filesystem.
     createFileSystem();
 
-    if (!isEmulator) {
+    if (!isEmulator && authType == AuthType.SharedKey) {
       final URI wasbUri = new URI(abfsUrlToWasbUrl(getTestUrl()));
       final AzureNativeFileSystemStore azureNativeFileSystemStore =
           new AzureNativeFileSystemStore();
@@ -234,6 +237,10 @@ public abstract class AbstractAbfsIntegrationTest extends
     return isEmulator;
   }
 
+  protected AuthType getAuthType() {
+    return this.authType;
+  }
+
   /**
    * Write a buffer to a file.
    * @param path path

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java
index 9c369bb..1c2083d 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsClient.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.fs.azurebfs;
 
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
-import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
 import org.apache.hadoop.fs.azurebfs.services.AbfsClient;
 import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation;
 import org.junit.Assert;
@@ -30,6 +29,7 @@ import org.junit.Test;
  */
 public final class ITestAbfsClient extends AbstractAbfsIntegrationTest {
   private static final int LIST_MAX_RESULTS = 5000;
+
   @Test
   public void testContinuationTokenHavingEqualSign() throws Exception {
     final AzureBlobFileSystem fs = this.getFileSystem();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java
index d696481..6207a47 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemBackCompat.java
@@ -22,8 +22,11 @@ import com.microsoft.azure.storage.CloudStorageAccount;
 import com.microsoft.azure.storage.blob.CloudBlobClient;
 import com.microsoft.azure.storage.blob.CloudBlobContainer;
 import com.microsoft.azure.storage.blob.CloudBlockBlob;
+
+import org.junit.Assume;
 import org.junit.Test;
 
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 
@@ -34,6 +37,7 @@ public class ITestAzureBlobFileSystemBackCompat extends
     AbstractAbfsIntegrationTest {
   public ITestAzureBlobFileSystemBackCompat() {
     super();
+    Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java
index 791694b..13abaf8 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFileStatus.java
@@ -34,9 +34,6 @@ public class ITestAzureBlobFileSystemFileStatus extends
     AbstractAbfsIntegrationTest {
   private static final Path TEST_FILE = new Path("testFile");
   private static final Path TEST_FOLDER = new Path("testDir");
-  public ITestAzureBlobFileSystemFileStatus() {
-    super();
-  }
 
   @Test
   public void testEnsureStatusWorksForRoot() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java
index e4acbae..c1022b0 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFinalize.java
@@ -25,12 +25,14 @@ import org.junit.Test;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
 
 /**
  * Test finalize() method when "fs.abfs.impl.disable.cache" is enabled.
  */
 public class ITestAzureBlobFileSystemFinalize extends AbstractAbfsScaleTest{
-  static final String DISABLE_CACHE_KEY = "fs.abfs.impl.disable.cache";
+  static final String DISABLE_ABFS_CACHE_KEY = "fs.abfs.impl.disable.cache";
+  static final String DISABLE_ABFSSS_CACHE_KEY = "fs.abfss.impl.disable.cache";
 
   public ITestAzureBlobFileSystemFinalize() throws Exception {
     super();
@@ -40,7 +42,9 @@ public class ITestAzureBlobFileSystemFinalize extends AbstractAbfsScaleTest{
   public void testFinalize() throws Exception {
     // Disable the cache for filesystem to make sure there is no reference.
     Configuration configuration = this.getConfiguration();
-    configuration.setBoolean(this.DISABLE_CACHE_KEY, true);
+    configuration.setBoolean(
+            this.getAuthType() == AuthType.SharedKey ? DISABLE_ABFS_CACHE_KEY : DISABLE_ABFSSS_CACHE_KEY,
+    true);
 
     AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(configuration);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java
index 2f40b64..b02d723 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemFlush.java
@@ -31,9 +31,9 @@ import java.io.IOException;
 import com.microsoft.azure.storage.blob.BlockEntry;
 import com.microsoft.azure.storage.blob.BlockListingFilter;
 import com.microsoft.azure.storage.blob.CloudBlockBlob;
-import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
 import org.hamcrest.core.IsEqual;
 import org.hamcrest.core.IsNot;
+import org.junit.Assume;
 import org.junit.Test;
 
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -42,6 +42,9 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
+import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+
 /**
  * Test flush operation.
  */
@@ -209,6 +212,8 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest {
 
   @Test
   public void testFlushWithFlushEnabled() throws Exception {
+    Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey);
+
     AzureBlobStorageTestAccount testAccount = createWasbTestAccount();
     String wasbUrl = testAccount.getFileSystem().getName();
     String abfsUrl = wasbUrlToAbfsUrl(wasbUrl);
@@ -228,6 +233,7 @@ public class ITestAzureBlobFileSystemFlush extends AbstractAbfsScaleTest {
 
   @Test
   public void testFlushWithFlushDisabled() throws Exception {
+    Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey);
     AzureBlobStorageTestAccount testAccount = createWasbTestAccount();
     String wasbUrl = testAccount.getFileSystem().getName();
     String abfsUrl = wasbUrlToAbfsUrl(wasbUrl);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java
new file mode 100644
index 0000000..f60740f
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemOauth.java
@@ -0,0 +1,176 @@
+/**
+ * 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.io.InputStream;
+import java.util.Map;
+
+import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
+import org.junit.Assume;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
+import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID;
+import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_ID;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_SECRET;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_DATA_READER_CLIENT_ID;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET;
+
+/**
+ * Test Azure Oauth with Blob Data contributor role and Blob Data Reader role.
+ * The Test AAD client need to be configured manually through Azure Portal, then save their properties in
+ * configuration files.
+ */
+public class ITestAzureBlobFileSystemOauth extends AbstractAbfsIntegrationTest{
+
+  private static final Path FILE_PATH = new Path("/testFile");
+  private static final Path EXISTED_FILE_PATH = new Path("/existedFile");
+  private static final Path EXISTED_FOLDER_PATH = new Path("/existedFolder");
+
+  public ITestAzureBlobFileSystemOauth() {
+    Assume.assumeTrue(this.getAuthType() == AuthType.OAuth);
+  }
+  /*
+  * BLOB DATA CONTRIBUTOR should have full access to the container and blobs in the container.
+  * */
+  @Test
+  public void testBlobDataContributor() throws Exception {
+    String clientId = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_ID);
+    Assume.assumeTrue("Contributor client id not provided", clientId != null);
+    String secret = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_SECRET);
+    Assume.assumeTrue("Contributor client secret not provided", secret != null);
+
+    prepareFiles();
+
+    final AzureBlobFileSystem fs = getBlobConributor();
+
+    // create and write into file in current container/fs
+    try(FSDataOutputStream stream = fs.create(FILE_PATH)) {
+      stream.write(0);
+    }
+    assertTrue(fs.exists(FILE_PATH));
+    FileStatus fileStatus = fs.getFileStatus(FILE_PATH);
+    assertEquals(1, fileStatus.getLen());
+    // delete file
+    assertTrue(fs.delete(FILE_PATH, true));
+    assertFalse(fs.exists(FILE_PATH));
+
+    // Verify Blob Data Contributor has full access to existed folder, file
+
+    // READ FOLDER
+    assertTrue(fs.exists(EXISTED_FOLDER_PATH));
+
+    //DELETE FOLDER
+    fs.delete(EXISTED_FOLDER_PATH, true);
+    assertFalse(fs.exists(EXISTED_FOLDER_PATH));
+
+    // READ FILE
+    try (FSDataInputStream stream = fs.open(EXISTED_FILE_PATH)) {
+      assertTrue(stream.read() != 0);
+    }
+
+    assertEquals(0, fs.getFileStatus(EXISTED_FILE_PATH).getLen());
+
+    // WRITE FILE
+    try (FSDataOutputStream stream = fs.append(EXISTED_FILE_PATH)) {
+      stream.write(0);
+    }
+
+    assertEquals(1, fs.getFileStatus(EXISTED_FILE_PATH).getLen());
+
+    // REMOVE FILE
+    fs.delete(EXISTED_FILE_PATH, true);
+    assertFalse(fs.exists(EXISTED_FILE_PATH));
+  }
+
+  /*
+   * BLOB DATA READER should have only READ access to the container and blobs in the container.
+   * */
+  @Test
+  public void testBlobDataReader() throws Exception {
+    String clientId = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_READER_CLIENT_ID);
+    Assume.assumeTrue("Reader client id not provided", clientId != null);
+    String secret = this.getConfiguration().get(TestConfigurationKeys.FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET);
+    Assume.assumeTrue("Reader client secret not provided", secret != null);
+
+    prepareFiles();
+    final AzureBlobFileSystem fs = getBlobReader();
+
+    // Use abfsStore in this test to verify the  ERROR code in AbfsRestOperationException
+    AzureBlobFileSystemStore abfsStore = fs.getAbfsStore();
+    // TEST READ FS
+    Map<String, String> properties = abfsStore.getFilesystemProperties();
+    // TEST READ FOLDER
+    assertTrue(fs.exists(EXISTED_FOLDER_PATH));
+
+    // TEST DELETE FOLDER
+    try {
+      abfsStore.delete(EXISTED_FOLDER_PATH, true);
+    } catch (AbfsRestOperationException e) {
+      assertEquals(AzureServiceErrorCode.AUTHORIZATION_PERMISSION_MISS_MATCH, e.getErrorCode());
+    }
+
+    // TEST READ  FILE
+    try (InputStream inputStream = abfsStore.openFileForRead(EXISTED_FILE_PATH, null)) {
+      assertTrue(inputStream.read() != 0);
+    }
+
+    // TEST WRITE FILE
+    try {
+      abfsStore.openFileForWrite(EXISTED_FILE_PATH, true);
+    } catch (AbfsRestOperationException e) {
+      assertEquals(AzureServiceErrorCode.AUTHORIZATION_PERMISSION_MISS_MATCH, e.getErrorCode());
+    }
+
+  }
+
+  private void prepareFiles() throws IOException {
+    // create test files/folders to verify access control diff between
+    // Blob data contributor and Blob data reader
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.create(EXISTED_FILE_PATH);
+    assertTrue(fs.exists(EXISTED_FILE_PATH));
+    fs.mkdirs(EXISTED_FOLDER_PATH);
+    assertTrue(fs.exists(EXISTED_FOLDER_PATH));
+  }
+
+  private AzureBlobFileSystem getBlobConributor() throws Exception {
+    Configuration configuration = this.getConfiguration();
+    configuration.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID + this.getAccountName(), configuration.get(FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_ID));
+    configuration.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET + this.getAccountName(), configuration.get(FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_SECRET));
+    return getFileSystem(configuration);
+  }
+
+  private AzureBlobFileSystem getBlobReader() throws Exception {
+    Configuration configuration = this.getConfiguration();
+    configuration.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_ID + this.getAccountName(), configuration.get(FS_AZURE_BLOB_DATA_READER_CLIENT_ID));
+    configuration.set(FS_AZURE_ACCOUNT_OAUTH_CLIENT_SECRET + this.getAccountName(), configuration.get(FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET));
+    return getFileSystem(configuration);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
index c61de67..13c5bc8 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.util.Random;
 import java.util.concurrent.Callable;
 
+import org.junit.Assume;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -34,6 +35,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -66,6 +68,7 @@ public class ITestAzureBlobFileSystemRandomRead extends
 
   public ITestAzureBlobFileSystemRandomRead() throws Exception {
     super();
+    Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java
index 3a44909..50b1828 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemInitialization.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
 
 /**
  * Test AzureBlobFileSystem initialization.
@@ -41,8 +42,10 @@ public class ITestFileSystemInitialization extends AbstractAbfsIntegrationTest {
     final String accountName = getAccountName();
     final String filesystem = getFileSystemName();
 
+    String scheme = this.getAuthType() == AuthType.SharedKey ? FileSystemUriSchemes.ABFS_SCHEME
+            : FileSystemUriSchemes.ABFS_SECURE_SCHEME;
     assertEquals(fs.getUri(),
-        new URI(FileSystemUriSchemes.ABFS_SCHEME,
+        new URI(scheme,
             filesystem + "@" + accountName,
             null,
             null,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java
index 5d1cf91..56a91d3 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestFileSystemRegistration.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
 
 /**
  * Test AzureBlobFileSystem registration.
@@ -79,8 +80,14 @@ public class ITestFileSystemRegistration extends AbstractAbfsIntegrationTest {
     AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.get(getConfiguration());
     assertNotNull("filesystem", fs);
 
-    Abfs afs = (Abfs) FileContext.getFileContext(getConfiguration()).getDefaultFileSystem();
-    assertNotNull("filecontext", afs);
+    if (this.getAuthType() == AuthType.OAuth) {
+      Abfss afs = (Abfss) FileContext.getFileContext(getConfiguration()).getDefaultFileSystem();
+      assertNotNull("filecontext", afs);
+    } else {
+      Abfs afs = (Abfs) FileContext.getFileContext(getConfiguration()).getDefaultFileSystem();
+      assertNotNull("filecontext", afs);
+    }
+
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
index a89c044..ff28d3e 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestWasbAbfsCompatibility.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory;
@@ -50,6 +51,7 @@ public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
 
   public ITestWasbAbfsCompatibility() throws Exception {
     Assume.assumeFalse("Emulator is not supported", isEmulator());
+    Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java
index fc7312a..67301c7 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java
@@ -28,6 +28,12 @@ public final class TestConfigurationKeys {
   public static final String FS_AZURE_TEST_HOST_PORT = "fs.azure.test.host.port";
   public static final String FS_AZURE_CONTRACT_TEST_URI = "fs.contract.test.fs.abfs";
 
+  public static final String FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_ID = "fs.azure.account.oauth2.contributor.client.id";
+  public static final String FS_AZURE_BLOB_DATA_CONTRIBUTOR_CLIENT_SECRET = "fs.azure.account.oauth2.contributor.client.secret";
+
+  public static final String FS_AZURE_BLOB_DATA_READER_CLIENT_ID = "fs.azure.account.oauth2.reader.client.id";
+  public static final String FS_AZURE_BLOB_DATA_READER_CLIENT_SECRET = "fs.azure.account.oauth2.reader.client.secret";
+
   public static final String ABFS_TEST_RESOURCE_XML = "azure-bfs-test.xml";
 
   public static final String ABFS_TEST_CONTAINER_PREFIX = "abfs-testcontainer-";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java
index ffd5bab..5505e6a 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
 import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
 import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
 
 /**
  * Bind ABFS contract tests to the Azure test setup/teardown.
@@ -32,18 +33,17 @@ import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
 public class ABFSContractTestBinding extends AbstractAbfsIntegrationTest {
   private final URI testUri;
 
-  public ABFSContractTestBinding(final boolean secure) throws Exception {
-    this(secure, true);
+  public ABFSContractTestBinding() throws Exception {
+    this(true);
   }
 
-  public ABFSContractTestBinding(final boolean secure,
+  public ABFSContractTestBinding(
       final boolean useExistingFileSystem) throws Exception{
-    super(secure);
     if (useExistingFileSystem) {
       Configuration configuration = getConfiguration();
       String testUrl = configuration.get(TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI);
 
-      if (secure) {
+      if (getAuthType() != AuthType.SharedKey) {
         testUrl = testUrl.replaceFirst(FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME);
       }
       setTestUrl(testUrl);
@@ -61,4 +61,8 @@ public class ABFSContractTestBinding extends AbstractAbfsIntegrationTest {
   public Configuration getConfiguration() {
     return super.getConfiguration();
   }
+
+  public boolean isSecureMode() {
+    return this.getAuthType() == AuthType.SharedKey ? false : true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java
index a302fcc..8a955bc 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractAppend.java
@@ -18,34 +18,23 @@
 
 package org.apache.hadoop.fs.azurebfs.contract;
 
-import java.util.Arrays;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractAppendTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.Test;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
 
 /**
  * Contract test for open operation.
  */
-@RunWith(Parameterized.class)
 public class ITestAbfsFileSystemContractAppend extends AbstractContractAppendTest {
-  @Parameterized.Parameters(name = "SecureMode={0}")
-  public static Iterable<Object[]> secure() {
-    return Arrays.asList(new Object[][] { {true}, {false} });
-  }
-
   private final boolean isSecure;
   private final ABFSContractTestBinding binding;
 
-  public ITestAbfsFileSystemContractAppend(final boolean secure) throws Exception {
-    this.isSecure = secure;
-    binding = new ABFSContractTestBinding(this.isSecure);
+  public ITestAbfsFileSystemContractAppend() throws Exception {
+    binding = new ABFSContractTestBinding();
+    this.isSecure = binding.isSecureMode();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java
index c31a6d2..383528b 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractConcat.java
@@ -17,11 +17,6 @@
  */
 package org.apache.hadoop.fs.azurebfs.contract;
 
-import java.util.Arrays;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractConcatTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
@@ -29,19 +24,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract;
 /**
  * Contract test for concat operation.
  */
-@RunWith(Parameterized.class)
 public class ITestAbfsFileSystemContractConcat extends AbstractContractConcatTest{
-  @Parameterized.Parameters(name = "SecureMode={0}")
-  public static Iterable<Object[]> secure() {
-    return Arrays.asList(new Object[][] { {true}, {false} });
-  }
-
   private final boolean isSecure;
   private final ABFSContractTestBinding binding;
 
-  public ITestAbfsFileSystemContractConcat(final boolean secure) throws Exception {
-    isSecure = secure;
-    binding = new ABFSContractTestBinding(isSecure);
+  public ITestAbfsFileSystemContractConcat() throws Exception {
+    binding = new ABFSContractTestBinding();
+    this.isSecure = binding.isSecureMode();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java
index ce4d229..3c3e949 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractCreate.java
@@ -18,11 +18,6 @@
 
 package org.apache.hadoop.fs.azurebfs.contract;
 
-import java.util.Arrays;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
@@ -30,19 +25,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract;
 /**
  * Contract test for create operation.
  */
-@RunWith(Parameterized.class)
 public class ITestAbfsFileSystemContractCreate extends AbstractContractCreateTest{
-  @Parameterized.Parameters(name = "SecureMode={0}")
-  public static Iterable<Object[]> secure() {
-    return Arrays.asList(new Object[][] { {true}, {false} });
-  }
-
   private final boolean isSecure;
   private final ABFSContractTestBinding binding;
 
-  public ITestAbfsFileSystemContractCreate(final boolean secure) throws Exception {
-    this.isSecure = secure;
-    binding = new ABFSContractTestBinding(this.isSecure);
+  public ITestAbfsFileSystemContractCreate() throws Exception {
+    binding = new ABFSContractTestBinding();
+    this.isSecure = binding.isSecureMode();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java
index 310731c..1d1136c 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDelete.java
@@ -18,11 +18,6 @@
 
 package org.apache.hadoop.fs.azurebfs.contract;
 
-import java.util.Arrays;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
@@ -30,19 +25,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract;
 /**
  * Contract test for delete operation.
  */
-@RunWith(Parameterized.class)
 public class ITestAbfsFileSystemContractDelete extends AbstractContractDeleteTest {
-  @Parameterized.Parameters(name = "SecureMode={0}")
-  public static Iterable<Object[]> secure() {
-    return Arrays.asList(new Object[][] { {true}, {false} });
-  }
-
   private final boolean isSecure;
   private final ABFSContractTestBinding binding;
 
-  public ITestAbfsFileSystemContractDelete(final boolean secure) throws Exception {
-    this.isSecure = secure;
-    binding = new ABFSContractTestBinding(isSecure);
+  public ITestAbfsFileSystemContractDelete() throws Exception {
+    binding = new ABFSContractTestBinding();
+    this.isSecure = binding.isSecureMode();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java
index c2cf255..544bbbf 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractDistCp.java
@@ -28,7 +28,7 @@ public class ITestAbfsFileSystemContractDistCp extends AbstractContractDistCpTes
   private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractDistCp() throws Exception {
-    binding = new ABFSContractTestBinding(false);
+    binding = new ABFSContractTestBinding();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java
index 9ad3b21..08b7eef 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractGetFileStatus.java
@@ -17,11 +17,6 @@
  */
 package org.apache.hadoop.fs.azurebfs.contract;
 
-import java.util.Arrays;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
@@ -29,19 +24,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract;
 /**
  * Contract test for getFileStatus operation.
  */
-@RunWith(Parameterized.class)
 public class ITestAbfsFileSystemContractGetFileStatus extends AbstractContractGetFileStatusTest {
-  @Parameterized.Parameters(name = "SecureMode={0}")
-  public static Iterable<Object[]> secure() {
-    return Arrays.asList(new Object[][] { {true}, {false} });
-  }
-
   private final boolean isSecure;
   private final ABFSContractTestBinding binding;
 
-  public ITestAbfsFileSystemContractGetFileStatus(final boolean secure) throws Exception {
-    this.isSecure = secure;
-    binding = new ABFSContractTestBinding(isSecure);
+  public ITestAbfsFileSystemContractGetFileStatus() throws Exception {
+    binding = new ABFSContractTestBinding();
+    this.isSecure = binding.isSecureMode();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java
index 6265ca1..7b78575 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractMkdir.java
@@ -18,11 +18,6 @@
 
 package org.apache.hadoop.fs.azurebfs.contract;
 
-import java.util.Arrays;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
@@ -30,19 +25,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract;
 /**
  * Contract test for mkdir operation.
  */
-@RunWith(Parameterized.class)
 public class ITestAbfsFileSystemContractMkdir extends AbstractContractMkdirTest {
-  @Parameterized.Parameters(name = "SecureMode={0}")
-  public static Iterable<Object[]> secure() {
-    return Arrays.asList(new Object[][] { {true}, {false} });
-  }
-
   private final boolean isSecure;
   private final ABFSContractTestBinding binding;
 
-  public ITestAbfsFileSystemContractMkdir(final boolean secure) throws Exception {
-    this.isSecure = secure;
-    binding = new ABFSContractTestBinding(secure);
+  public ITestAbfsFileSystemContractMkdir() throws Exception {
+    binding = new ABFSContractTestBinding();
+    this.isSecure = binding.isSecureMode();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java
index ae4bb2a..41f691d 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractOpen.java
@@ -18,11 +18,6 @@
 
 package org.apache.hadoop.fs.azurebfs.contract;
 
-import java.util.Arrays;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
@@ -30,19 +25,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract;
 /**
  * Contract test for open operation.
  */
-@RunWith(Parameterized.class)
 public class ITestAbfsFileSystemContractOpen extends AbstractContractOpenTest {
-  @Parameterized.Parameters(name = "SecureMode={0}")
-  public static Iterable<Object[]> secure() {
-    return Arrays.asList(new Object[][] { {true}, {false} });
-  }
-
   private final boolean isSecure;
   private final ABFSContractTestBinding binding;
 
-  public ITestAbfsFileSystemContractOpen(final boolean secure) throws Exception {
-    this.isSecure = secure;
-    binding = new ABFSContractTestBinding(this.isSecure);
+  public ITestAbfsFileSystemContractOpen() throws Exception {
+    binding = new ABFSContractTestBinding();
+    this.isSecure = binding.isSecureMode();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java
index 6e6a728..82f104a 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRename.java
@@ -18,11 +18,6 @@
 
 package org.apache.hadoop.fs.azurebfs.contract;
 
-import java.util.Arrays;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
@@ -30,19 +25,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract;
 /**
  * Contract test for rename operation.
  */
-@RunWith(Parameterized.class)
 public class ITestAbfsFileSystemContractRename extends AbstractContractRenameTest {
-  @Parameterized.Parameters(name = "SecureMode={0}")
-  public static Iterable<Object[]> secure() {
-    return Arrays.asList(new Object[][] { {true}, {false} });
-  }
-
   private final boolean isSecure;
   private final ABFSContractTestBinding binding;
 
-  public ITestAbfsFileSystemContractRename(final boolean secure) throws Exception {
-    this.isSecure = secure;
-    binding = new ABFSContractTestBinding(this.isSecure);
+  public ITestAbfsFileSystemContractRename() throws Exception {
+    binding = new ABFSContractTestBinding();
+    this.isSecure = binding.isSecureMode();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java
index 01dea2d..5b5493f 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractRootDirectory.java
@@ -17,31 +17,21 @@
  */
 package org.apache.hadoop.fs.azurebfs.contract;
 
-import java.util.Arrays;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 import org.junit.Ignore;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 
 /**
  * Contract test for root directory operation.
  */
-@RunWith(Parameterized.class)
 public class ITestAbfsFileSystemContractRootDirectory extends AbstractContractRootDirectoryTest {
-  @Parameterized.Parameters(name = "SecureMode={0}")
-  public static Iterable<Object[]> secure() {
-    return Arrays.asList(new Object[][] { {true}, {false} });
-  }
-
   private final boolean isSecure;
   private final ABFSContractTestBinding binding;
 
-  public ITestAbfsFileSystemContractRootDirectory(final boolean secure) throws Exception {
-    this.isSecure = secure;
-    binding = new ABFSContractTestBinding(secure);
+  public ITestAbfsFileSystemContractRootDirectory() throws Exception {
+    binding = new ABFSContractTestBinding();
+    this.isSecure = binding.isSecureMode();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java
index 5ed7466..fc235e3 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSecureDistCp.java
@@ -28,7 +28,7 @@ public class ITestAbfsFileSystemContractSecureDistCp extends AbstractContractDis
   private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractSecureDistCp() throws Exception {
-    binding = new ABFSContractTestBinding(true);
+    binding = new ABFSContractTestBinding();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java
index 5e0ea0c..4529e75 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSeek.java
@@ -18,11 +18,6 @@
 
 package org.apache.hadoop.fs.azurebfs.contract;
 
-import java.util.Arrays;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
@@ -30,19 +25,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract;
 /**
  * Contract test for seek operation.
  */
-@RunWith(Parameterized.class)
 public class ITestAbfsFileSystemContractSeek extends AbstractContractSeekTest{
-  @Parameterized.Parameters(name = "SecureMode={0}")
-  public static Iterable<Object[]> secure() {
-    return Arrays.asList(new Object[][] { {true}, {false} });
-  }
-
   private final boolean isSecure;
   private final ABFSContractTestBinding binding;
 
-  public ITestAbfsFileSystemContractSeek(final boolean secure) throws Exception {
-    this.isSecure = secure;
-    binding = new ABFSContractTestBinding(this.isSecure);
+  public ITestAbfsFileSystemContractSeek() throws Exception {
+    binding = new ABFSContractTestBinding();
+    this.isSecure = binding.isSecureMode();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java
index 8d23b0b..6c4f9ba 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContractSetTimes.java
@@ -17,11 +17,6 @@
  */
 package org.apache.hadoop.fs.azurebfs.contract;
 
-import java.util.Arrays;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractSetTimesTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
@@ -29,19 +24,13 @@ import org.apache.hadoop.fs.contract.AbstractFSContract;
 /**
  * Contract test for setTimes operation.
  */
-@RunWith(Parameterized.class)
 public class ITestAbfsFileSystemContractSetTimes extends AbstractContractSetTimesTest {
-  @Parameterized.Parameters(name = "SecureMode={0}")
-  public static Iterable<Object[]> secure() {
-    return Arrays.asList(new Object[][] { {true}, {false} });
-  }
-
   private final boolean isSecure;
   private final ABFSContractTestBinding binding;
 
-  public ITestAbfsFileSystemContractSetTimes(final boolean secure) throws Exception {
-    this.isSecure = secure;
-    binding = new ABFSContractTestBinding(this.isSecure);
+  public ITestAbfsFileSystemContractSetTimes() throws Exception {
+    binding = new ABFSContractTestBinding();
+    this.isSecure = binding.isSecureMode();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
index d8854a2..a9fa2d7 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAzureBlobFileSystemBasics.java
@@ -40,7 +40,7 @@ public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest {
   public ITestAzureBlobFileSystemBasics() throws Exception {
     // If all contract tests are running in parallel, some root level tests in FileSystemContractBaseTest will fail
     // due to the race condition. Hence for this contract test it should be tested in different container
-    binding = new ABFSContractTestBinding(false, false);
+    binding = new ABFSContractTestBinding(false);
   }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java
index 7bb27fc..a2fdd09 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java
@@ -21,11 +21,11 @@ package org.apache.hadoop.fs.azurebfs.services;
 import java.net.URL;
 import java.util.regex.Pattern;
 
-import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
-import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.hadoop.fs.azurebfs.AbfsConfiguration;
+import org.apache.hadoop.fs.azurebfs.utils.SSLSocketFactoryEx;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys;
 
@@ -40,7 +40,7 @@ public final class TestAbfsClient {
                                  AbfsConfiguration config,
                                  boolean includeSSLProvider) {
     AbfsClient client = new AbfsClient(baseUrl, null,
-        config, null);
+        config, null, null);
     String sslProviderName = null;
     if (includeSSLProvider) {
       sslProviderName = SSLSocketFactoryEx.getDefaultFactory().getProviderName();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestQueryParams.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestQueryParams.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestQueryParams.java
new file mode 100644
index 0000000..e6c6993
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestQueryParams.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.HashMap;
+import java.util.Map;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.azurebfs.oauth2.QueryParams;
+/**
+ * Test query params serialization.
+ */
+public class TestQueryParams {
+  private static final String SEPARATOR = "&";
+  private static final String[][] PARAM_ARRAY = {{"K0", "V0"}, {"K1", "V1"}, {"K2", "V2"}};
+
+  @Test
+  public void testOneParam() {
+    String key = PARAM_ARRAY[0][0];
+    String value = PARAM_ARRAY[0][1];
+
+    Map<String, String> paramMap = new HashMap<>();
+    paramMap.put(key, value);
+
+    QueryParams qp = new QueryParams();
+    qp.add(key, value);
+    Assert.assertEquals(key + "=" + value, qp.serialize());
+  }
+
+  @Test
+  public void testMultipleParams() {
+    QueryParams qp = new QueryParams();
+    for (String[] entry : PARAM_ARRAY) {
+      qp.add(entry[0], entry[1]);
+    }
+    Map<String, String> paramMap = constructMap(qp.serialize());
+    Assert.assertEquals(PARAM_ARRAY.length, paramMap.size());
+
+    for (String[] entry : PARAM_ARRAY) {
+      Assert.assertTrue(paramMap.containsKey(entry[0]));
+      Assert.assertEquals(entry[1], paramMap.get(entry[0]));
+    }
+  }
+
+  private Map<String, String> constructMap(String input) {
+    String[] entries = input.split(SEPARATOR);
+    Map<String, String> paramMap = new HashMap<>();
+    for (String entry : entries) {
+      String[] keyValue = entry.split("=");
+      paramMap.put(keyValue[0], keyValue[1]);
+    }
+    return paramMap;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java
index 9051a72..ef4ddb9 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/CleanUpAbfsTestContainer.java
@@ -21,9 +21,13 @@ import com.microsoft.azure.storage.CloudStorageAccount;
 import com.microsoft.azure.storage.blob.CloudBlobClient;
 import com.microsoft.azure.storage.blob.CloudBlobContainer;
 
-import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
+import org.junit.Assume;
 import org.junit.Test;
 
+import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.services.AuthType;
+
 import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.ABFS_TEST_CONTAINER_PREFIX;
 
 /**
@@ -31,7 +35,12 @@ import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.ABFS
  * In that case, dev can use this tool to list and delete all test containers.
  * By default, all test container used in E2E tests sharing same prefix: "abfs-testcontainer-"
  */
-public class CleanUpAbfsTestContainer {
+public class CleanUpAbfsTestContainer extends AbstractAbfsIntegrationTest{
+
+  public CleanUpAbfsTestContainer() {
+    Assume.assumeTrue(this.getAuthType() == AuthType.SharedKey);
+  }
+
   @Test
   public void testEnumContainers() throws Throwable {
     int count = 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9149b970/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml b/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml
index 464a8e6..9b908b0 100644
--- a/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml
+++ b/hadoop-tools/hadoop-azure/src/test/resources/azure-bfs-test.xml
@@ -23,6 +23,27 @@
   </property>
 
   <property>
+    <name>fs.azure.account.auth.type.{YOURACCOUNT}.dfs.core.windows.net</name>
+    <value>{AUTH TYPE}</value>
+    <description>The auth type can be : SharedKey, OAuth, Custom. By default "SharedKey" is used.</description>
+  </property>
+
+   <property>
+    <name>fs.contract.test.fs.abfs</name>
+    <value>abfs://{CONTAINERNAME}@{ACCOUNTNAME}.dfs.core.windows.net/value>
+    <description>The name of the azure file system for testing.</description>
+  </property>
+
+  <property>
+    <name>fs.contract.test.fs.abfss</name>
+    <value>abfss://{CONTAINERNAME}@{ACCOUNTNAME}.dfs.core.windows.net/value>
+    <description>The name of the azure file system for testing.</description>
+  </property>
+  -->
+
+  <!--If auth type is "SharedKey", provide SharedKey credentials below -->
+  <!--
+  <property>
     <name>fs.azure.account.key.{YOURACCOUNT}.dfs.core.windows.net</name>
     <value>{ACCOUNTKEY}</value>
   </property>
@@ -31,14 +52,115 @@
     <name>fs.azure.test.account.key.{YOURACCOUNT}.dfs.core.windows.net</name>
     <value>{ACCOUNTKEY}</value>
   </property>
+  -->
 
+  <!--If auth type is "OAuth", set below properties, AAD client and tenant related properties can be obtained through Azure Portal-->
+  <!--
   <property>
-    <name>fs.contract.test.fs.abfs</name>
-    <value>abfs://{CONTAINERNAME}@{ACCOUNTNAME}.dfs.core.windows.net/value>
-    <description>The name of the azure file system for testing.</description>
+    <name>fs.azure.account.oauth.provider.type.{YOURACCOUNT}.dfs.core.windows.net</name>
+    <value>org.apache.hadoop.fs.azurebfs.oauth2.{Token Provider Class name}</value>
+    <description>The full name of token provider class name.</description>
+  </property>
+ -->
+
+  <!--If "ClientCredsTokenProvider" is set as key provider, set auth endpoint, client id and secret below-->
+  <!--
+   <property>
+    <name>fs.azure.account.oauth2.client.endpoint.{YOURACCOUNT}.dfs.core.windows.net</name>
+    <value>https://login.microsoftonline.com/{TENANTID}/oauth2/token</value>
+    <description>Token end point, this can be found through Azure portal</description>
+  </property>
+
+   <property>
+     <name>fs.azure.account.oauth2.client.id.{YOURACCOUNT}.dfs.core.windows.net</name>
+     <value>{client id}</value>
+     <description>AAD client id.</description>
+   </property>
+
+   <property>
+     <name>fs.azure.account.oauth2.client.secret.{YOURACCOUNT}.dfs.core.windows.net</name>
+     <value>{client secret}</value>
+   </property>
+ -->
+
+  <!--If "UserPasswordTokenProvider" is set as key provider, set auth endpoint, use name and password below-->
+  <!--
+   <property>
+    <name>fs.azure.account.oauth2.client.endpoint.{YOURACCOUNT}.dfs.core.windows.net</name>
+    <value>https://login.microsoftonline.com/{TENANTID}/oauth2/token</value>
+    <description>Token end point, this can be found through Azure portal</description>
   </property>
 
+   <property>
+     <name>fs.azure.account.oauth2.user.name.{YOURACCOUNT}.dfs.core.windows.net</name>
+     <value>{user name}</value>
+   </property>
+
+   <property>
+     <name>fs.azure.account.oauth2.user.password.{YOURACCOUNT}.dfs.core.windows.net</name>
+     <value>{user password}</value>
+   </property>
+ -->
+
+  <!--If "MsiTokenProvider" is set as key provider, set tenantGuid and client id below-->
+  <!--
+   <property>
+     <name>fs.azure.account.oauth2.msi.tenant.{YOURACCOUNT}.dfs.core.windows.net</name>
+     <value>{tenantGuid}</value>
+     <description>msi tenantGuid.</description>
+   </property>
+
+   <property>
+     <name>fs.azure.account.oauth2.client.id.{YOURACCOUNT}.dfs.core.windows.net</name>
+     <value>{client id}</value>
+     <description>AAD client id.</description>
+   </property>
+  -->
+
+  <!--If "RefreshTokenBasedTokenProvider" is set as key provider, set refresh token and client id below-->
+  <!--
+   <property>
+     <name>fs.azure.account.oauth2.refresh.token.{YOURACCOUNT}.dfs.core.windows.net</name>
+     <value>{refresh token}</value>
+     <description>refresh token.</description>
+   </property>
+
+   <property>
+     <name>fs.azure.account.oauth2.client.id.{YOURACCOUNT}.dfs.core.windows.net</name>
+     <value>{client id}</value>
+     <description>AAD client id.</description>
+   </property>
+  -->
+
+  <!--Below four configure properties are provided for Blob Data Contributor a
+  nd Blob Data Reader OAuth access test "ITestAzureBlobFileSystemOauth",
+  using"ClientCredsTokenProvider" as key
+  provider. These are for test only.-->
+
+  <!--
+  <property>
+    <name>fs.azure.account.oauth2.contributor.client.id</name>
+    <value>{ID of client which has Data Contributor role}</value>
+  </property>
+
+  <property>
+    <name>fs.azure.account.oauth2.contributor.client.secret</name>
+    <value>{secret of client which has Data Contributor role}</value>
+  </property>
+
+  <property>
+    <name>fs.azure.account.oauth2.reader.client.id</name>
+    <value>{ID of client which has Data Reader role}</value>
+  </property>
+
+  <property>
+    <name>fs.azure.account.oauth2.reader.client.secret</name>
+    <value>{Secret of client which has Data Reader role}</value>
+  </property>
   -->
+
+
+
   <!-- Save above configuration properties in a separate file named -->
   <!-- azure-bfs-auth-keys.xml in the same directory as this file. -->
   <!-- DO NOT ADD azure-bfs-auth-keys.xml TO REVISION CONTROL.  The keys to your -->


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org