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 tm...@apache.org on 2018/08/11 05:37:38 UTC

[34/50] [abbrv] hadoop git commit: HADOOP-15446. ABFS: tune imports & javadocs; stabilise tests. Contributed by Steve Loughran and Da Zhou.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 7010e74..a89c044 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
@@ -17,59 +17,62 @@
  */
 package org.apache.hadoop.fs.azurebfs;
 
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
 
-import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import java.io.BufferedReader;
-import java.io.InputStreamReader;
+import org.apache.hadoop.fs.FSDataOutputStream;
+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.contract.ContractTestUtils;
 
-import static junit.framework.TestCase.assertEquals;
-import static junit.framework.TestCase.assertFalse;
-import static junit.framework.TestCase.assertTrue;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists;
 
 /**
  * Test compatibility between ABFS client and WASB client.
  */
-public class ITestWasbAbfsCompatibility extends DependencyInjectedTest {
+public class ITestWasbAbfsCompatibility extends AbstractAbfsIntegrationTest {
   private static final String WASB_TEST_CONTEXT = "wasb test file";
   private static final String ABFS_TEST_CONTEXT = "abfs test file";
   private static final String TEST_CONTEXT = "THIS IS FOR TEST";
 
-  public ITestWasbAbfsCompatibility() throws Exception {
-    super();
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestWasbAbfsCompatibility.class);
 
-    Assume.assumeFalse(this.isEmulator());
+  public ITestWasbAbfsCompatibility() throws Exception {
+    Assume.assumeFalse("Emulator is not supported", isEmulator());
   }
 
   @Test
   public void testListFileStatus() throws Exception {
     // crate file using abfs
-    AzureBlobFileSystem fs = this.getFileSystem();
-    NativeAzureFileSystem wasb = this.getWasbFileSystem();
+    AzureBlobFileSystem fs = getFileSystem();
+    NativeAzureFileSystem wasb = getWasbFileSystem();
 
     Path path1 = new Path("/testfiles/~12/!008/3/abFsTestfile");
-    FSDataOutputStream abfsStream = fs.create(path1, true);
-    abfsStream.write(ABFS_TEST_CONTEXT.getBytes());
-    abfsStream.flush();
-    abfsStream.hsync();
-    abfsStream.close();
+    try(FSDataOutputStream abfsStream = fs.create(path1, true)) {
+      abfsStream.write(ABFS_TEST_CONTEXT.getBytes());
+      abfsStream.flush();
+      abfsStream.hsync();
+    }
 
     // create file using wasb
     Path path2 = new Path("/testfiles/~12/!008/3/nativeFsTestfile");
-    System.out.println(wasb.getUri());
-    FSDataOutputStream nativeFsStream = wasb.create(path2, true);
-    nativeFsStream.write(WASB_TEST_CONTEXT.getBytes());
-    nativeFsStream.flush();
-    nativeFsStream.hsync();
-    nativeFsStream.close();
+    LOG.info("{}", wasb.getUri());
+    try(FSDataOutputStream nativeFsStream = wasb.create(path2, true)) {
+      nativeFsStream.write(WASB_TEST_CONTEXT.getBytes());
+      nativeFsStream.flush();
+      nativeFsStream.hsync();
+    }
     // list file using abfs and wasb
     FileStatus[] abfsFileStatus = fs.listStatus(new Path("/testfiles/~12/!008/3/"));
     FileStatus[] nativeFsFileStatus = wasb.listStatus(new Path("/testfiles/~12/!008/3/"));
@@ -83,52 +86,34 @@ public class ITestWasbAbfsCompatibility extends DependencyInjectedTest {
     boolean[] createFileWithAbfs = new boolean[]{false, true, false, true};
     boolean[] readFileWithAbfs = new boolean[]{false, true, true, false};
 
-    AzureBlobFileSystem abfs = this.getFileSystem();
-    NativeAzureFileSystem wasb = this.getWasbFileSystem();
+    AzureBlobFileSystem abfs = getFileSystem();
+    NativeAzureFileSystem wasb = getWasbFileSystem();
 
-    FileSystem fs;
-    BufferedReader br = null;
     for (int i = 0; i< 4; i++) {
-      try {
-        Path path = new Path("/testfiles/~12/!008/testfile" + i);
-        if (createFileWithAbfs[i]) {
-          fs = abfs;
-        } else {
-          fs = wasb;
-        }
-
-        // Write
-        FSDataOutputStream nativeFsStream = fs.create(path, true);
+      Path path = new Path("/testfiles/~12/!008/testfile" + i);
+      final FileSystem createFs = createFileWithAbfs[i] ? abfs : wasb;
+
+      // Write
+      try(FSDataOutputStream nativeFsStream = createFs.create(path, true)) {
         nativeFsStream.write(TEST_CONTEXT.getBytes());
         nativeFsStream.flush();
         nativeFsStream.hsync();
-        nativeFsStream.close();
-
-        // Check file status
-        assertEquals(true, fs.exists(path));
-        assertEquals(false, fs.getFileStatus(path).isDirectory());
-
-        // Read
-        if (readFileWithAbfs[i]) {
-          fs = abfs;
-        } else {
-          fs = wasb;
-        }
-        FSDataInputStream inputStream = fs.open(path);
-        br = new BufferedReader(new InputStreamReader(fs.open(path)));
+      }
+
+      // Check file status
+      ContractTestUtils.assertIsFile(createFs, path);
+
+      // Read
+      final FileSystem readFs = readFileWithAbfs[i] ? abfs : wasb;
+
+      try(BufferedReader br =new BufferedReader(new InputStreamReader(readFs.open(path)))) {
         String line = br.readLine();
-        assertEquals(TEST_CONTEXT, line);
-
-        // Remove file
-        fs.delete(path, true);
-        assertFalse(fs.exists(path));
-      } catch (Exception e) {
-        e.printStackTrace();
-      } finally {
-        if (br != null) {
-          br.close();
-        }
+        assertEquals("Wrong text from " + readFs,
+            TEST_CONTEXT, line);
       }
+
+      // Remove file
+      assertDeleted(readFs, path, true);
     }
   }
 
@@ -137,32 +122,22 @@ public class ITestWasbAbfsCompatibility extends DependencyInjectedTest {
     boolean[] createDirWithAbfs = new boolean[]{false, true, false, true};
     boolean[] readDirWithAbfs = new boolean[]{false, true, true, false};
 
-    AzureBlobFileSystem abfs = this.getFileSystem();
-    NativeAzureFileSystem wasb = this.getWasbFileSystem();
+    AzureBlobFileSystem abfs = getFileSystem();
+    NativeAzureFileSystem wasb = getWasbFileSystem();
 
-    FileSystem fs;
     for (int i = 0; i < 4; i++) {
       Path path = new Path("/testDir/t" + i);
       //create
-      if (createDirWithAbfs[i]) {
-        fs = abfs;
-      } else {
-        fs = wasb;
-      }
-      assertTrue(fs.mkdirs(path));
+      final FileSystem createFs = createDirWithAbfs[i] ? abfs : wasb;
+      assertTrue(createFs.mkdirs(path));
       //check
-      assertTrue(fs.exists(path));
+      assertPathExists(createFs, "Created dir not found with " + createFs, path);
       //read
-      if (readDirWithAbfs[i]) {
-        fs = abfs;
-      } else {
-        fs = wasb;
-      }
-      assertTrue(fs.exists(path));
-      FileStatus dirStatus = fs.getFileStatus(path);
-      assertTrue(dirStatus.isDirectory());
-      fs.delete(path, true);
-      assertFalse(fs.exists(path));
+      final FileSystem readFs = readDirWithAbfs[i] ? abfs : wasb;
+      assertPathExists(readFs, "Created dir not found with " + readFs,
+          path);
+      assertIsDirectory(readFs, path);
+      assertDeleted(readFs, path, true);
     }
   }
 
@@ -171,17 +146,18 @@ public class ITestWasbAbfsCompatibility extends DependencyInjectedTest {
   public void testUrlConversion(){
     String abfsUrl = "abfs://abcde-1111-1111-1111-1111@xxxx.dfs.xxx.xxx.xxxx.xxxx";
     String wabsUrl = "wasb://abcde-1111-1111-1111-1111@xxxx.blob.xxx.xxx.xxxx.xxxx";
-    Assert.assertEquals(abfsUrl, wasbUrlToAbfsUrl(wabsUrl));
-    Assert.assertEquals(wabsUrl, abfsUrlToWasbUrl(abfsUrl));
+    assertEquals(abfsUrl, wasbUrlToAbfsUrl(wabsUrl));
+    assertEquals(wabsUrl, abfsUrlToWasbUrl(abfsUrl));
   }
 
   @Test
   public void testSetWorkingDirectory() throws Exception {
     //create folders
-    AzureBlobFileSystem abfs = this.getFileSystem();
-    NativeAzureFileSystem wasb = this.getWasbFileSystem();
+    AzureBlobFileSystem abfs = getFileSystem();
+    NativeAzureFileSystem wasb = getWasbFileSystem();
 
-    assertTrue(abfs.mkdirs(new Path("/d1/d2/d3/d4")));
+    Path d1d4 = new Path("/d1/d2/d3/d4");
+    assertMkdirs(abfs, d1d4);
 
     //set working directory to path1
     Path path1 = new Path("/d1/d2");
@@ -195,8 +171,8 @@ public class ITestWasbAbfsCompatibility extends DependencyInjectedTest {
     wasb.setWorkingDirectory(path2);
     abfs.setWorkingDirectory(path2);
 
-    Path path3 = new Path("/d1/d2/d3/d4");
+    Path path3 = d1d4;
     assertEquals(path3, wasb.getWorkingDirectory());
     assertEquals(path3, abfs.getWorkingDirectory());
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 4b44765..fc7312a 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
@@ -18,14 +18,9 @@
 
 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 TestConfigurationKeys {
   public static final String FS_AZURE_TEST_ACCOUNT_NAME = "fs.azure.test.account.name";
   public static final String FS_AZURE_TEST_ACCOUNT_KEY_PREFIX = "fs.azure.test.account.key.";
@@ -33,5 +28,9 @@ 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 ABFS_TEST_RESOURCE_XML = "azure-bfs-test.xml";
+
+  public static final String ABFS_TEST_CONTAINER_PREFIX = "abfs-testcontainer-";
+
   private TestConfigurationKeys() {}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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
new file mode 100644
index 0000000..ffd5bab
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ABFSContractTestBinding.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
+ *
+ *     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.contract;
+
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+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;
+
+/**
+ * Bind ABFS contract tests to the Azure test setup/teardown.
+ */
+public class ABFSContractTestBinding extends AbstractAbfsIntegrationTest {
+  private final URI testUri;
+
+  public ABFSContractTestBinding(final boolean secure) throws Exception {
+    this(secure, true);
+  }
+
+  public ABFSContractTestBinding(final boolean secure,
+      final boolean useExistingFileSystem) throws Exception{
+    super(secure);
+    if (useExistingFileSystem) {
+      Configuration configuration = getConfiguration();
+      String testUrl = configuration.get(TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI);
+
+      if (secure) {
+        testUrl = testUrl.replaceFirst(FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME);
+      }
+      setTestUrl(testUrl);
+
+      this.testUri = new URI(testUrl);
+      //Get container for contract tests
+      configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, this.testUri.toString());
+      String[] splitAuthority = this.testUri.getAuthority().split("\\@");
+      setFileSystemName(splitAuthority[0]);
+    } else {
+      this.testUri = new URI(super.getTestUrl());
+    }
+  }
+
+  public Configuration getConfiguration() {
+    return super.getConfiguration();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java
new file mode 100644
index 0000000..d365e6e
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java
@@ -0,0 +1,65 @@
+/**
+ * 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.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
+import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
+import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
+
+/**
+ * Azure BlobFileSystem Contract. Test paths are created using any maven fork
+ * identifier, if defined. This guarantees paths unique to tests
+ * running in parallel.
+ */
+public class AbfsFileSystemContract extends AbstractBondedFSContract {
+
+  public static final String CONTRACT_XML = "abfs.xml";
+  private final boolean isSecure;
+
+  protected AbfsFileSystemContract(final Configuration conf, boolean secure) {
+    super(conf);
+    //insert the base features
+    addConfResource(CONTRACT_XML);
+    this.isSecure = secure;
+  }
+
+  @Override
+  public String getScheme() {
+    return isSecure ?
+        FileSystemUriSchemes.ABFS_SECURE_SCHEME
+        : FileSystemUriSchemes.ABFS_SCHEME;
+  }
+
+  @Override
+  public Path getTestPath() {
+    return new Path(UriUtils.generateUniqueTestPath());
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "AbfsFileSystemContract{");
+    sb.append("isSecure=").append(isSecure);
+    sb.append(super.toString());
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java
deleted file mode 100644
index 5fc81ce..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/DependencyInjectedContractTest.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.azurebfs.contract;
-
-import java.net.URI;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
-import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
-import org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys;
-
-/**
- * Dependency inject for ABFS contract tests.
- */
-public class DependencyInjectedContractTest extends DependencyInjectedTest {
-  private final URI testUri;
-
-  public DependencyInjectedContractTest(final boolean secure) throws Exception {
-    this(secure, true);
-  }
-
-  public DependencyInjectedContractTest(final boolean secure, final boolean useExistedFileSystem) throws Exception{
-    super(secure);
-    if (useExistedFileSystem) {
-      Configuration configuration = getConfiguration();
-      String testUrl = configuration.get(TestConfigurationKeys.FS_AZURE_CONTRACT_TEST_URI);
-
-      if (secure) {
-        testUrl = testUrl.replaceFirst(FileSystemUriSchemes.ABFS_SCHEME, FileSystemUriSchemes.ABFS_SECURE_SCHEME);
-      }
-      updateTestUrl(testUrl);
-
-      this.testUri = new URI(testUrl);
-      //Get container for contract tests
-      configuration.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, this.testUri.toString());
-      String[] splitAuthority = this.testUri.getAuthority().split("\\@");
-      updateFileSystemName(splitAuthority[0]);
-    } else {
-      this.testUri = new URI(super.getTestUrl());
-    }
-  }
-
-  public Configuration getConfiguration() {
-    return super.getConfiguration();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java
deleted file mode 100644
index 7f7a09a..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/ITestAbfsFileSystemContract.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.azurebfs.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes;
-import org.apache.hadoop.fs.azurebfs.utils.UriUtils;
-import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
-
-/**
- * Azure BlobFileSystem Contract. Test paths are created using any maven fork
- * identifier, if defined. This guarantees paths unique to tests
- * running in parallel.
- */
-public class ITestAbfsFileSystemContract extends AbstractBondedFSContract {
-
-  public static final String CONTRACT_XML = "abfs.xml";
-  private final boolean isSecure;
-
-  protected ITestAbfsFileSystemContract(final Configuration conf, boolean secure) {
-    super(conf);
-    //insert the base features
-    addConfResource(CONTRACT_XML);
-    this.isSecure = secure;
-  }
-
-  @Override
-  public String getScheme() {
-    return isSecure ? FileSystemUriSchemes.ABFS_SECURE_SCHEME : FileSystemUriSchemes.ABFS_SCHEME;
-  }
-
-  @Override
-  public Path getTestPath() {
-    Path path = new Path(UriUtils.generateUniqueTestPath());
-    return path;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 d4cca14..a302fcc 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
@@ -20,6 +20,7 @@ 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;
 
@@ -40,31 +41,32 @@ public class ITestAbfsFileSystemContractAppend extends AbstractContractAppendTes
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractAppend(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+    binding = new ABFSContractTestBinding(this.isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
 
   @Override
+  @Test
   public void testRenameFileBeingAppended() throws Throwable {
     skip("Skipping as renaming an opened file is not supported");
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 4f724e2..c31a6d2 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
@@ -37,26 +37,26 @@ public class ITestAbfsFileSystemContractConcat extends AbstractContractConcatTes
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractConcat(final boolean secure) throws Exception {
-    this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure);
+    isSecure = secure;
+    binding = new ABFSContractTestBinding(isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 16b959f..ce4d229 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
@@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractCreate extends AbstractContractCreateTes
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractCreate(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+    binding = new ABFSContractTestBinding(this.isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 fabd3273..310731c 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
@@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractDelete extends AbstractContractDeleteTes
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractDelete(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure);
+    binding = new ABFSContractTestBinding(isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 a1360e4..c2cf255 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
@@ -25,20 +25,20 @@ import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
  * Contract test for distCp operation.
  */
 public class ITestAbfsFileSystemContractDistCp extends AbstractContractDistCpTest {
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractDistCp() throws Exception {
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(false);
+    binding = new ABFSContractTestBinding(false);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
-  protected ITestAbfsFileSystemContract createContract(Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, false);
+  protected AbfsFileSystemContract createContract(Configuration conf) {
+    return new AbfsFileSystemContract(conf, false);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 5bb41ad..9ad3b21 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
@@ -37,26 +37,26 @@ public class ITestAbfsFileSystemContractGetFileStatus extends AbstractContractGe
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractGetFileStatus(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(isSecure);
+    binding = new ABFSContractTestBinding(isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return this.binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, this.isSecure);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 9d732d5..6265ca1 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
@@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractMkdir extends AbstractContractMkdirTest
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractMkdir(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(secure);
+    binding = new ABFSContractTestBinding(secure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 a71149b..ae4bb2a 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
@@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractOpen extends AbstractContractOpenTest {
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractOpen(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+    binding = new ABFSContractTestBinding(this.isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 6d1c4ae..6e6a728 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
@@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractRename extends AbstractContractRenameTes
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractRename(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+    binding = new ABFSContractTestBinding(this.isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 46072ad..01dea2d 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
@@ -37,31 +37,31 @@ public class ITestAbfsFileSystemContractRootDirectory extends AbstractContractRo
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractRootDirectory(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(secure);
+    binding = new ABFSContractTestBinding(secure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
 
   @Override
   @Ignore("ABFS always return false when non-recursively remove root dir")
   public void testRmNonEmptyRootDirNonRecursive() throws Throwable {
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 1780f6f..5ed7466 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
@@ -25,20 +25,20 @@ import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
  * Contract test for secure distCP operation.
  */
 public class ITestAbfsFileSystemContractSecureDistCp extends AbstractContractDistCpTest {
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractSecureDistCp() throws Exception {
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(true);
+    binding = new ABFSContractTestBinding(true);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
-  protected ITestAbfsFileSystemContract createContract(Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, true);
+  protected AbfsFileSystemContract createContract(Configuration conf) {
+    return new AbfsFileSystemContract(conf, true);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 aeeb042..5e0ea0c 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
@@ -38,26 +38,26 @@ public class ITestAbfsFileSystemContractSeek extends AbstractContractSeekTest{
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractSeek(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+    binding = new ABFSContractTestBinding(this.isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 ea9392d..8d23b0b 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
@@ -37,26 +37,26 @@ public class ITestAbfsFileSystemContractSetTimes extends AbstractContractSetTime
   }
 
   private final boolean isSecure;
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   public ITestAbfsFileSystemContractSetTimes(final boolean secure) throws Exception {
     this.isSecure = secure;
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(this.isSecure);
+    binding = new ABFSContractTestBinding(this.isSecure);
   }
 
   @Override
   public void setup() throws Exception {
-    dependencyInjectedContractTest.initialize();
+    binding.setup();
     super.setup();
   }
 
   @Override
   protected Configuration createConfiguration() {
-    return this.dependencyInjectedContractTest.getConfiguration();
+    return binding.getConfiguration();
   }
 
   @Override
   protected AbstractFSContract createContract(final Configuration conf) {
-    return new ITestAbfsFileSystemContract(conf, this.isSecure);
+    return new AbfsFileSystemContract(conf, isSecure);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 9f3b4a7..d8854a2 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
@@ -22,8 +22,8 @@ import java.io.IOException;
 import org.apache.hadoop.fs.FileSystemContractBaseTest;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 
-import org.junit.After;
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -35,26 +35,31 @@ import static org.junit.Assert.assertTrue;
  * Basic Contract test for Azure BlobFileSystem.
  */
 public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest {
-  private final DependencyInjectedContractTest dependencyInjectedContractTest;
+  private final ABFSContractTestBinding binding;
 
   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
-    dependencyInjectedContractTest = new DependencyInjectedContractTest(false, false);
+    binding = new ABFSContractTestBinding(false, false);
   }
 
+
   @Before
   public void setUp() throws Exception {
-    this.dependencyInjectedContractTest.initialize();
-    fs = this.dependencyInjectedContractTest.getFileSystem();
+    binding.setup();
+    fs = binding.getFileSystem();
   }
 
-  @After
-  public void testCleanup() throws Exception {
+  @Override
+  public void tearDown() throws Exception {
     // This contract test is not using existing container for test,
     // instead it creates its own temp container for test, hence we need to destroy
     // it after the test.
-    this.dependencyInjectedContractTest.testCleanup();
+    try {
+      super.tearDown();
+    } finally {
+      binding.teardown();
+    }
   }
 
   @Test
@@ -82,7 +87,7 @@ public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest {
     Path filePath = path("testListStatus/file");
 
     assertTrue(fs.mkdirs(folderPath));
-    fs.create(filePath);
+    ContractTestUtils.touch(fs, filePath);
 
     FileStatus[] listFolderStatus;
     listFolderStatus = fs.listStatus(path("testListStatus"));
@@ -97,4 +102,4 @@ public class ITestAzureBlobFileSystemBasics extends FileSystemContractBaseTest {
   @Ignore("Not implemented in ABFS yet")
   public void testMkdirsWithUmask() throws Exception {
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java
index 03320d6..e9e90c9 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/diagnostics/TestConfigurationValidators.java
@@ -19,10 +19,11 @@
 package org.apache.hadoop.fs.azurebfs.diagnostics;
 
 import org.apache.commons.codec.Charsets;
+
+import org.junit.Assert;
 import org.junit.Test;
 
 
-import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
 import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
 import org.apache.commons.codec.binary.Base64;
 
@@ -36,7 +37,7 @@ import static org.junit.Assert.assertEquals;
 /**
  * Test configuration validators.
  */
-public class TestConfigurationValidators extends DependencyInjectedTest {
+public class TestConfigurationValidators extends Assert {
 
   private static final String FAKE_KEY = "FakeKey";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.java
new file mode 100644
index 0000000..dd06fe3
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsReadWriteAndSeek.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.services;
+
+import java.util.Arrays;
+import java.util.Random;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsScaleTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_BUFFER_SIZE;
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
+
+/**
+ * Test read, write and seek.
+ * Uses package-private methods in AbfsConfiguration, which is why it is in
+ * this package.
+ */
+@RunWith(Parameterized.class)
+public class ITestAbfsReadWriteAndSeek extends AbstractAbfsScaleTest {
+  private static final Path TEST_PATH = new Path("/testfile");
+
+  @Parameterized.Parameters(name = "Size={0}")
+  public static Iterable<Object[]> sizes() {
+    return Arrays.asList(new Object[][]{{MIN_BUFFER_SIZE},
+        {DEFAULT_READ_BUFFER_SIZE},
+        {MAX_BUFFER_SIZE}});
+  }
+
+  private final int size;
+
+  public ITestAbfsReadWriteAndSeek(final int size) {
+    this.size = size;
+  }
+
+  @Test
+  public void testReadAndWriteWithDifferentBufferSizesAndSeek() throws Exception {
+    testReadWriteAndSeek(size);
+  }
+
+  private void testReadWriteAndSeek(int bufferSize) throws Exception {
+    final AzureBlobFileSystem fs = getFileSystem();
+    final AbfsConfiguration abfsConfiguration = new AbfsConfiguration(getConfiguration());
+
+    abfsConfiguration.setWriteBufferSize(bufferSize);
+    abfsConfiguration.setReadBufferSize(bufferSize);
+
+
+    final byte[] b = new byte[2 * bufferSize];
+    new Random().nextBytes(b);
+    try(final FSDataOutputStream stream = fs.create(TEST_PATH)) {
+      stream.write(b);
+    }
+
+    final byte[] readBuffer = new byte[2 * bufferSize];
+    int result;
+    try(final FSDataInputStream inputStream = fs.open(TEST_PATH)) {
+      inputStream.seek(bufferSize);
+      result = inputStream.read(readBuffer, bufferSize, bufferSize);
+      assertNotEquals(-1, result);
+      inputStream.seek(0);
+      result = inputStream.read(readBuffer, 0, bufferSize);
+    }
+    assertNotEquals("data read in final read()", -1, result);
+    assertArrayEquals(readBuffer, b);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java
deleted file mode 100644
index cabe049..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestReadWriteAndSeek.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.azurebfs.services;
-
-import java.util.Random;
-
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
-import org.apache.hadoop.fs.azurebfs.DependencyInjectedTest;
-
-import org.junit.Test;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MIN_BUFFER_SIZE;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE;
-import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_BUFFER_SIZE;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertNotEquals;
-
-/**
- * Test read, write and seek.
- */
-public class ITestReadWriteAndSeek extends DependencyInjectedTest {
-  private static final Path TEST_PATH = new Path("/testfile");
-  public ITestReadWriteAndSeek() {
-    super();
-  }
-
-  @Test
-  public void testReadAndWriteWithDifferentBufferSizesAndSeek() throws Exception {
-    testReadWriteAndSeek(MIN_BUFFER_SIZE);
-    testReadWriteAndSeek(DEFAULT_READ_BUFFER_SIZE);
-    testReadWriteAndSeek(MAX_BUFFER_SIZE);
-  }
-
-  private void testReadWriteAndSeek(int bufferSize) throws Exception {
-    final AzureBlobFileSystem fs = this.getFileSystem();
-    final AbfsConfiguration abfsConfiguration = new AbfsConfiguration(this.getConfiguration());
-
-    fs.create(TEST_PATH);
-    abfsConfiguration.setWriteBufferSize(bufferSize);
-    abfsConfiguration.setReadBufferSize(bufferSize);
-
-    final FSDataOutputStream stream = fs.create(TEST_PATH);
-
-    final byte[] b = new byte[2 * bufferSize];
-    new Random().nextBytes(b);
-    stream.write(b);
-    stream.close();
-
-    final byte[] r = new byte[2 * bufferSize];
-    final FSDataInputStream inputStream = fs.open(TEST_PATH);
-    inputStream.seek(bufferSize);
-    int result = inputStream.read(r, bufferSize, bufferSize);
-    assertNotEquals(-1, result);
-
-    inputStream.seek(0);
-    result = inputStream.read(r, 0, bufferSize);
-    assertNotEquals(-1, result);
-    assertArrayEquals(r, b);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
index 441a35a..ebaafa4 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsConfigurationFieldsValidation.java
@@ -48,7 +48,7 @@ import org.junit.Test;
  * Test ConfigurationServiceFieldsValidation.
  */
 public class TestAbfsConfigurationFieldsValidation {
-  private AbfsConfiguration abfsConfiguration;
+  private final AbfsConfiguration abfsConfiguration;
 
   private static final String INT_KEY= "intKey";
   private static final String LONG_KEY= "longKey";
@@ -89,12 +89,10 @@ public class TestAbfsConfigurationFieldsValidation {
   private boolean boolField;
 
   public TestAbfsConfigurationFieldsValidation() throws Exception {
-    super();
     Base64 base64 = new Base64();
     this.encodedString = new String(base64.encode("base64Value".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
     this.encodedAccountKey = new String(base64.encode("someAccountKey".getBytes(Charsets.UTF_8)), Charsets.UTF_8);
-    Configuration configuration = new Configuration();
-    configuration.addResource("azure-bfs-test.xml");
+    Configuration configuration = new Configuration(false);
     configuration.set(INT_KEY, "1234565");
     configuration.set(LONG_KEY, "4194304");
     configuration.set(STRING_KEY, "stringValue");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 2716bff..9051a72 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
@@ -24,7 +24,7 @@ import com.microsoft.azure.storage.blob.CloudBlobContainer;
 import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
 import org.junit.Test;
 
-import static org.apache.hadoop.fs.azurebfs.DependencyInjectedTest.TEST_CONTAINER_PREFIX;
+import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.ABFS_TEST_CONTAINER_PREFIX;
 
 /**
  * If unit tests were interrupted and crushed accidentally, the test containers won't be deleted.
@@ -38,7 +38,7 @@ public class CleanUpAbfsTestContainer {
     CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount();
     CloudBlobClient blobClient = storageAccount.createCloudBlobClient();
     Iterable<CloudBlobContainer> containers
-            = blobClient.listContainers(TEST_CONTAINER_PREFIX);
+            = blobClient.listContainers(ABFS_TEST_CONTAINER_PREFIX);
     for (CloudBlobContainer container : containers) {
       count++;
       System.out.println(String.format("Container %s URI %s",
@@ -54,7 +54,7 @@ public class CleanUpAbfsTestContainer {
     CloudStorageAccount storageAccount = AzureBlobStorageTestAccount.createTestAccount();
     CloudBlobClient blobClient = storageAccount.createCloudBlobClient();
     Iterable<CloudBlobContainer> containers
-            = blobClient.listContainers(TEST_CONTAINER_PREFIX);
+            = blobClient.listContainers(ABFS_TEST_CONTAINER_PREFIX);
     for (CloudBlobContainer container : containers) {
       System.out.println(String.format("Container %s URI %s",
               container.getName(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml b/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml
index caf8677..d065ace 100644
--- a/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml
+++ b/hadoop-tools/hadoop-azure/src/test/resources/abfs.xml
@@ -1,3 +1,4 @@
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <!--
   ~ Licensed to the Apache Software Foundation (ASF) under one
   ~  or more contributor license agreements.  See the NOTICE file
@@ -15,7 +16,6 @@
   ~  See the License for the specific language governing permissions and
   ~  limitations under the License.
   -->
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <configuration xmlns:xi="http://www.w3.org/2001/XInclude">
     <property>
         <name>fs.contract.test.root-tests-enabled</name>
@@ -61,4 +61,4 @@
         <name>fs.contract.supports-getfilestatus</name>
         <value>true</value>
     </property>
-</configuration>
\ No newline at end of file
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/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 508e5f6..464a8e6 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
@@ -1,4 +1,5 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <!--
   Licensed under the Apache License, Version 2.0 (the "License");
   you may not use this file except in compliance with the License.
@@ -13,7 +14,6 @@
   limitations under the License.
 -->
 
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <configuration xmlns:xi="http://www.w3.org/2001/XInclude">
 
   <!--
@@ -63,14 +63,4 @@
   <!--<value>false</value>-->
   <!--</property>-->
 
-  <property>
-    <name>fs.AbstractFileSystem.abfs.impl</name>
-    <value>org.apache.hadoop.fs.azurebfs.Abfs</value>
-  </property>
-
-  <property>
-    <name>fs.AbstractFileSystem.abfss.impl</name>
-    <value>org.apache.hadoop.fs.azurebfs.Abfss</value>
-  </property>
-
-</configuration>
\ No newline at end of file
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75b184c6/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
index fd0cd9d..bac431d 100644
--- a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
+++ b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
@@ -26,3 +26,35 @@ log4j.logger.org.apache.hadoop.fs.azure.AzureFileSystemThreadPoolExecutor=DEBUG
 log4j.logger.org.apache.hadoop.fs.azure.BlockBlobAppendStream=DEBUG
 log4j.logger.org.apache.hadoop.fs.azurebfs.contracts.services.TracingService=TRACE
 log4j.logger.org.apache.hadoop.fs.azurebfs.services.AbfsClient=DEBUG
+
+# after here: turn off log messages from other parts of the system
+# which only clutter test reports.
+log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
+log4j.logger.org.apache.hadoop.conf.Configuration.deprecation=WARN
+log4j.logger.org.apache.hadoop.util.GSet=WARN
+# MiniDFS clusters can be noisy
+log4j.logger.org.apache.hadoop.hdfs.server=ERROR
+log4j.logger.org.apache.hadoop.metrics2=WARN
+log4j.logger.org.apache.hadoop.net.NetworkTopology=WARN
+log4j.logger.org.apache.hadoop.util.JvmPauseMonitor=WARN
+log4j.logger.org.apache.hadoop.ipc=WARN
+log4j.logger.org.apache.hadoop.http=WARN
+log4j.logger.org.apache.hadoop.security.authentication.server.AuthenticationFilter=WARN
+log4j.logger.org.apache.hadoop.util.HostsFileReader=WARN
+log4j.logger.org.apache.commons.beanutils=WARN
+log4j.logger.org.apache.hadoop.hdfs.StateChange=WARN
+log4j.logger.BlockStateChange=WARN
+log4j.logger.org.apache.hadoop.hdfs.DFSUtil=WARN
+## YARN can be noisy too
+log4j.logger.org.apache.hadoop.yarn.server.resourcemanager.scheduler=WARN
+log4j.logger.org.apache.hadoop.yarn.server.nodemanager=WARN
+log4j.logger.org.apache.hadoop.yarn.event=WARN
+log4j.logger.org.apache.hadoop.yarn.util.ResourceCalculatorPlugin=ERROR
+log4j.logger.org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor=WARN
+log4j.logger.org.apache.hadoop.mapred.IndexCache=WARN
+log4j.logger.org.apache.hadoop.yarn.webapp.WebApps=WARN
+log4j.logger.org.apache.hadoop.yarn.server.resourcemanager.security=WARN
+log4j.logger.org.apache.hadoop.yarn.util.AbstractLivelinessMonitor=WARN
+log4j.logger.org.apache.hadoop.security.token.delegation=WARN
+log4j.logger.org.apache.hadoop.mapred.ShuffleHandler=WARN
+log4j.logger.org.apache.hadoop.ipc.Server=WARN


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