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 ae...@apache.org on 2017/09/18 22:11:44 UTC

[01/50] [abbrv] hadoop git commit: Revert "HDFS-10391. Always enable NameNode service RPC port. Contributed by Gergely Novak."

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 44d086733 -> 19f7f8751


http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
index 4d04970..b3bb3dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
@@ -171,18 +171,15 @@ public class TestEditLogTailer {
     for (int i = 0; i < 5; i++) {
       try {
         // Have to specify IPC ports so the NNs can talk to each other.
-        int[] ports = ServerSocketUtil.getPorts(6);
+        int[] ports = ServerSocketUtil.getPorts(3);
         MiniDFSNNTopology topology = new MiniDFSNNTopology()
             .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
                 .addNN(new MiniDFSNNTopology.NNConf("nn1")
-                    .setIpcPort(ports[0])
-                    .setServicePort(ports[1]))
+                    .setIpcPort(ports[0]))
                 .addNN(new MiniDFSNNTopology.NNConf("nn2")
-                    .setIpcPort(ports[2])
-                    .setServicePort(ports[3]))
+                    .setIpcPort(ports[1]))
                 .addNN(new MiniDFSNNTopology.NNConf("nn3")
-                    .setIpcPort(ports[4])
-                    .setServicePort(ports[5])));
+                    .setIpcPort(ports[2])));
 
         cluster = new MiniDFSCluster.Builder(conf)
           .nnTopology(topology)
@@ -222,14 +219,11 @@ public class TestEditLogTailer {
     MiniDFSNNTopology topology = new MiniDFSNNTopology()
         .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
             .addNN(new MiniDFSNNTopology.NNConf("nn1")
-                .setIpcPort(ServerSocketUtil.getPort(0, 100))
-                .setServicePort(ServerSocketUtil.getPort(0, 100)))
+                .setIpcPort(ServerSocketUtil.getPort(0, 100)))
             .addNN(new MiniDFSNNTopology.NNConf("nn2")
-                .setIpcPort(ServerSocketUtil.getPort(0, 100))
-                .setServicePort(ServerSocketUtil.getPort(0, 100)))
+                .setIpcPort(ServerSocketUtil.getPort(0, 100)))
             .addNN(new MiniDFSNNTopology.NNConf("nn3")
-                .setIpcPort(ServerSocketUtil.getPort(0, 100))
-                .setServicePort(ServerSocketUtil.getPort(0, 100))));
+                .setIpcPort(ServerSocketUtil.getPort(0, 100))));
 
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .nnTopology(topology)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java
index f2274f9..f1f5793 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hdfs.tools;
 
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -180,11 +179,9 @@ public class TestDFSHAAdmin {
     Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol)
         .getServiceStatus();
     assertEquals(0, runTool("-getAllServiceState"));
-    assertOutputContains(String.format("%-50s %-10s", (HOST_A + ":" +
-            DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT),
+    assertOutputContains(String.format("%-50s %-10s", (HOST_A + ":" + 12345),
         STANDBY_READY_RESULT.getState()));
-    assertOutputContains(String.format("%-50s %-10s", (HOST_B + ":" +
-            DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT),
+    assertOutputContains(String.format("%-50s %-10s", (HOST_B + ":" + 12345),
         STANDBY_READY_RESULT.getState()));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java
index d49a3cf..a21a31d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java
@@ -77,7 +77,7 @@ public class TestDFSHAAdminMiniCluster {
     tool.setErrOut(new PrintStream(errOutBytes));
     cluster.waitActive();
     
-    nn1Port = cluster.getNameNodeServicePort(0);
+    nn1Port = cluster.getNameNodePort(0);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSZKFailoverController.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSZKFailoverController.java
index 9943260..bbb787e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSZKFailoverController.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSZKFailoverController.java
@@ -88,11 +88,9 @@ public class TestDFSZKFailoverController extends ClientBaseWithFixes {
     MiniDFSNNTopology topology = new MiniDFSNNTopology()
     .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
         .addNN(new MiniDFSNNTopology.NNConf("nn1")
-            .setIpcPort(ServerSocketUtil.getPort(10021, 100))
-            .setServicePort(ServerSocketUtil.getPort(10025, 100)))
+            .setIpcPort(ServerSocketUtil.getPort(10021, 100)))
         .addNN(new MiniDFSNNTopology.NNConf("nn2")
-            .setIpcPort(ServerSocketUtil.getPort(10022, 100))
-            .setServicePort(ServerSocketUtil.getPort(10026, 100))));
+            .setIpcPort(ServerSocketUtil.getPort(10022, 100))));
     cluster = new MiniDFSCluster.Builder(conf)
         .nnTopology(topology)
         .numDataNodes(0)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
index 36d57a9..942719e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
@@ -24,7 +24,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICES;
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -122,13 +121,13 @@ public class TestGetConf {
       TestType type, HdfsConfiguration conf) throws IOException {
     switch (type) {
     case NAMENODE:
-      return DFSUtil.getNNServiceRpcAddresses(conf);
+      return DFSUtil.getNNServiceRpcAddressesForCluster(conf);
     case BACKUP:
       return DFSUtil.getBackupNodeAddresses(conf);
     case SECONDARY:
       return DFSUtil.getSecondaryNameNodeAddresses(conf);
     case NNRPCADDRESSES:
-      return DFSUtil.getNNServiceRpcAddresses(conf);
+      return DFSUtil.getNNServiceRpcAddressesForCluster(conf);
     }
     return null;
   }
@@ -279,12 +278,10 @@ public class TestGetConf {
   public void testNonFederation() throws Exception {
     HdfsConfiguration conf = new HdfsConfiguration(false);
   
-    // Returned namenode address should match the default service address
+    // Returned namenode address should match default address
     conf.set(FS_DEFAULT_NAME_KEY, "hdfs://localhost:1000");
-    verifyAddresses(conf, TestType.NAMENODE, false, "localhost:" +
-        DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
-    verifyAddresses(conf, TestType.NNRPCADDRESSES, true, "localhost:" +
-        DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
+    verifyAddresses(conf, TestType.NAMENODE, false, "localhost:1000");
+    verifyAddresses(conf, TestType.NNRPCADDRESSES, true, "localhost:1000");
   
     // Returned address should match backupnode RPC address
     conf.set(DFS_NAMENODE_BACKUP_ADDRESS_KEY,"localhost:1001");
@@ -300,14 +297,12 @@ public class TestGetConf {
     conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY, "localhost:1001");
     verifyAddresses(conf, TestType.NAMENODE, false, "localhost:1000");
     verifyAddresses(conf, TestType.NNRPCADDRESSES, true, "localhost:1000");
-
-    // Returned namenode address should match the default service address
+  
+    // Returned address should match RPC address
     conf = new HdfsConfiguration();
     conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY, "localhost:1001");
-    verifyAddresses(conf, TestType.NAMENODE, false, "localhost:" +
-        DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
-    verifyAddresses(conf, TestType.NNRPCADDRESSES, true, "localhost:" +
-        DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
+    verifyAddresses(conf, TestType.NAMENODE, false, "localhost:1001");
+    verifyAddresses(conf, TestType.NNRPCADDRESSES, true, "localhost:1001");
   }
 
   /**
@@ -335,6 +330,23 @@ public class TestGetConf {
     verifyAddresses(conf, TestType.BACKUP, false, backupAddresses);
     verifyAddresses(conf, TestType.SECONDARY, false, secondaryAddresses);
     verifyAddresses(conf, TestType.NNRPCADDRESSES, true, nnAddresses);
+  
+    // Test to ensure namenode, backup, secondary namenode addresses and 
+    // namenode rpc addresses are  returned from federation configuration. 
+    // Returned namenode addresses are based on regular RPC address
+    // in the absence of service RPC address.
+    conf = new HdfsConfiguration(false);
+    setupNameServices(conf, nsCount);
+    nnAddresses = setupAddress(conf,
+        DFS_NAMENODE_RPC_ADDRESS_KEY, nsCount, 1000);
+    backupAddresses = setupAddress(conf,
+        DFS_NAMENODE_BACKUP_ADDRESS_KEY, nsCount, 2000);
+    secondaryAddresses = setupAddress(conf,
+        DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY, nsCount, 3000);
+    verifyAddresses(conf, TestType.NAMENODE, false, nnAddresses);
+    verifyAddresses(conf, TestType.BACKUP, false, backupAddresses);
+    verifyAddresses(conf, TestType.SECONDARY, false, secondaryAddresses);
+    verifyAddresses(conf, TestType.NNRPCADDRESSES, true, nnAddresses);
   }
   
   @Test(timeout=10000)


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


[41/50] [abbrv] hadoop git commit: YARN-7172. ResourceCalculator.fitsIn() should not take a cluster resource parameter. (Sen Zhao via wangda)

Posted by ae...@apache.org.
YARN-7172. ResourceCalculator.fitsIn() should not take a cluster resource parameter. (Sen Zhao via wangda)

Change-Id: Icc3670c9381ce7591ca69ec12da5aa52d3612d34


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e81596d0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e81596d0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e81596d0

Branch: refs/heads/HDFS-7240
Commit: e81596d06d226f1cfa44b2390ce3095ed4dee621
Parents: 8d7cc22
Author: Wangda Tan <wa...@apache.org>
Authored: Sun Sep 17 21:20:43 2017 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Sun Sep 17 21:20:43 2017 -0700

----------------------------------------------------------------------
 .../resource/DefaultResourceCalculator.java     |  3 +-
 .../resource/DominantResourceCalculator.java    |  2 +-
 .../yarn/util/resource/ResourceCalculator.java  |  3 +-
 .../hadoop/yarn/util/resource/Resources.java    |  4 +--
 .../util/resource/TestResourceCalculator.java   | 24 +++++++-------
 .../server/resourcemanager/RMServerUtils.java   |  3 +-
 .../CapacitySchedulerPreemptionUtils.java       |  4 +--
 ...QueuePriorityContainerCandidateSelector.java |  5 ++-
 .../ReservedContainerCandidatesSelector.java    | 34 +++++++++-----------
 .../scheduler/capacity/AbstractCSQueue.java     |  2 +-
 .../allocator/RegularContainerAllocator.java    |  8 ++---
 .../scheduler/common/fica/FiCaSchedulerApp.java | 21 +++++-------
 .../scheduler/capacity/TestReservations.java    | 20 +++++-------
 13 files changed, 55 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
index bdf60bd..7f155e7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
@@ -123,8 +123,7 @@ public class DefaultResourceCalculator extends ResourceCalculator {
   }
 
   @Override
-  public boolean fitsIn(Resource cluster,
-      Resource smaller, Resource bigger) {
+  public boolean fitsIn(Resource smaller, Resource bigger) {
     return smaller.getMemorySize() <= bigger.getMemorySize();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
index d64f03e..ca828a5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
@@ -538,7 +538,7 @@ public class DominantResourceCalculator extends ResourceCalculator {
   }
 
   @Override
-  public boolean fitsIn(Resource cluster, Resource smaller, Resource bigger) {
+  public boolean fitsIn(Resource smaller, Resource bigger) {
     int maxLength = ResourceUtils.getNumberOfKnownResourceTypes();
     for (int i = 0; i < maxLength; i++) {
       ResourceInformation sResourceInformation = smaller

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
index 398dac5..d59560f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
@@ -225,8 +225,7 @@ public abstract class ResourceCalculator {
   /**
    * Check if a smaller resource can be contained by bigger resource.
    */
-  public abstract boolean fitsIn(Resource cluster,
-      Resource smaller, Resource bigger);
+  public abstract boolean fitsIn(Resource smaller, Resource bigger);
 
   /**
    * Check if resource has any major resource types (which are all NodeManagers

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
index 325bce4..793aebf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
@@ -437,9 +437,9 @@ public class Resources {
     return true;
   }
 
-  public static boolean fitsIn(ResourceCalculator rc, Resource cluster,
+  public static boolean fitsIn(ResourceCalculator rc,
       Resource smaller, Resource bigger) {
-    return rc.fitsIn(cluster, smaller, bigger);
+    return rc.fitsIn(smaller, bigger);
   }
   
   public static Resource componentwiseMin(Resource lhs, Resource rhs) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java
index 5b4155c..5f3ed19 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java
@@ -24,14 +24,13 @@ import java.util.Collection;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
 @RunWith(Parameterized.class)
 public class TestResourceCalculator {
@@ -64,29 +63,28 @@ public class TestResourceCalculator {
   
   @Test(timeout = 10000)
   public void testFitsIn() {
-    Resource cluster = Resource.newInstance(1024, 1);
 
     if (resourceCalculator instanceof DefaultResourceCalculator) {
-      assertTrue(resourceCalculator.fitsIn(cluster,
+      Assert.assertTrue(resourceCalculator.fitsIn(
           Resource.newInstance(1, 2), Resource.newInstance(2, 1)));
-      assertTrue(resourceCalculator.fitsIn(cluster,
+      Assert.assertTrue(resourceCalculator.fitsIn(
           Resource.newInstance(1, 2), Resource.newInstance(2, 2)));
-      assertTrue(resourceCalculator.fitsIn(cluster,
+      Assert.assertTrue(resourceCalculator.fitsIn(
           Resource.newInstance(1, 2), Resource.newInstance(1, 2)));
-      assertTrue(resourceCalculator.fitsIn(cluster,
+      Assert.assertTrue(resourceCalculator.fitsIn(
           Resource.newInstance(1, 2), Resource.newInstance(1, 1)));
-      assertFalse(resourceCalculator.fitsIn(cluster,
+      Assert.assertFalse(resourceCalculator.fitsIn(
           Resource.newInstance(2, 1), Resource.newInstance(1, 2)));
     } else if (resourceCalculator instanceof DominantResourceCalculator) {
-      assertFalse(resourceCalculator.fitsIn(cluster,
+      Assert.assertFalse(resourceCalculator.fitsIn(
           Resource.newInstance(1, 2), Resource.newInstance(2, 1)));
-      assertTrue(resourceCalculator.fitsIn(cluster,
+      Assert.assertTrue(resourceCalculator.fitsIn(
           Resource.newInstance(1, 2), Resource.newInstance(2, 2)));
-      assertTrue(resourceCalculator.fitsIn(cluster,
+      Assert.assertTrue(resourceCalculator.fitsIn(
           Resource.newInstance(1, 2), Resource.newInstance(1, 2)));
-      assertFalse(resourceCalculator.fitsIn(cluster,
+      Assert.assertFalse(resourceCalculator.fitsIn(
           Resource.newInstance(1, 2), Resource.newInstance(1, 1)));
-      assertFalse(resourceCalculator.fitsIn(cluster,
+      Assert.assertFalse(resourceCalculator.fitsIn(
           Resource.newInstance(2, 1), Resource.newInstance(1, 2)));
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
index 2aae3a5..4934243 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
@@ -300,8 +300,7 @@ public class RMServerUtils {
     // Target resource of the increase request is more than NM can offer
     ResourceScheduler scheduler = rmContext.getScheduler();
     RMNode rmNode = request.getSchedulerNode().getRMNode();
-    if (!Resources.fitsIn(scheduler.getResourceCalculator(),
-        scheduler.getClusterResource(), targetResource,
+    if (!Resources.fitsIn(scheduler.getResourceCalculator(), targetResource,
         rmNode.getTotalCapability())) {
       String msg = "Target resource=" + targetResource + " of containerId="
           + containerId + " is more than node's total resource="

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java
index 0ae3ef0..f097e9c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java
@@ -156,8 +156,8 @@ public class CapacitySchedulerPreemptionUtils {
     if (null != toObtainByPartition
         && Resources.greaterThan(rc, clusterResource, toObtainByPartition,
             Resources.none())
-        && Resources.fitsIn(rc, clusterResource,
-            rmContainer.getAllocatedResource(), totalPreemptionAllowed)
+        && Resources.fitsIn(rc, rmContainer.getAllocatedResource(),
+            totalPreemptionAllowed)
         && !Resources.isAnyMajorResourceZero(rc, toObtainByPartition)) {
       Resources.subtractFrom(toObtainByPartition,
           rmContainer.getAllocatedResource());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/QueuePriorityContainerCandidateSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/QueuePriorityContainerCandidateSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/QueuePriorityContainerCandidateSelector.java
index c730a2d..7b7404c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/QueuePriorityContainerCandidateSelector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/QueuePriorityContainerCandidateSelector.java
@@ -229,8 +229,7 @@ public class QueuePriorityContainerCandidateSelector
 
     // If we already can allocate the reserved container after preemption,
     // skip following steps
-    if (Resources.fitsIn(rc, clusterResource, lacking,
-        Resources.none())) {
+    if (Resources.fitsIn(rc, lacking, Resources.none())) {
       return true;
     }
 
@@ -270,7 +269,7 @@ public class QueuePriorityContainerCandidateSelector
       }
 
       // Lacking <= 0 means we can allocate the reserved container
-      if (Resources.fitsIn(rc, clusterResource, lacking, Resources.none())) {
+      if (Resources.fitsIn(rc, lacking, Resources.none())) {
         return true;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ReservedContainerCandidatesSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ReservedContainerCandidatesSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ReservedContainerCandidatesSelector.java
index de23d0a..ff100d9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ReservedContainerCandidatesSelector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ReservedContainerCandidatesSelector.java
@@ -87,8 +87,8 @@ public class ReservedContainerCandidatesSelector
 
     // Get list of nodes for preemption, ordered by preemption cost
     List<NodeForPreemption> nodesForPreemption = getNodesForPreemption(
-        clusterResource, queueToPreemptableResourceByPartition,
-        selectedCandidates, totalPreemptedResourceAllowed);
+        queueToPreemptableResourceByPartition, selectedCandidates,
+        totalPreemptedResourceAllowed);
 
     for (NodeForPreemption nfp : nodesForPreemption) {
       RMContainer reservedContainer = nfp.schedulerNode.getReservedContainer();
@@ -97,9 +97,8 @@ public class ReservedContainerCandidatesSelector
       }
 
       NodeForPreemption preemptionResult = getPreemptionCandidatesOnNode(
-          nfp.schedulerNode, clusterResource,
-          queueToPreemptableResourceByPartition, selectedCandidates,
-          totalPreemptedResourceAllowed, false);
+          nfp.schedulerNode, queueToPreemptableResourceByPartition,
+          selectedCandidates, totalPreemptedResourceAllowed, false);
       if (null != preemptionResult) {
         for (RMContainer c : preemptionResult.selectedContainers) {
           ApplicationAttemptId appId = c.getApplicationAttemptId();
@@ -135,8 +134,7 @@ public class ReservedContainerCandidatesSelector
     return preemptable;
   }
 
-  private boolean tryToPreemptFromQueue(Resource cluster, String queueName,
-      String partitionName,
+  private boolean tryToPreemptFromQueue(String queueName, String partitionName,
       Map<String, Map<String, Resource>> queueToPreemptableResourceByPartition,
       Resource required, Resource totalPreemptionAllowed, boolean readOnly) {
     Resource preemptable = getPreemptableResource(queueName, partitionName,
@@ -145,11 +143,11 @@ public class ReservedContainerCandidatesSelector
       return false;
     }
 
-    if (!Resources.fitsIn(rc, cluster, required, preemptable)) {
+    if (!Resources.fitsIn(rc, required, preemptable)) {
       return false;
     }
 
-    if (!Resources.fitsIn(rc, cluster, required, totalPreemptionAllowed)) {
+    if (!Resources.fitsIn(rc, required, totalPreemptionAllowed)) {
       return false;
     }
 
@@ -165,7 +163,6 @@ public class ReservedContainerCandidatesSelector
   /**
    * Try to check if we can preempt resources for reserved container in given node
    * @param node
-   * @param cluster
    * @param queueToPreemptableResourceByPartition it's a map of
    *                 <queueName, <partition, preemptable-resource>>
    * @param readOnly do we want to modify preemptable resource after we selected
@@ -174,7 +171,7 @@ public class ReservedContainerCandidatesSelector
    * to satisfy reserved resource
    */
   private NodeForPreemption getPreemptionCandidatesOnNode(
-      FiCaSchedulerNode node, Resource cluster,
+      FiCaSchedulerNode node,
       Map<String, Map<String, Resource>> queueToPreemptableResourceByPartition,
       Map<ApplicationAttemptId, Set<RMContainer>> selectedCandidates,
       Resource totalPreemptionAllowed, boolean readOnly) {
@@ -204,8 +201,7 @@ public class ReservedContainerCandidatesSelector
     String partition = node.getPartition();
 
     // Avoid preempt any container if required <= available + killable
-    if (Resources.fitsIn(rc, cluster, reservedContainer.getReservedResource(),
-        cur)) {
+    if (Resources.fitsIn(rc, reservedContainer.getReservedResource(), cur)) {
       return null;
     }
 
@@ -232,9 +228,9 @@ public class ReservedContainerCandidatesSelector
 
       // Can we preempt container c?
       // Check if we have quota to preempt this container
-      boolean canPreempt = tryToPreemptFromQueue(cluster, containerQueueName,
-          partition, queueToPreemptableResourceByPartition,
-          c.getAllocatedResource(), totalPreemptionAllowed, readOnly);
+      boolean canPreempt = tryToPreemptFromQueue(containerQueueName, partition,
+          queueToPreemptableResourceByPartition, c.getAllocatedResource(),
+          totalPreemptionAllowed, readOnly);
 
       // If we can, add to selected container, and change resource accordingly.
       if (canPreempt) {
@@ -246,7 +242,7 @@ public class ReservedContainerCandidatesSelector
           Resources.addTo(totalSelected, c.getAllocatedResource());
         }
         Resources.addTo(cur, c.getAllocatedResource());
-        if (Resources.fitsIn(rc, cluster,
+        if (Resources.fitsIn(rc,
             reservedContainer.getReservedResource(), cur)) {
           canAllocateReservedContainer = true;
           break;
@@ -282,7 +278,7 @@ public class ReservedContainerCandidatesSelector
     return nfp;
   }
 
-  private List<NodeForPreemption> getNodesForPreemption(Resource cluster,
+  private List<NodeForPreemption> getNodesForPreemption(
       Map<String, Map<String, Resource>> queueToPreemptableResourceByPartition,
       Map<ApplicationAttemptId, Set<RMContainer>> selectedCandidates,
       Resource totalPreemptionAllowed) {
@@ -292,7 +288,7 @@ public class ReservedContainerCandidatesSelector
     for (FiCaSchedulerNode node : preemptionContext.getScheduler()
         .getAllNodes()) {
       if (node.getReservedContainer() != null) {
-        NodeForPreemption nfp = getPreemptionCandidatesOnNode(node, cluster,
+        NodeForPreemption nfp = getPreemptionCandidatesOnNode(node,
             queueToPreemptableResourceByPartition, selectedCandidates,
             totalPreemptionAllowed, true);
         if (null != nfp) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
index d7c452a..250f4e6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
@@ -909,7 +909,7 @@ public abstract class AbstractCSQueue implements CSQueue {
           maxResourceLimit = labelManager.getResourceByLabel(
               schedulerContainer.getNodePartition(), cluster);
         }
-        if (!Resources.fitsIn(resourceCalculator, cluster,
+        if (!Resources.fitsIn(resourceCalculator,
             Resources.add(queueUsage.getUsed(partition), netAllocated),
             maxResourceLimit)) {
           if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
index f753d31..72dfbdd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
@@ -548,10 +548,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
         toKillContainers.add(killableContainer);
         Resources.addTo(availableAndKillable,
                         killableContainer.getAllocatedResource());
-        if (Resources.fitsIn(rc,
-                             clusterResource,
-                             capability,
-                             availableAndKillable)) {
+        if (Resources.fitsIn(rc, capability, availableAndKillable)) {
           // Stop if we find enough spaces
           availableContainers = 1;
           break;
@@ -579,8 +576,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
             // under the limit.
             resourceNeedToUnReserve = capability;
           }
-          unreservedContainer =
-              application.findNodeToUnreserve(clusterResource, node,
+          unreservedContainer = application.findNodeToUnreserve(node,
                   schedulerKey, resourceNeedToUnReserve);
           // When (minimum-unreserved-resource > 0 OR we cannot allocate
           // new/reserved

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
index 17bb104..a12c5ec 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
@@ -314,7 +314,6 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
   }
 
   private boolean commonCheckContainerAllocation(
-      Resource cluster,
       ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode> allocation,
       SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode> schedulerContainer) {
     // Make sure node is not reserved by anyone else
@@ -355,8 +354,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
         }
       }
     }
-    if (!Resources.fitsIn(rc, cluster,
-        allocation.getAllocatedOrReservedResource(),
+    if (!Resources.fitsIn(rc, allocation.getAllocatedOrReservedResource(),
         availableResource)) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Node doesn't have enough available resource, asked="
@@ -419,8 +417,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
           // Common part of check container allocation regardless if it is a
           // increase container or regular container
-          commonCheckContainerAllocation(cluster, allocation,
-              schedulerContainer);
+          commonCheckContainerAllocation(allocation, schedulerContainer);
         } else {
           // Container reserved first time will be NEW, after the container
           // accepted & confirmed, it will become RESERVED state
@@ -721,9 +718,8 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
   }
 
   @VisibleForTesting
-  public NodeId getNodeIdToUnreserve(
-      SchedulerRequestKey schedulerKey, Resource resourceNeedUnreserve,
-      ResourceCalculator rc, Resource clusterResource) {
+  public NodeId getNodeIdToUnreserve(SchedulerRequestKey schedulerKey,
+      Resource resourceNeedUnreserve, ResourceCalculator resourceCalculator) {
     // first go around make this algorithm simple and just grab first
     // reservation that has enough resources
     Map<NodeId, RMContainer> reservedContainers = this.reservedContainers.get(
@@ -738,7 +734,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
         // make sure we unreserve one with at least the same amount of
         // resources, otherwise could affect capacity limits
-        if (Resources.fitsIn(rc, clusterResource, resourceNeedUnreserve,
+        if (Resources.fitsIn(resourceCalculator, resourceNeedUnreserve,
             reservedResource)) {
           if (LOG.isDebugEnabled()) {
             LOG.debug(
@@ -806,14 +802,13 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
   }
 
   @VisibleForTesting
-  public RMContainer findNodeToUnreserve(Resource clusterResource,
-      FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
-      Resource minimumUnreservedResource) {
+  public RMContainer findNodeToUnreserve(FiCaSchedulerNode node,
+      SchedulerRequestKey schedulerKey, Resource minimumUnreservedResource) {
     try {
       readLock.lock();
       // need to unreserve some other container first
       NodeId idToUnreserve = getNodeIdToUnreserve(schedulerKey,
-          minimumUnreservedResource, rc, clusterResource);
+          minimumUnreservedResource, rc);
       if (idToUnreserve == null) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("checked to see if could unreserve for app but nothing "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
index 5e6548b..32f022f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
@@ -896,8 +896,7 @@ public class TestReservations {
     String host_1 = "host_1";
     FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0,
         8 * GB);
-    
-    Resource clusterResource = Resources.createResource(2 * 8 * GB);
+
 
     // Setup resource-requests
     Priority p = TestUtils.createMockPriority(5);
@@ -933,28 +932,27 @@ public class TestReservations {
         node_0.getNodeID(), "user", rmContext);
 
     // no reserved containers
-    NodeId unreserveId =
-        app_0.getNodeIdToUnreserve(priorityMap, capability,
-            cs.getResourceCalculator(), clusterResource);
+    NodeId unreserveId = app_0.getNodeIdToUnreserve(priorityMap, capability,
+            cs.getResourceCalculator());
     assertEquals(null, unreserveId);
 
     // no reserved containers - reserve then unreserve
     app_0.reserve(node_0, priorityMap, rmContainer_1, container_1);
     app_0.unreserve(priorityMap, node_0, rmContainer_1);
     unreserveId = app_0.getNodeIdToUnreserve(priorityMap, capability,
-        cs.getResourceCalculator(), clusterResource);
+        cs.getResourceCalculator());
     assertEquals(null, unreserveId);
 
     // no container large enough is reserved
     app_0.reserve(node_0, priorityMap, rmContainer_1, container_1);
     unreserveId = app_0.getNodeIdToUnreserve(priorityMap, capability,
-        cs.getResourceCalculator(), clusterResource);
+        cs.getResourceCalculator());
     assertEquals(null, unreserveId);
 
     // reserve one that is now large enough
     app_0.reserve(node_1, priorityMap, rmContainer, container);
     unreserveId = app_0.getNodeIdToUnreserve(priorityMap, capability,
-        cs.getResourceCalculator(), clusterResource);
+        cs.getResourceCalculator());
     assertEquals(node_1.getNodeID(), unreserveId);
   }
 
@@ -1001,16 +999,14 @@ public class TestReservations {
         node_1.getNodeID(), "user", rmContext);
 
     // nothing reserved
-    RMContainer toUnreserveContainer =
-        app_0.findNodeToUnreserve(csContext.getClusterResource(), node_1,
+    RMContainer toUnreserveContainer = app_0.findNodeToUnreserve(node_1,
             priorityMap, capability);
     assertTrue(toUnreserveContainer == null);
 
     // reserved but scheduler doesn't know about that node.
     app_0.reserve(node_1, priorityMap, rmContainer, container);
     node_1.reserveResource(app_0, priorityMap, rmContainer);
-    toUnreserveContainer =
-        app_0.findNodeToUnreserve(csContext.getClusterResource(), node_1,
+    toUnreserveContainer = app_0.findNodeToUnreserve(node_1,
             priorityMap, capability);
     assertTrue(toUnreserveContainer == null);
   }


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


[15/50] [abbrv] hadoop git commit: HDFS-12395. Support erasure coding policy operations in namenode edit log. Contributed by Sammi Chen

Posted by ae...@apache.org.
HDFS-12395. Support erasure coding policy operations in namenode edit log. Contributed by Sammi Chen


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/08d996d3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/08d996d3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/08d996d3

Branch: refs/heads/HDFS-7240
Commit: 08d996d3e9265efad737efad50cbc5b10a0202f8
Parents: ae8f55b
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Sep 15 09:43:39 2017 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Fri Sep 15 09:43:39 2017 +0800

----------------------------------------------------------------------
 .../server/namenode/FSDirErasureCodingOp.java   |  53 ++-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |  37 +++
 .../hdfs/server/namenode/FSEditLogLoader.java   |  43 +++
 .../hdfs/server/namenode/FSEditLogOp.java       | 325 +++++++++++++++++++
 .../hdfs/server/namenode/FSEditLogOpCodes.java  |   5 +
 .../server/namenode/FSImageSerialization.java   |  45 +++
 .../hdfs/server/namenode/FSNamesystem.java      |  38 ++-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  52 ++-
 .../src/site/markdown/HdfsEditsViewer.md        |   4 +
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  30 ++
 .../hdfs/TestDFSInotifyEventInputStream.java    |   2 +-
 .../server/namenode/TestFSEditLogLoader.java    |  83 +++++
 .../TestOfflineEditsViewer.java                 |   2 +-
 .../hadoop-hdfs/src/test/resources/editsStored  | Bin 5850 -> 6293 bytes
 .../src/test/resources/editsStored.xml          |  90 ++++-
 15 files changed, 779 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d996d3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
index 381a871..181b147 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
@@ -208,10 +208,22 @@ final class FSDirErasureCodingOp {
     return fsd.getAuditFileInfo(iip);
   }
 
-  static ErasureCodingPolicy addErasureCodePolicy(final FSNamesystem fsn,
-      ErasureCodingPolicy policy) {
+  /**
+   * Add an erasure coding policy.
+   *
+   * @param fsn namespace
+   * @param policy the new policy to be added into system
+   * @param logRetryCache whether to record RPC ids in editlog for retry cache
+   *                      rebuilding
+   * @throws IOException
+   */
+  static ErasureCodingPolicy addErasureCodingPolicy(final FSNamesystem fsn,
+      ErasureCodingPolicy policy, final boolean logRetryCache) {
     Preconditions.checkNotNull(policy);
-    return fsn.getErasureCodingPolicyManager().addPolicy(policy);
+    ErasureCodingPolicy retPolicy =
+        fsn.getErasureCodingPolicyManager().addPolicy(policy);
+    fsn.getEditLog().logAddErasureCodingPolicy(policy, logRetryCache);
+    return retPolicy;
   }
 
   /**
@@ -219,24 +231,47 @@ final class FSDirErasureCodingOp {
    *
    * @param fsn namespace
    * @param ecPolicyName the name of the policy to be removed
+   * @param logRetryCache whether to record RPC ids in editlog for retry cache
+   *                      rebuilding
    * @throws IOException
    */
-  static void removeErasureCodePolicy(final FSNamesystem fsn,
-      String ecPolicyName) throws IOException {
+  static void removeErasureCodingPolicy(final FSNamesystem fsn,
+      String ecPolicyName, final boolean logRetryCache) throws IOException {
     Preconditions.checkNotNull(ecPolicyName);
     fsn.getErasureCodingPolicyManager().removePolicy(ecPolicyName);
+    fsn.getEditLog().logRemoveErasureCodingPolicy(ecPolicyName, logRetryCache);
   }
 
-  static void enableErasureCodePolicy(final FSNamesystem fsn,
-      String ecPolicyName) throws IOException {
+  /**
+   * Enable an erasure coding policy.
+   *
+   * @param fsn namespace
+   * @param ecPolicyName the name of the policy to be enabled
+   * @param logRetryCache whether to record RPC ids in editlog for retry cache
+   *                      rebuilding
+   * @throws IOException
+   */
+  static void enableErasureCodingPolicy(final FSNamesystem fsn,
+      String ecPolicyName, final boolean logRetryCache) throws IOException {
     Preconditions.checkNotNull(ecPolicyName);
     fsn.getErasureCodingPolicyManager().enablePolicy(ecPolicyName);
+    fsn.getEditLog().logEnableErasureCodingPolicy(ecPolicyName, logRetryCache);
   }
 
-  static void disableErasureCodePolicy(final FSNamesystem fsn,
-      String ecPolicyName) throws IOException {
+  /**
+   * Disable an erasure coding policy.
+   *
+   * @param fsn namespace
+   * @param ecPolicyName the name of the policy to be disabled
+   * @param logRetryCache whether to record RPC ids in editlog for retry cache
+   *                      rebuilding
+   * @throws IOException
+   */
+  static void disableErasureCodingPolicy(final FSNamesystem fsn,
+      String ecPolicyName, final boolean logRetryCache) throws IOException {
     Preconditions.checkNotNull(ecPolicyName);
     fsn.getErasureCodingPolicyManager().disablePolicy(ecPolicyName);
+    fsn.getEditLog().logDisableErasureCodingPolicy(ecPolicyName, logRetryCache);
   }
 
   private static List<XAttr> removeErasureCodingPolicyXAttr(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d996d3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index d3f4447..a8f5bfa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -97,6 +98,10 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TruncateOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddErasureCodingPolicyOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.EnableErasureCodingPolicyOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DisableErasureCodingPolicyOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveErasureCodingPolicyOp;
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
@@ -1228,6 +1233,38 @@ public class FSEditLog implements LogsPurgeable {
     logEdit(op);
   }
 
+  void logAddErasureCodingPolicy(ErasureCodingPolicy ecPolicy,
+      boolean toLogRpcIds) {
+    AddErasureCodingPolicyOp op =
+        AddErasureCodingPolicyOp.getInstance(cache.get());
+    op.setErasureCodingPolicy(ecPolicy);
+    logRpcIds(op, toLogRpcIds);
+    logEdit(op);
+  }
+
+  void logEnableErasureCodingPolicy(String ecPolicyName, boolean toLogRpcIds) {
+    EnableErasureCodingPolicyOp op =
+        EnableErasureCodingPolicyOp.getInstance(cache.get());
+    op.setErasureCodingPolicy(ecPolicyName);
+    logRpcIds(op, toLogRpcIds);
+    logEdit(op);
+  }
+
+  void logDisableErasureCodingPolicy(String ecPolicyName, boolean toLogRpcIds) {
+    DisableErasureCodingPolicyOp op =
+        DisableErasureCodingPolicyOp.getInstance(cache.get());
+    op.setErasureCodingPolicy(ecPolicyName);
+    logRpcIds(op, toLogRpcIds);
+    logEdit(op);
+  }
+
+  void logRemoveErasureCodingPolicy(String ecPolicyName, boolean toLogRpcIds) {
+    RemoveErasureCodingPolicyOp op =
+        RemoveErasureCodingPolicyOp.getInstance(cache.get());
+    op.setErasureCodingPolicy(ecPolicyName);
+    logRpcIds(op, toLogRpcIds);
+    logEdit(op);
+  }
   /**
    * Get all the journals this edit log is currently operating on.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d996d3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index bc62a7e..a21b8ea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -96,6 +96,14 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TruncateOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp
+    .AddErasureCodingPolicyOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp
+    .RemoveErasureCodingPolicyOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp
+    .EnableErasureCodingPolicyOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp
+    .DisableErasureCodingPolicyOp;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
@@ -958,6 +966,41 @@ public class FSEditLogLoader {
           setStoragePolicyOp.policyId);
       break;
     }
+    case OP_ADD_ERASURE_CODING_POLICY:
+      AddErasureCodingPolicyOp addOp = (AddErasureCodingPolicyOp) op;
+      fsNamesys.getErasureCodingPolicyManager().addPolicy(
+          addOp.getEcPolicy());
+
+      if (toAddRetryCache) {
+        fsNamesys.addCacheEntryWithPayload(op.rpcClientId, op.rpcCallId,
+            addOp.getEcPolicy());
+      }
+      break;
+    case OP_ENABLE_ERASURE_CODING_POLICY:
+      EnableErasureCodingPolicyOp enableOp = (EnableErasureCodingPolicyOp) op;
+      fsNamesys.getErasureCodingPolicyManager().enablePolicy(
+          enableOp.getEcPolicy());
+      if (toAddRetryCache) {
+        fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
+      }
+      break;
+    case OP_DISABLE_ERASURE_CODING_POLICY:
+      DisableErasureCodingPolicyOp disableOp =
+          (DisableErasureCodingPolicyOp) op;
+      fsNamesys.getErasureCodingPolicyManager().disablePolicy(
+          disableOp.getEcPolicy());
+      if (toAddRetryCache) {
+        fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
+      }
+      break;
+    case OP_REMOVE_ERASURE_CODING_POLICY:
+      RemoveErasureCodingPolicyOp removeOp = (RemoveErasureCodingPolicyOp) op;
+      fsNamesys.getErasureCodingPolicyManager().removePolicy(
+          removeOp.getEcPolicy());
+      if (toAddRetryCache) {
+        fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
+      }
+      break;
     default:
       throw new IOException("Invalid operation read " + op.opCode);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d996d3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
index f93e867..2dc9d33 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD;
+import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_ERASURE_CODING_POLICY;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_APPEND;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_BLOCK;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_CACHE_DIRECTIVE;
@@ -31,7 +32,9 @@ import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_CONCAT_
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_CREATE_SNAPSHOT;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_DELETE;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_DELETE_SNAPSHOT;
+import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_DISABLE_ERASURE_CODING_POLICY;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_DISALLOW_SNAPSHOT;
+import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ENABLE_ERASURE_CODING_POLICY;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_END_LOG_SEGMENT;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_GET_DELEGATION_TOKEN;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_INVALID;
@@ -41,6 +44,7 @@ import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_MODIFY_
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_REASSIGN_LEASE;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_REMOVE_CACHE_DIRECTIVE;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_REMOVE_CACHE_POOL;
+import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_REMOVE_ERASURE_CODING_POLICY;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_REMOVE_XATTR;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENAME;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENAME_OLD;
@@ -75,7 +79,9 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumMap;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.zip.CheckedInputStream;
 import java.util.zip.Checksum;
 
@@ -100,6 +106,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
@@ -119,6 +126,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableFactories;
 import org.apache.hadoop.io.WritableFactory;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.ipc.ClientId;
 import org.apache.hadoop.ipc.RpcConstants;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
@@ -4339,6 +4347,323 @@ public abstract class FSEditLogOp {
       this.len = in.readLong();
     }
   }
+
+  /**
+   * Operation corresponding to add an erasure coding policy.
+   */
+  static class AddErasureCodingPolicyOp extends FSEditLogOp {
+    private ErasureCodingPolicy ecPolicy;
+
+    AddErasureCodingPolicyOp() {
+      super(OP_ADD_ERASURE_CODING_POLICY);
+    }
+
+    static AddErasureCodingPolicyOp getInstance(OpInstanceCache cache) {
+      return (AddErasureCodingPolicyOp) cache
+          .get(OP_ADD_ERASURE_CODING_POLICY);
+    }
+
+    @Override
+    void resetSubFields() {
+      this.ecPolicy = null;
+    }
+
+    public ErasureCodingPolicy getEcPolicy() {
+      return this.ecPolicy;
+    }
+
+    public AddErasureCodingPolicyOp setErasureCodingPolicy(
+        ErasureCodingPolicy policy) {
+      Preconditions.checkNotNull(policy.getName());
+      Preconditions.checkNotNull(policy.getSchema());
+      Preconditions.checkArgument(policy.getCellSize() > 0);
+      this.ecPolicy = policy;
+      return this;
+    }
+
+    @Override
+    void readFields(DataInputStream in, int logVersion) throws IOException {
+      this.ecPolicy = FSImageSerialization.readErasureCodingPolicy(in);
+      readRpcIds(in, logVersion);
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out) throws IOException {
+      Preconditions.checkNotNull(ecPolicy);
+      FSImageSerialization.writeErasureCodingPolicy(out, ecPolicy);
+      writeRpcIds(rpcClientId, rpcCallId, out);
+    }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      Preconditions.checkNotNull(ecPolicy);
+      XMLUtils.addSaxString(contentHandler, "CODEC", ecPolicy.getCodecName());
+      XMLUtils.addSaxString(contentHandler, "DATAUNITS",
+          Integer.toString(ecPolicy.getNumDataUnits()));
+      XMLUtils.addSaxString(contentHandler, "PARITYUNITS",
+          Integer.toString(ecPolicy.getNumParityUnits()));
+      XMLUtils.addSaxString(contentHandler, "CELLSIZE",
+          Integer.toString(ecPolicy.getCellSize()));
+
+      Map<String, String> extraOptions = ecPolicy.getSchema().getExtraOptions();
+      if (extraOptions == null || extraOptions.isEmpty()) {
+        XMLUtils.addSaxString(contentHandler, "EXTRAOPTIONS",
+            Integer.toString(0));
+        appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
+        return;
+      }
+
+      XMLUtils.addSaxString(contentHandler, "EXTRAOPTIONS",
+          Integer.toString(extraOptions.size()));
+
+      for (Map.Entry<String, String> entry : extraOptions.entrySet()) {
+        contentHandler.startElement("", "", "EXTRAOPTION",
+            new AttributesImpl());
+        XMLUtils.addSaxString(contentHandler, "KEY", entry.getKey());
+        XMLUtils.addSaxString(contentHandler, "VALUE", entry.getValue());
+        contentHandler.endElement("", "", "EXTRAOPTION");
+      }
+      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
+    }
+
+    @Override
+    void fromXml(Stanza st) throws InvalidXmlException {
+      final String codecName = st.getValue("CODEC");
+      final int dataUnits = Integer.parseInt(st.getValue("DATAUNITS"));
+      final int parityUnits = Integer.parseInt(st.getValue("PARITYUNITS"));
+      final int cellSize = Integer.parseInt(st.getValue("CELLSIZE"));
+      final int extraOptionNum = Integer.parseInt(st.getValue("EXTRAOPTIONS"));
+
+      ECSchema schema;
+      if (extraOptionNum == 0) {
+        schema = new ECSchema(codecName, dataUnits, parityUnits, null);
+      } else {
+        Map<String, String> extraOptions = new HashMap<String, String>();
+        List<Stanza> stanzas = st.getChildren("EXTRAOPTION");
+        for (Stanza a: stanzas) {
+          extraOptions.put(a.getValue("KEY"), a.getValue("VALUE"));
+        }
+        schema = new ECSchema(codecName, dataUnits, parityUnits, extraOptions);
+      }
+      this.ecPolicy = new ErasureCodingPolicy(schema, cellSize);
+      readRpcIdsFromXml(st);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("AddErasureCodingPolicy [");
+      builder.append(ecPolicy.toString());
+
+      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
+      builder.append("]");
+      return builder.toString();
+    }
+  }
+
+  /**
+   * Operation corresponding to enable an erasure coding policy.
+   */
+  static class EnableErasureCodingPolicyOp extends FSEditLogOp {
+    private String ecPolicyName;
+
+    EnableErasureCodingPolicyOp() {
+      super(OP_ENABLE_ERASURE_CODING_POLICY);
+    }
+
+    static EnableErasureCodingPolicyOp getInstance(OpInstanceCache cache) {
+      return (EnableErasureCodingPolicyOp) cache
+          .get(OP_ENABLE_ERASURE_CODING_POLICY);
+    }
+
+    @Override
+    void resetSubFields() {
+      this.ecPolicyName = null;
+    }
+
+    public String getEcPolicy() {
+      return this.ecPolicyName;
+    }
+
+    public EnableErasureCodingPolicyOp setErasureCodingPolicy(
+        String policyName) {
+      Preconditions.checkNotNull(policyName);
+      this.ecPolicyName = policyName;
+      return this;
+    }
+
+    @Override
+    void readFields(DataInputStream in, int logVersion) throws IOException {
+      this.ecPolicyName = FSImageSerialization.readString(in);
+      readRpcIds(in, logVersion);
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out) throws IOException {
+      Preconditions.checkNotNull(ecPolicyName);
+      FSImageSerialization.writeString(ecPolicyName, out);
+      writeRpcIds(rpcClientId, rpcCallId, out);
+    }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      Preconditions.checkNotNull(ecPolicyName);
+      XMLUtils.addSaxString(contentHandler, "POLICYNAME", this.ecPolicyName);
+      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
+    }
+
+    @Override
+    void fromXml(Stanza st) throws InvalidXmlException {
+      this.ecPolicyName = st.getValue("POLICYNAME");
+      readRpcIdsFromXml(st);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("EnableErasureCodingPolicy [");
+      builder.append(ecPolicyName);
+
+      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
+      builder.append("]");
+      return builder.toString();
+    }
+  }
+
+  /**
+   * Operation corresponding to disable an erasure coding policy.
+   */
+  static class DisableErasureCodingPolicyOp extends FSEditLogOp {
+    private String ecPolicyName;
+
+    DisableErasureCodingPolicyOp() {
+      super(OP_DISABLE_ERASURE_CODING_POLICY);
+    }
+
+    static DisableErasureCodingPolicyOp getInstance(OpInstanceCache cache) {
+      return (DisableErasureCodingPolicyOp) cache
+          .get(OP_DISABLE_ERASURE_CODING_POLICY);
+    }
+
+    @Override
+    void resetSubFields() {
+      this.ecPolicyName = null;
+    }
+
+    public String getEcPolicy() {
+      return this.ecPolicyName;
+    }
+
+    public DisableErasureCodingPolicyOp setErasureCodingPolicy(
+        String policyName) {
+      Preconditions.checkNotNull(policyName);
+      this.ecPolicyName = policyName;
+      return this;
+    }
+
+    @Override
+    void readFields(DataInputStream in, int logVersion) throws IOException {
+      this.ecPolicyName = FSImageSerialization.readString(in);
+      readRpcIds(in, logVersion);
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeString(ecPolicyName, out);
+      writeRpcIds(rpcClientId, rpcCallId, out);
+    }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "POLICYNAME", this.ecPolicyName);
+      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
+    }
+
+    @Override
+    void fromXml(Stanza st) throws InvalidXmlException {
+      this.ecPolicyName = st.getValue("POLICYNAME");
+      readRpcIdsFromXml(st);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("DisableErasureCodingPolicy [");
+      builder.append(ecPolicyName);
+
+      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
+      builder.append("]");
+      return builder.toString();
+    }
+  }
+
+  /**
+   * Operation corresponding to remove an erasure coding policy.
+   */
+  static class RemoveErasureCodingPolicyOp extends FSEditLogOp {
+    private String ecPolicyName;
+
+    RemoveErasureCodingPolicyOp() {
+      super(OP_REMOVE_ERASURE_CODING_POLICY);
+    }
+
+    static RemoveErasureCodingPolicyOp getInstance(OpInstanceCache cache) {
+      return (RemoveErasureCodingPolicyOp) cache
+          .get(OP_REMOVE_ERASURE_CODING_POLICY);
+    }
+
+    @Override
+    void resetSubFields() {
+      this.ecPolicyName = null;
+    }
+
+    public String getEcPolicy() {
+      return this.ecPolicyName;
+    }
+
+    public RemoveErasureCodingPolicyOp setErasureCodingPolicy(
+        String policyName) {
+      Preconditions.checkNotNull(policyName);
+      this.ecPolicyName = policyName;
+      return this;
+    }
+
+    @Override
+    void readFields(DataInputStream in, int logVersion) throws IOException {
+      this.ecPolicyName = FSImageSerialization.readString(in);
+      readRpcIds(in, logVersion);
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeString(ecPolicyName, out);
+      writeRpcIds(rpcClientId, rpcCallId, out);
+    }
+
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "POLICYNAME", this.ecPolicyName);
+      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
+    }
+
+    @Override
+    void fromXml(Stanza st) throws InvalidXmlException {
+      this.ecPolicyName = st.getValue("POLICYNAME");
+      readRpcIdsFromXml(st);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("RemoveErasureCodingPolicy [");
+      builder.append(ecPolicyName);
+
+      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
+      builder.append("]");
+      return builder.toString();
+    }
+  }
+
   /**
    * Operation corresponding to upgrade
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d996d3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java
index 3f8feba..ce42e3f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java
@@ -80,6 +80,11 @@ public enum FSEditLogOpCodes {
   OP_TRUNCATE                   ((byte) 46, TruncateOp.class),
   OP_APPEND                     ((byte) 47, AppendOp.class),
   OP_SET_QUOTA_BY_STORAGETYPE   ((byte) 48, SetQuotaByStorageTypeOp.class),
+  OP_ADD_ERASURE_CODING_POLICY  ((byte) 49, AddErasureCodingPolicyOp.class),
+  OP_ENABLE_ERASURE_CODING_POLICY((byte) 50, EnableErasureCodingPolicyOp.class),
+  OP_DISABLE_ERASURE_CODING_POLICY((byte) 51,
+      DisableErasureCodingPolicyOp.class),
+  OP_REMOVE_ERASURE_CODING_POLICY((byte) 52, RemoveErasureCodingPolicyOp.class),
 
   // Note that the current range of the valid OP code is 0~127
   OP_INVALID                    ((byte) -1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d996d3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
index fd2700b..4d8b627 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
@@ -21,6 +21,8 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -33,6 +35,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
@@ -46,6 +49,7 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.xml.sax.ContentHandler;
 import org.xml.sax.SAXException;
 
@@ -750,4 +754,45 @@ public class FSImageSerialization {
     return info;
   }
 
+  public static void writeErasureCodingPolicy(DataOutputStream out,
+      ErasureCodingPolicy ecPolicy) throws IOException {
+    writeString(ecPolicy.getSchema().getCodecName(), out);
+    writeInt(ecPolicy.getNumDataUnits(), out);
+    writeInt(ecPolicy.getNumParityUnits(), out);
+    writeInt(ecPolicy.getCellSize(), out);
+
+    Map<String, String> extraOptions = ecPolicy.getSchema().getExtraOptions();
+    if (extraOptions == null || extraOptions.isEmpty()) {
+      writeInt(0, out);
+      return;
+    }
+
+    writeInt(extraOptions.size(), out);
+    for (Map.Entry<String, String> entry : extraOptions.entrySet()) {
+      writeString(entry.getKey(), out);
+      writeString(entry.getValue(), out);
+    }
+  }
+
+  public static ErasureCodingPolicy readErasureCodingPolicy(DataInput in)
+      throws IOException {
+    String codecName = readString(in);
+    int numDataUnits = readInt(in);
+    int numParityUnits = readInt(in);
+    int cellSize = readInt(in);
+
+    int size = readInt(in);
+    Map<String, String> extraOptions = new HashMap<>(size);
+
+    if (size != 0) {
+      for (int i = 0; i < size; i++) {
+        String key = readString(in);
+        String value = readString(in);
+        extraOptions.put(key, value);
+      }
+    }
+    ECSchema ecSchema = new ECSchema(codecName, numDataUnits,
+        numParityUnits, extraOptions);
+    return new ErasureCodingPolicy(ecSchema, cellSize);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d996d3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index d9f3c0e..029557f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -7189,10 +7189,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /**
    * Add multiple erasure coding policies to the ErasureCodingPolicyManager.
    * @param policies The policies to add.
+   * @param logRetryCache whether to record RPC ids in editlog for retry cache
+   *                      rebuilding
    * @return The according result of add operation.
    */
-  AddECPolicyResponse[] addErasureCodingPolicies(ErasureCodingPolicy[] policies)
-      throws IOException {
+  AddECPolicyResponse[] addErasureCodingPolicies(ErasureCodingPolicy[] policies,
+      final boolean logRetryCache) throws IOException {
     final String operationName = "addErasureCodingPolicies";
     String addECPolicyName = "";
     checkOperation(OperationCategory.WRITE);
@@ -7201,12 +7203,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
+      checkNameNodeSafeMode("Cannot add erasure coding policy");
       for (ErasureCodingPolicy policy : policies) {
         try {
-          checkOperation(OperationCategory.WRITE);
-          checkNameNodeSafeMode("Cannot add erasure coding policy");
           ErasureCodingPolicy newPolicy =
-              FSDirErasureCodingOp.addErasureCodePolicy(this, policy);
+              FSDirErasureCodingOp.addErasureCodingPolicy(this, policy,
+                  logRetryCache);
           addECPolicyName = newPolicy.getName();
           responses.add(new AddECPolicyResponse(newPolicy));
         } catch (HadoopIllegalArgumentException e) {
@@ -7227,9 +7229,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /**
    * Remove an erasure coding policy.
    * @param ecPolicyName the name of the policy to be removed
+   * @param logRetryCache whether to record RPC ids in editlog for retry cache
+   *                      rebuilding
    * @throws IOException
    */
-  void removeErasureCodingPolicy(String ecPolicyName) throws IOException {
+  void removeErasureCodingPolicy(String ecPolicyName,
+      final boolean logRetryCache) throws IOException {
     final String operationName = "removeErasureCodingPolicy";
     checkOperation(OperationCategory.WRITE);
     boolean success = false;
@@ -7238,23 +7243,27 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot remove erasure coding policy "
           + ecPolicyName);
-      FSDirErasureCodingOp.removeErasureCodePolicy(this, ecPolicyName);
+      FSDirErasureCodingOp.removeErasureCodingPolicy(this, ecPolicyName,
+          logRetryCache);
       success = true;
     } finally {
       writeUnlock(operationName);
       if (success) {
         getEditLog().logSync();
       }
-      logAuditEvent(success, operationName, null, null, null);
+      logAuditEvent(success, operationName, ecPolicyName, null, null);
     }
   }
 
   /**
    * Enable an erasure coding policy.
    * @param ecPolicyName the name of the policy to be enabled
+   * @param logRetryCache whether to record RPC ids in editlog for retry cache
+   *                      rebuilding
    * @throws IOException
    */
-  void enableErasureCodingPolicy(String ecPolicyName) throws IOException {
+  void enableErasureCodingPolicy(String ecPolicyName,
+      final boolean logRetryCache) throws IOException {
     final String operationName = "enableErasureCodingPolicy";
     checkOperation(OperationCategory.WRITE);
     boolean success = false;
@@ -7264,7 +7273,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot enable erasure coding policy "
           + ecPolicyName);
-      FSDirErasureCodingOp.enableErasureCodePolicy(this, ecPolicyName);
+      FSDirErasureCodingOp.enableErasureCodingPolicy(this, ecPolicyName,
+          logRetryCache);
       success = true;
     } finally {
       writeUnlock(operationName);
@@ -7278,9 +7288,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /**
    * Disable an erasure coding policy.
    * @param ecPolicyName the name of the policy to be disabled
+   * @param logRetryCache whether to record RPC ids in editlog for retry cache
+   *                      rebuilding
    * @throws IOException
    */
-  void disableErasureCodingPolicy(String ecPolicyName) throws IOException {
+  void disableErasureCodingPolicy(String ecPolicyName,
+      final boolean logRetryCache) throws IOException {
     final String operationName = "disableErasureCodingPolicy";
     checkOperation(OperationCategory.WRITE);
     boolean success = false;
@@ -7290,7 +7303,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot disable erasure coding policy "
           + ecPolicyName);
-      FSDirErasureCodingOp.disableErasureCodePolicy(this, ecPolicyName);
+      FSDirErasureCodingOp.disableErasureCodingPolicy(this, ecPolicyName,
+          logRetryCache);
       success = true;
     } finally {
       writeUnlock(operationName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d996d3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 1ef3f55..09c9d83 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -2337,7 +2337,21 @@ public class NameNodeRpcServer implements NamenodeProtocols {
       ErasureCodingPolicy[] policies) throws IOException {
     checkNNStartup();
     namesystem.checkSuperuserPrivilege();
-    return namesystem.addErasureCodingPolicies(policies);
+    final CacheEntryWithPayload cacheEntry =
+        RetryCache.waitForCompletion(retryCache, null);
+    if (cacheEntry != null && cacheEntry.isSuccess()) {
+      return (AddECPolicyResponse[]) cacheEntry.getPayload();
+    }
+    boolean success = false;
+    AddECPolicyResponse[] responses = new AddECPolicyResponse[0];
+    try {
+      responses =
+          namesystem.addErasureCodingPolicies(policies, cacheEntry != null);
+      success = true;
+    } finally {
+      RetryCache.setState(cacheEntry, success, responses);
+    }
+    return responses;
   }
 
   @Override
@@ -2345,7 +2359,17 @@ public class NameNodeRpcServer implements NamenodeProtocols {
       throws IOException {
     checkNNStartup();
     namesystem.checkSuperuserPrivilege();
-    namesystem.removeErasureCodingPolicy(ecPolicyName);
+    final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
+    if (cacheEntry != null && cacheEntry.isSuccess()) {
+      return;
+    }
+    boolean success = false;
+    try {
+      namesystem.removeErasureCodingPolicy(ecPolicyName, cacheEntry != null);
+      success = true;
+    } finally {
+      RetryCache.setState(cacheEntry, success);
+    }
   }
 
   @Override // ClientProtocol
@@ -2353,7 +2377,17 @@ public class NameNodeRpcServer implements NamenodeProtocols {
       throws IOException {
     checkNNStartup();
     namesystem.checkSuperuserPrivilege();
-    namesystem.enableErasureCodingPolicy(ecPolicyName);
+    final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
+    if (cacheEntry != null && cacheEntry.isSuccess()) {
+      return;
+    }
+    boolean success = false;
+    try {
+      namesystem.enableErasureCodingPolicy(ecPolicyName, cacheEntry != null);
+      success = true;
+    } finally {
+      RetryCache.setState(cacheEntry, success);
+    }
   }
 
   @Override // ClientProtocol
@@ -2361,7 +2395,17 @@ public class NameNodeRpcServer implements NamenodeProtocols {
       throws IOException {
     checkNNStartup();
     namesystem.checkSuperuserPrivilege();
-    namesystem.disableErasureCodingPolicy(ecPolicyName);
+    final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
+    if (cacheEntry != null && cacheEntry.isSuccess()) {
+      return;
+    }
+    boolean success = false;
+    try {
+      namesystem.disableErasureCodingPolicy(ecPolicyName, cacheEntry != null);
+      success = true;
+    } finally {
+      RetryCache.setState(cacheEntry, success);
+    }
   }
 
   @Override // ReconfigurationProtocol

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d996d3/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsEditsViewer.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsEditsViewer.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsEditsViewer.md
index 4ab07ce..ce798b7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsEditsViewer.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsEditsViewer.md
@@ -108,6 +108,10 @@ The output result of this processor should be like the following output:
        ...some output omitted...
        OP_APPEND                      ( 47): 1
        OP_SET_QUOTA_BY_STORAGETYPE    ( 48): 1
+       OP_ADD_ERASURE_CODING_POLICY   ( 49): 0
+       OP_ENABLE_ERASURE_CODING_POLICY  ( 50): 1
+       OP_DISABLE_ERASURE_CODING_POLICY ( 51): 0
+       OP_REMOVE_ERASURE_CODING_POLICY  ( 52): 0
        OP_INVALID                     ( -1): 0
 
 The output is formatted as a colon separated two column table: OpCode and OpCodeCount. Each OpCode corresponding to the specific operation(s) in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d996d3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index de1a8ad..c6fe1a2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -108,6 +108,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
+import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
@@ -163,6 +164,8 @@ import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.hdfs.tools.JMXGet;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
@@ -1464,6 +1467,33 @@ public class DFSTestUtil {
         new byte[]{0x37, 0x38, 0x39});
     // OP_REMOVE_XATTR
     filesystem.removeXAttr(pathConcatTarget, "user.a2");
+
+    // OP_ADD_ERASURE_CODING_POLICY
+    ErasureCodingPolicy newPolicy1 =
+        new ErasureCodingPolicy(ErasureCodeConstants.RS_3_2_SCHEMA, 8 * 1024);
+    ErasureCodingPolicy[] policyArray = new ErasureCodingPolicy[] {newPolicy1};
+    AddECPolicyResponse[] responses =
+        filesystem.addErasureCodingPolicies(policyArray);
+    newPolicy1 = responses[0].getPolicy();
+
+    // OP_ADD_ERASURE_CODING_POLICY - policy with extra options
+    Map<String, String> extraOptions = new HashMap<String, String>();
+    extraOptions.put("dummyKey", "dummyValue");
+    ECSchema schema =
+        new ECSchema(ErasureCodeConstants.RS_CODEC_NAME, 6, 10, extraOptions);
+    ErasureCodingPolicy newPolicy2 = new ErasureCodingPolicy(schema, 4 * 1024);
+    policyArray = new ErasureCodingPolicy[] {newPolicy2};
+    responses = filesystem.addErasureCodingPolicies(policyArray);
+    newPolicy2 = responses[0].getPolicy();
+    // OP_ENABLE_ERASURE_CODING_POLICY
+    filesystem.enableErasureCodingPolicy(newPolicy1.getName());
+    filesystem.enableErasureCodingPolicy(newPolicy2.getName());
+    // OP_DISABLE_ERASURE_CODING_POLICY
+    filesystem.disableErasureCodingPolicy(newPolicy1.getName());
+    filesystem.disableErasureCodingPolicy(newPolicy2.getName());
+    // OP_REMOVE_ERASURE_CODING_POLICY
+    filesystem.removeErasureCodingPolicy(newPolicy1.getName());
+    filesystem.removeErasureCodingPolicy(newPolicy2.getName());
   }
 
   public static void abortStream(DFSOutputStream out) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d996d3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
index 97f34f2..1b462a9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInotifyEventInputStream.java
@@ -72,7 +72,7 @@ public class TestDFSInotifyEventInputStream {
    */
   @Test
   public void testOpcodeCount() {
-    Assert.assertEquals(50, FSEditLogOpCodes.values().length);
+    Assert.assertEquals(54, FSEditLogOpCodes.values().length);
   }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d996d3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
index ae62dab..ec80bff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
@@ -47,8 +47,10 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.StripedFileTestUtil;
+import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
@@ -57,6 +59,7 @@ import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.log4j.Level;
@@ -712,4 +715,84 @@ public class TestFSEditLogLoader {
       }
     }
   }
+
+  @Test
+  public void testErasureCodingPolicyOperations() throws IOException {
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    final int blockSize = 16 * 1024;
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+
+      // 1. add new policy
+      ECSchema schema = new ECSchema("rs", 5, 3);
+      int cellSize = 2 * 1024;
+      ErasureCodingPolicy newPolicy =
+          new ErasureCodingPolicy(schema, cellSize, (byte) 0);
+      ErasureCodingPolicy[] policyArray = new ErasureCodingPolicy[]{newPolicy};
+      AddECPolicyResponse[] responses =
+          fs.addErasureCodingPolicies(policyArray);
+      assertEquals(1, responses.length);
+      assertTrue(responses[0].isSucceed());
+      newPolicy = responses[0].getPolicy();
+
+      // Restart NameNode without saving namespace
+      cluster.restartNameNodes();
+      cluster.waitActive();
+
+      // check if new policy is reapplied through edit log
+      ErasureCodingPolicy ecPolicy =
+          ErasureCodingPolicyManager.getInstance().getByID(newPolicy.getId());
+      assertEquals(ErasureCodingPolicyState.DISABLED, ecPolicy.getState());
+
+      // 2. enable policy
+      fs.enableErasureCodingPolicy(newPolicy.getName());
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      ecPolicy =
+          ErasureCodingPolicyManager.getInstance().getByID(newPolicy.getId());
+      assertEquals(ErasureCodingPolicyState.ENABLED, ecPolicy.getState());
+
+      // create a new file, use the policy
+      final Path dirPath = new Path("/striped");
+      final Path filePath = new Path(dirPath, "file");
+      final int fileLength = blockSize * newPolicy.getNumDataUnits();
+      fs.mkdirs(dirPath);
+      fs.setErasureCodingPolicy(dirPath, newPolicy.getName());
+      final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength);
+      DFSTestUtil.writeFile(fs, filePath, bytes);
+
+      // 3. disable policy
+      fs.disableErasureCodingPolicy(newPolicy.getName());
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      ecPolicy =
+          ErasureCodingPolicyManager.getInstance().getByID(newPolicy.getId());
+      assertEquals(ErasureCodingPolicyState.DISABLED, ecPolicy.getState());
+      // read file
+      DFSTestUtil.readFileAsBytes(fs, filePath);
+
+      // 4. remove policy
+      fs.removeErasureCodingPolicy(newPolicy.getName());
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      ecPolicy =
+          ErasureCodingPolicyManager.getInstance().getByID(newPolicy.getId());
+      assertEquals(ErasureCodingPolicyState.REMOVED, ecPolicy.getState());
+      // read file
+      DFSTestUtil.readFileAsBytes(fs, filePath);
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d996d3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
index bbad73c..9e1fa79 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
@@ -182,7 +182,7 @@ public class TestOfflineEditsViewer {
         hasAllOpCodes(editsStored));
     assertTrue("Reference XML edits and parsed to XML should be same",
         FileUtils.contentEqualsIgnoreEOL(new File(editsStoredXml),
-          new File(editsStoredParsedXml), "UTF-8"));
+            new File(editsStoredParsedXml), "UTF-8"));
     assertTrue(
         "Reference edits and reparsed (bin to XML to bin) should be same",
         filesEqualIgnoreTrailingZeros(editsStored, editsStoredReparsed));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d996d3/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
index e271cb5..8029575 100644
Binary files a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08d996d3/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
index f901145..0a1c25e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
@@ -1179,23 +1179,107 @@
     </DATA>
   </RECORD>
   <RECORD>
-    <OPCODE>OP_ROLLING_UPGRADE_START</OPCODE>
+    <OPCODE>OP_ADD_ERASURE_CODING_POLICY</OPCODE>
     <DATA>
       <TXID>89</TXID>
+      <CODEC>rs</CODEC>
+      <DATAUNITS>3</DATAUNITS>
+      <PARITYUNITS>2</PARITYUNITS>
+      <CELLSIZE>8192</CELLSIZE>
+      <EXTRAOPTIONS>0</EXTRAOPTIONS>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>84</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_ADD_ERASURE_CODING_POLICY</OPCODE>
+    <DATA>
+      <TXID>90</TXID>
+      <CODEC>rs</CODEC>
+      <DATAUNITS>6</DATAUNITS>
+      <PARITYUNITS>10</PARITYUNITS>
+      <CELLSIZE>4096</CELLSIZE>
+      <EXTRAOPTIONS>1</EXTRAOPTIONS>
+      <EXTRAOPTION>
+        <KEY>dummyKey</KEY>
+        <VALUE>dummyValue</VALUE>
+      </EXTRAOPTION>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>85</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_ENABLE_ERASURE_CODING_POLICY</OPCODE>
+    <DATA>
+      <TXID>91</TXID>
+      <POLICYNAME>RS-3-2-8k</POLICYNAME>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>86</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_ENABLE_ERASURE_CODING_POLICY</OPCODE>
+    <DATA>
+      <TXID>92</TXID>
+      <POLICYNAME>RS-6-10-4k</POLICYNAME>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>87</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_DISABLE_ERASURE_CODING_POLICY</OPCODE>
+    <DATA>
+      <TXID>93</TXID>
+      <POLICYNAME>RS-3-2-8k</POLICYNAME>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>88</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_DISABLE_ERASURE_CODING_POLICY</OPCODE>
+    <DATA>
+      <TXID>94</TXID>
+      <POLICYNAME>RS-6-10-4k</POLICYNAME>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>89</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_REMOVE_ERASURE_CODING_POLICY</OPCODE>
+    <DATA>
+      <TXID>95</TXID>
+      <POLICYNAME>RS-3-2-8k</POLICYNAME>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>90</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_REMOVE_ERASURE_CODING_POLICY</OPCODE>
+    <DATA>
+      <TXID>96</TXID>
+      <POLICYNAME>RS-6-10-4k</POLICYNAME>
+      <RPC_CLIENTID>7334ec24-dd6b-4efd-807d-ed0d18625534</RPC_CLIENTID>
+      <RPC_CALLID>91</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_ROLLING_UPGRADE_START</OPCODE>
+    <DATA>
+      <TXID>97</TXID>
       <STARTTIME>1422406383706</STARTTIME>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ROLLING_UPGRADE_FINALIZE</OPCODE>
     <DATA>
-      <TXID>90</TXID>
+      <TXID>98</TXID>
       <FINALIZETIME>1422406383706</FINALIZETIME>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_END_LOG_SEGMENT</OPCODE>
     <DATA>
-      <TXID>91</TXID>
+      <TXID>99</TXID>
     </DATA>
   </RECORD>
 </EDITS>


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


[10/50] [abbrv] hadoop git commit: HADOOP-14738 Remove S3N and obsolete bits of S3A; rework docs. Contributed by Steve Loughran.

Posted by ae...@apache.org.
HADOOP-14738 Remove S3N and obsolete bits of S3A; rework docs.  Contributed by Steve Loughran.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/49467165
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/49467165
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/49467165

Branch: refs/heads/HDFS-7240
Commit: 49467165a57fb77932d1d526796624b88ebacd91
Parents: 7ee02d1
Author: Aaron Fabbri <fa...@apache.org>
Authored: Thu Sep 14 09:58:17 2017 -0700
Committer: Aaron Fabbri <fa...@apache.org>
Committed: Thu Sep 14 14:10:48 2017 -0700

----------------------------------------------------------------------
 .../hadoop-client-minicluster/pom.xml           |   23 -
 .../src/main/conf/log4j.properties              |    2 -
 .../src/main/resources/core-default.xml         |  124 +-
 .../src/site/markdown/filesystem/filesystem.md  |    6 +-
 .../site/markdown/filesystem/introduction.md    |    7 +-
 .../src/site/markdown/filesystem/testing.md     |   10 +-
 .../conf/TestCommonConfigurationFields.java     |    6 +-
 .../apache/hadoop/conf/TestConfigRedactor.java  |    1 -
 .../hadoop/fs/FileSystemContractBaseTest.java   |   41 +-
 .../src/test/resources/core-site.xml            |    6 -
 .../src/test/resources/jets3t.properties        |   16 -
 hadoop-project/pom.xml                          |    5 -
 hadoop-project/src/site/markdown/index.md.vm    |   10 +
 .../hadoop-aws/dev-support/findbugs-exclude.xml |    4 -
 hadoop-tools/hadoop-aws/pom.xml                 |    7 -
 .../org/apache/hadoop/fs/s3a/Constants.java     |    3 +
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |   71 +-
 .../apache/hadoop/fs/s3a/S3AOutputStream.java   |  143 --
 .../apache/hadoop/fs/s3native/FileMetadata.java |   59 -
 .../s3native/Jets3tNativeFileSystemStore.java   |  481 -----
 .../fs/s3native/NativeFileSystemStore.java      |   67 -
 .../hadoop/fs/s3native/NativeS3FileSystem.java  |  799 +-------
 .../hadoop/fs/s3native/PartialListing.java      |   64 -
 .../hadoop/fs/s3native/S3Credentials.java       |  100 -
 .../apache/hadoop/fs/s3native/S3Exception.java  |   39 -
 .../s3native/S3NativeFileSystemConfigKeys.java  |   66 -
 .../org/apache/hadoop/fs/s3native/package.html  |    5 +-
 .../markdown/tools/hadoop-aws/encryption.md     |  427 +++++
 .../src/site/markdown/tools/hadoop-aws/index.md | 1753 ++++++------------
 .../site/markdown/tools/hadoop-aws/s3guard.md   |   19 +-
 .../src/site/markdown/tools/hadoop-aws/s3n.md   |   52 +
 .../site/markdown/tools/hadoop-aws/testing.md   |   91 +-
 .../tools/hadoop-aws/troubleshooting_s3a.md     |  701 ++++++-
 .../fs/contract/s3a/ITestS3AContractDistCp.java |    1 -
 .../fs/contract/s3n/ITestS3NContractCreate.java |   41 -
 .../fs/contract/s3n/ITestS3NContractDelete.java |   34 -
 .../fs/contract/s3n/ITestS3NContractMkdir.java  |   34 -
 .../fs/contract/s3n/ITestS3NContractOpen.java   |   34 -
 .../fs/contract/s3n/ITestS3NContractRename.java |   35 -
 .../contract/s3n/ITestS3NContractRootDir.java   |   35 -
 .../fs/contract/s3n/ITestS3NContractSeek.java   |   34 -
 .../fs/contract/s3n/NativeS3Contract.java       |   50 -
 .../hadoop/fs/s3a/ITestS3ABlockOutputArray.java |    1 -
 ...ITestS3AEncryptionSSECBlockOutputStream.java |    1 -
 ...onSSEKMSUserDefinedKeyBlockOutputStream.java |    4 +-
 ...TestS3AEncryptionSSES3BlockOutputStream.java |    1 -
 .../fs/s3a/scale/AbstractSTestS3AHugeFiles.java |    1 -
 .../scale/ITestS3AHugeFilesClassicOutput.java   |   41 -
 ...ITestInMemoryNativeS3FileSystemContract.java |   33 -
 .../ITestJets3tNativeFileSystemStore.java       |  133 --
 .../ITestJets3tNativeS3FileSystemContract.java  |   33 -
 .../s3native/InMemoryNativeFileSystemStore.java |  213 ---
 .../NativeS3FileSystemContractBaseTest.java     |  266 ---
 .../fs/s3native/S3NInMemoryFileSystem.java      |   32 -
 .../hadoop/fs/s3native/TestS3Credentials.java   |  129 --
 .../fs/s3native/TestS3NInMemoryFileSystem.java  |   69 -
 .../src/test/resources/contract/s3n.xml         |  110 --
 .../src/test/resources/log4j.properties         |    3 +
 58 files changed, 1853 insertions(+), 4723 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-client-modules/hadoop-client-minicluster/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-minicluster/pom.xml b/hadoop-client-modules/hadoop-client-minicluster/pom.xml
index 0c87866..00f2d25 100644
--- a/hadoop-client-modules/hadoop-client-minicluster/pom.xml
+++ b/hadoop-client-modules/hadoop-client-minicluster/pom.xml
@@ -355,29 +355,6 @@
       <optional>true</optional>
     </dependency>
     <dependency>
-      <groupId>net.java.dev.jets3t</groupId>
-      <artifactId>jets3t</artifactId>
-      <optional>true</optional>
-      <exclusions>
-        <exclusion>
-          <groupId>commons-codec</groupId>
-          <artifactId>commons-codec</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>commons-logging</groupId>
-          <artifactId>commons-logging</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.httpcomponents</groupId>
-          <artifactId>httpclient</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.httpcomponents</groupId>
-          <artifactId>httpcore</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
       <groupId>com.jcraft</groupId>
       <artifactId>jsch</artifactId>
       <optional>true</optional>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties b/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
index 3752ad1..bc1fa6c 100644
--- a/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
+++ b/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
@@ -182,8 +182,6 @@ log4j.appender.DNMETRICSRFA.MaxFileSize=64MB
 #log4j.logger.org.apache.hadoop.mapred.TaskTracker=DEBUG
 #log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=DEBUG
 
-# Jets3t library
-log4j.logger.org.jets3t.service.impl.rest.httpclient.RestS3Service=ERROR
 
 # AWS SDK & S3A FileSystem
 log4j.logger.com.amazonaws=ERROR

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 6cce647..a11e7c3 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -872,30 +872,6 @@
 </property>
 
 <property>
-  <name>fs.s3n.buffer.dir</name>
-  <value>${hadoop.tmp.dir}/s3n</value>
-  <description>Determines where on the local filesystem the s3n:// filesystem
-  should store files before sending them to S3
-  (or after retrieving them from S3).
-  </description>
-</property>
-
-<property>
-  <name>fs.s3n.maxRetries</name>
-  <value>4</value>
-  <description>The maximum number of retries for reading or writing files to S3,
-  before we signal failure to the application.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3n.sleepTimeSeconds</name>
-  <value>10</value>
-  <description>The number of seconds to sleep between each S3 retry.
-  </description>
-</property>
-
-<property>
   <name>fs.swift.impl</name>
   <value>org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem</value>
   <description>The implementation class of the OpenStack Swift Filesystem</description>
@@ -912,56 +888,6 @@
 </property>
 
 <property>
-  <name>fs.s3n.awsAccessKeyId</name>
-  <description>AWS access key ID used by S3 native file system.</description>
-</property>
-
-<property>
-  <name>fs.s3n.awsSecretAccessKey</name>
-  <description>AWS secret key used by S3 native file system.</description>
-</property>
-
-<property>
-  <name>fs.s3n.block.size</name>
-  <value>67108864</value>
-  <description>Block size to use when reading files using the native S3
-  filesystem (s3n: URIs).</description>
-</property>
-
-<property>
-  <name>fs.s3n.multipart.uploads.enabled</name>
-  <value>false</value>
-  <description>Setting this property to true enables multiple uploads to
-  native S3 filesystem. When uploading a file, it is split into blocks
-  if the size is larger than fs.s3n.multipart.uploads.block.size.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3n.multipart.uploads.block.size</name>
-  <value>67108864</value>
-  <description>The block size for multipart uploads to native S3 filesystem.
-  Default size is 64MB.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3n.multipart.copy.block.size</name>
-  <value>5368709120</value>
-  <description>The block size for multipart copy in native S3 filesystem.
-  Default size is 5GB.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3n.server-side-encryption-algorithm</name>
-  <value></value>
-  <description>Specify a server-side encryption algorithm for S3.
-  Unset by default, and the only other currently allowable value is AES256.
-  </description>
-</property>
-
-<property>
   <name>fs.s3a.access.key</name>
   <description>AWS access key ID used by S3A file system. Omit for IAM role-based or provider-based authentication.</description>
 </property>
@@ -1235,21 +1161,11 @@
 </property>
 
 <property>
-  <name>fs.s3a.fast.upload</name>
-  <value>false</value>
-  <description>
-    Use the incremental block-based fast upload mechanism with
-    the buffering mechanism set in fs.s3a.fast.upload.buffer.
-  </description>
-</property>
-
-<property>
   <name>fs.s3a.fast.upload.buffer</name>
   <value>disk</value>
   <description>
-    The buffering mechanism to use when using S3A fast upload
-    (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
-    This configuration option has no effect if fs.s3a.fast.upload is false.
+    The buffering mechanism to for data being written.
+    Values: disk, array, bytebuffer.
 
     "disk" will use the directories listed in fs.s3a.buffer.dir as
     the location(s) to save data prior to being uploaded.
@@ -1803,42 +1719,6 @@
   <description>Replication factor</description>
 </property>
 
-<!-- s3native File System -->
-
-<property>
-  <name>s3native.stream-buffer-size</name>
-  <value>4096</value>
-  <description>The size of buffer to stream files.
-  The size of this buffer should probably be a multiple of hardware
-  page size (4096 on Intel x86), and it determines how much data is
-  buffered during read and write operations.</description>
-</property>
-
-<property>
-  <name>s3native.bytes-per-checksum</name>
-  <value>512</value>
-  <description>The number of bytes per checksum.  Must not be larger than
-  s3native.stream-buffer-size</description>
-</property>
-
-<property>
-  <name>s3native.client-write-packet-size</name>
-  <value>65536</value>
-  <description>Packet size for clients to write</description>
-</property>
-
-<property>
-  <name>s3native.blocksize</name>
-  <value>67108864</value>
-  <description>Block size</description>
-</property>
-
-<property>
-  <name>s3native.replication</name>
-  <value>3</value>
-  <description>Replication factor</description>
-</property>
-
 <!-- FTP file system -->
 <property>
   <name>ftp.stream-buffer-size</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
index 1e522c7..e67cbe3 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
@@ -605,7 +605,7 @@ The result is `FSDataOutputStream`, which through its operations may generate ne
  clients creating files with `overwrite==true` to fail if the file is created
  by another client between the two tests.
 
-* S3N, S3A, Swift and potentially other Object Stores do not currently change the FS state
+* S3A, Swift and potentially other Object Stores do not currently change the FS state
 until the output stream `close()` operation is completed.
 This MAY be a bug, as it allows >1 client to create a file with `overwrite==false`,
  and potentially confuse file/directory logic
@@ -961,7 +961,7 @@ The outcome is no change to FileSystem state, with a return value of false.
 
     FS' = FS; result = False
 
-*Local Filesystem, S3N*
+*Local Filesystem*
 
 The outcome is as a normal rename, with the additional (implicit) feature
 that the parent directories of the destination also exist.
@@ -1262,4 +1262,4 @@ It currently supports to query:
  * `StreamCapabilties.HFLUSH` ("*hflush*"): the capability to flush out the data
  in client's buffer.
  * `StreamCapabilities.HSYNC` ("*hsync*"): capability to flush out the data in
- client's buffer and the disk device.
\ No newline at end of file
+ client's buffer and the disk device.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/introduction.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/introduction.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/introduction.md
index 12a7967..37191a5 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/introduction.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/introduction.md
@@ -29,11 +29,10 @@ return codes of Unix filesystem actions as a reference. Even so, there
 are places where HDFS diverges from the expected behaviour of a POSIX
 filesystem.
 
-The behaviour of other Hadoop filesystems are not as rigorously tested.
-The bundled S3N and S3A FileSystem clients make Amazon's S3 Object Store ("blobstore")
+The bundled S3A FileSystem clients make Amazon's S3 Object Store ("blobstore")
 accessible through the FileSystem API. The Swift FileSystem driver provides similar
-functionality for the OpenStack Swift blobstore. The Azure object storage
-FileSystem talks to Microsoft's Azure equivalent. All of these
+functionality for the OpenStack Swift blobstore. The Azure WASB and ADL object
+storage FileSystems talks to Microsoft's Azure storage. All of these
 bind to object stores, which do have different behaviors, especially regarding
 consistency guarantees, and atomicity of operations.
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/testing.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/testing.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/testing.md
index 6823e0c..4c6fa3f 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/testing.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/testing.md
@@ -195,21 +195,21 @@ equivalent. Furthermore, the build MUST be configured to never bundle this file
 In addition, `src/test/resources/auth-keys.xml` will need to be created.  It can be a copy of `contract-test-options.xml`.
 The `AbstractFSContract` class automatically loads this resource file if present; specific keys for specific test cases can be added.
 
-As an example, here are what S3N test keys look like:
+As an example, here are what S3A test keys look like:
 
     <configuration>
       <property>
-        <name>fs.contract.test.fs.s3n</name>
-        <value>s3n://tests3contract</value>
+        <name>fs.contract.test.fs.s3a</name>
+        <value>s3a://tests3contract</value>
       </property>
 
       <property>
-        <name>fs.s3n.awsAccessKeyId</name>
+        <name>fs.s3a.access.key</name>
         <value>DONOTPCOMMITTHISKEYTOSCM</value>
       </property>
 
       <property>
-        <name>fs.s3n.awsSecretAccessKey</name>
+        <name>fs.s3a.secret.key</name>
         <value>DONOTEVERSHARETHISSECRETKEY!</value>
       </property>
     </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
index 3324886..864c10c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
@@ -94,14 +94,10 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase {
     xmlPropsToSkipCompare.add("hadoop.tmp.dir");
     xmlPropsToSkipCompare.add("nfs3.mountd.port");
     xmlPropsToSkipCompare.add("nfs3.server.port");
-    xmlPropsToSkipCompare.add("test.fs.s3n.name");
     xmlPropsToSkipCompare.add("fs.viewfs.rename.strategy");
 
-    // S3N/S3A properties are in a different subtree.
-    // - org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys
+    // S3A properties are in a different subtree.
     xmlPrefixToSkipCompare.add("fs.s3a.");
-    xmlPrefixToSkipCompare.add("fs.s3n.");
-    xmlPrefixToSkipCompare.add("s3native.");
 
     // WASB properties are in a different subtree.
     // - org.apache.hadoop.fs.azure.NativeAzureFileSystem

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigRedactor.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigRedactor.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigRedactor.java
index 4790f7c..3133942 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigRedactor.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigRedactor.java
@@ -54,7 +54,6 @@ public class TestConfigRedactor {
         "fs.s3a.bucket.BUCKET.secret.key",
         "fs.s3a.server-side-encryption.key",
         "fs.s3a.bucket.engineering.server-side-encryption.key",
-        "fs.s3n.awsSecretKey",
         "fs.azure.account.key.abcdefg.blob.core.windows.net",
         "fs.adl.oauth2.refresh.token",
         "fs.adl.oauth2.credential",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
index 9d8cd64..b49dd53 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
@@ -246,39 +246,18 @@ public abstract class FileSystemContractBaseTest {
 
   @Test
   public void testMkdirsWithUmask() throws Exception {
-    if (!isS3(fs)) {
-      Configuration conf = fs.getConf();
-      String oldUmask = conf.get(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY);
-      try {
-        conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, TEST_UMASK);
-        final Path dir = path("newDir");
-        assertTrue(fs.mkdirs(dir, new FsPermission((short) 0777)));
-        FileStatus status = fs.getFileStatus(dir);
-        assertTrue(status.isDirectory());
-        assertEquals((short) 0715, status.getPermission().toShort());
-      } finally {
-        conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, oldUmask);
-      }
-    }
-  }
-
-  /**
-   * Skip permission tests for S3FileSystem until HDFS-1333 is fixed.
-   * Classes that do not implement {@link FileSystem#getScheme()} method
-   * (e.g {@link RawLocalFileSystem}) will throw an
-   * {@link UnsupportedOperationException}.
-   * @param fileSystem FileSystem object to determine if it is S3 or not
-   * @return true if S3 false in any other case
-   */
-  private boolean isS3(FileSystem fileSystem) {
+    Configuration conf = fs.getConf();
+    String oldUmask = conf.get(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY);
     try {
-      if (fileSystem.getScheme().equals("s3n")) {
-        return true;
-      }
-    } catch (UnsupportedOperationException e) {
-      LOG.warn("Unable to determine the schema of filesystem.");
+      conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, TEST_UMASK);
+      final Path dir = path("newDir");
+      assertTrue(fs.mkdirs(dir, new FsPermission((short) 0777)));
+      FileStatus status = fs.getFileStatus(dir);
+      assertTrue(status.isDirectory());
+      assertEquals((short) 0715, status.getPermission().toShort());
+    } finally {
+      conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, oldUmask);
     }
-    return false;
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-common-project/hadoop-common/src/test/resources/core-site.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/resources/core-site.xml b/hadoop-common-project/hadoop-common/src/test/resources/core-site.xml
index d85472c..d9144eb 100644
--- a/hadoop-common-project/hadoop-common/src/test/resources/core-site.xml
+++ b/hadoop-common-project/hadoop-common/src/test/resources/core-site.xml
@@ -45,12 +45,6 @@
    This is required by FTPFileSystem</description>
 </property>
 
-<property>
-  <name>test.fs.s3n.name</name>
-  <value>s3n:///</value>
-  <description>The name of the s3n file system for testing.</description>
-</property>
-
 <!-- Turn security off for tests by default -->
 <property>
   <name>hadoop.security.authentication</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-common-project/hadoop-common/src/test/resources/jets3t.properties
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/resources/jets3t.properties b/hadoop-common-project/hadoop-common/src/test/resources/jets3t.properties
deleted file mode 100644
index 09cc463..0000000
--- a/hadoop-common-project/hadoop-common/src/test/resources/jets3t.properties
+++ /dev/null
@@ -1,16 +0,0 @@
-#   Licensed under the Apache License, Version 2.0 (the "License");
-#   you may not use this file except in compliance with the License.
-#   You may obtain a copy of the License at
-#
-#       http://www.apache.org/licenses/LICENSE-2.0
-#
-#   Unless required by applicable law or agreed to in writing, software
-#   distributed under the License is distributed on an "AS IS" BASIS,
-#   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#   See the License for the specific language governing permissions and
-#   limitations under the License.
-
-# Speed up the s3native jets3t test
-
-s3service.max-thread-count=10
-threaded-service.max-thread-count=10

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 8bb3834..220bb62 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -812,11 +812,6 @@
         </exclusions>
       </dependency>
       <dependency>
-        <groupId>net.java.dev.jets3t</groupId>
-        <artifactId>jets3t</artifactId>
-        <version>0.9.0</version>
-      </dependency>
-      <dependency>
         <groupId>com.amazonaws</groupId>
         <artifactId>aws-java-sdk-bundle</artifactId>
         <version>${aws-java-sdk.version}</version>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-project/src/site/markdown/index.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-project/src/site/markdown/index.md.vm b/hadoop-project/src/site/markdown/index.md.vm
index bb7bda2..d9443d6 100644
--- a/hadoop-project/src/site/markdown/index.md.vm
+++ b/hadoop-project/src/site/markdown/index.md.vm
@@ -204,6 +204,16 @@ in both the task configuration and as a Java option.
 Existing configs that already specify both are not affected by this change.
 See the full release notes of MAPREDUCE-5785 for more details.
 
+S3Guard: Consistency and Metadata Caching for the S3A filesystem client
+---------------------
+
+[HADOOP-13345](https://issues.apache.org/jira/browse/HADOOP-13345) adds an
+optional feature to the S3A client of Amazon S3 storage: the ability to use
+a DynamoDB table as a fast and consistent store of file and directory
+metadata.
+
+See [S3Guard](./hadoop-aws/tools/hadoop-aws/s3guard.html) for more details.
+
 Getting Started
 ===============
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
index 82ec16e..2615566 100644
--- a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
+++ b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
@@ -22,10 +22,6 @@
     <Method name="checkPath" />
     <Bug pattern="ES_COMPARING_STRINGS_WITH_EQ" />
   </Match>
-  <!-- S3n warnings about malicious code aren't that relevant given its limited future. -->
-  <Match>
-    <Class name="org.apache.hadoop.fs.s3.INode" />
-  </Match>
   <!-- Redundant null check makes code clearer, future-proof here. -->
   <Match>
     <Class name="org.apache.hadoop.fs.s3a.S3AFileSystem" />

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml
index 6bab9a7..47788cd 100644
--- a/hadoop-tools/hadoop-aws/pom.xml
+++ b/hadoop-tools/hadoop-aws/pom.xml
@@ -193,7 +193,6 @@
                   <excludes>
                     <exclude>**/ITestJets3tNativeS3FileSystemContract.java</exclude>
                     <exclude>**/ITestS3AContractRootDir.java</exclude>
-                    <exclude>**/ITestS3NContractRootDir.java</exclude>
                     <exclude>**/ITestS3AFileContextStatistics.java</exclude>
                     <exclude>**/ITestS3AEncryptionSSEC*.java</exclude>
                     <exclude>**/ITestS3AHuge*.java</exclude>
@@ -226,7 +225,6 @@
                   <includes>
                     <include>**/ITestJets3tNativeS3FileSystemContract.java</include>
                     <include>**/ITestS3AContractRootDir.java</include>
-                    <include>**/ITestS3NContractRootDir.java</include>
                     <include>**/ITestS3AFileContextStatistics.java</include>
                     <include>**/ITestS3AHuge*.java</include>
                     <include>**/ITestS3AEncryptionSSEC*.java</include>
@@ -429,11 +427,6 @@
       <type>test-jar</type>
     </dependency>
     <dependency>
-      <groupId>net.java.dev.jets3t</groupId>
-      <artifactId>jets3t</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
       <groupId>com.amazonaws</groupId>
       <artifactId>aws-java-sdk-bundle</artifactId>
       <scope>compile</scope>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
index 4e2af3a..d278bdf 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
@@ -136,7 +136,10 @@ public final class Constants {
   public static final String BUFFER_DIR = "fs.s3a.buffer.dir";
 
   // switch to the fast block-by-block upload mechanism
+  // this is the only supported upload mechanism
+  @Deprecated
   public static final String FAST_UPLOAD = "fs.s3a.fast.upload";
+  @Deprecated
   public static final boolean DEFAULT_FAST_UPLOAD = false;
 
   //initial size of memory buffer for a fast upload

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index e76ef0b..f4709a7 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -163,7 +163,6 @@ public class S3AFileSystem extends FileSystem {
 
   // The maximum number of entries that can be deleted in any call to s3
   private static final int MAX_ENTRIES_TO_DELETE = 1000;
-  private boolean blockUploadEnabled;
   private String blockOutputBuffer;
   private S3ADataBlocks.BlockFactory blockFactory;
   private int blockOutputActiveBlocks;
@@ -281,21 +280,20 @@ public class S3AFileSystem extends FileSystem {
       inputPolicy = S3AInputPolicy.getPolicy(
           conf.getTrimmed(INPUT_FADVISE, INPUT_FADV_NORMAL));
 
-      blockUploadEnabled = conf.getBoolean(FAST_UPLOAD, DEFAULT_FAST_UPLOAD);
-
-      if (blockUploadEnabled) {
-        blockOutputBuffer = conf.getTrimmed(FAST_UPLOAD_BUFFER,
-            DEFAULT_FAST_UPLOAD_BUFFER);
-        partSize = ensureOutputParameterInRange(MULTIPART_SIZE, partSize);
-        blockFactory = S3ADataBlocks.createFactory(this, blockOutputBuffer);
-        blockOutputActiveBlocks = intOption(conf,
-            FAST_UPLOAD_ACTIVE_BLOCKS, DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS, 1);
-        LOG.debug("Using S3ABlockOutputStream with buffer = {}; block={};" +
-                " queue limit={}",
-            blockOutputBuffer, partSize, blockOutputActiveBlocks);
-      } else {
-        LOG.debug("Using S3AOutputStream");
+      boolean blockUploadEnabled = conf.getBoolean(FAST_UPLOAD, true);
+
+      if (!blockUploadEnabled) {
+        LOG.warn("The \"slow\" output stream is no longer supported");
       }
+      blockOutputBuffer = conf.getTrimmed(FAST_UPLOAD_BUFFER,
+          DEFAULT_FAST_UPLOAD_BUFFER);
+      partSize = ensureOutputParameterInRange(MULTIPART_SIZE, partSize);
+      blockFactory = S3ADataBlocks.createFactory(this, blockOutputBuffer);
+      blockOutputActiveBlocks = intOption(conf,
+          FAST_UPLOAD_ACTIVE_BLOCKS, DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS, 1);
+      LOG.debug("Using S3ABlockOutputStream with buffer = {}; block={};" +
+              " queue limit={}",
+          blockOutputBuffer, partSize, blockOutputActiveBlocks);
 
       metadataStore = S3Guard.getMetadataStore(this);
       allowAuthoritative = conf.getBoolean(METADATASTORE_AUTHORITATIVE,
@@ -644,33 +642,18 @@ public class S3AFileSystem extends FileSystem {
 
     }
     instrumentation.fileCreated();
-    FSDataOutputStream output;
-    if (blockUploadEnabled) {
-      output = new FSDataOutputStream(
-          new S3ABlockOutputStream(this,
-              key,
-              new SemaphoredDelegatingExecutor(boundedThreadPool,
-                  blockOutputActiveBlocks, true),
-              progress,
-              partSize,
-              blockFactory,
-              instrumentation.newOutputStreamStatistics(statistics),
-              new WriteOperationHelper(key)
-          ),
-          null);
-    } else {
-
-      // We pass null to FSDataOutputStream so it won't count writes that
-      // are being buffered to a file
-      output = new FSDataOutputStream(
-          new S3AOutputStream(getConf(),
-              this,
-              key,
-              progress
-          ),
-          null);
-    }
-    return output;
+    return new FSDataOutputStream(
+        new S3ABlockOutputStream(this,
+            key,
+            new SemaphoredDelegatingExecutor(boundedThreadPool,
+                blockOutputActiveBlocks, true),
+            progress,
+            partSize,
+            blockFactory,
+            instrumentation.newOutputStreamStatistics(statistics),
+            new WriteOperationHelper(key)
+        ),
+        null);
   }
 
   /**
@@ -2471,7 +2454,9 @@ public class S3AFileSystem extends FileSystem {
       sb.append(", cannedACL=").append(cannedACL.toString());
     }
     sb.append(", readAhead=").append(readAhead);
-    sb.append(", blockSize=").append(getDefaultBlockSize());
+    if (getConf() != null) {
+      sb.append(", blockSize=").append(getDefaultBlockSize());
+    }
     sb.append(", multiPartThreshold=").append(multiPartThreshold);
     if (serverSideEncryptionAlgorithm != null) {
       sb.append(", serverSideEncryptionAlgorithm='")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
deleted file mode 100644
index e723b75..0000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
+++ /dev/null
@@ -1,143 +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.s3a;
-
-import com.amazonaws.AmazonClientException;
-import com.amazonaws.services.s3.model.ObjectMetadata;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.util.Progressable;
-
-import org.slf4j.Logger;
-
-import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.io.OutputStream;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import static org.apache.hadoop.fs.s3a.S3AUtils.*;
-
-/**
- * Output stream to save data to S3.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class S3AOutputStream extends OutputStream {
-  private final OutputStream backupStream;
-  private final File backupFile;
-  private final AtomicBoolean closed = new AtomicBoolean(false);
-  private final String key;
-  private final Progressable progress;
-  private final S3AFileSystem fs;
-
-  public static final Logger LOG = S3AFileSystem.LOG;
-
-  public S3AOutputStream(Configuration conf,
-      S3AFileSystem fs,
-      String key,
-      Progressable progress)
-      throws IOException {
-    this.key = key;
-    this.progress = progress;
-    this.fs = fs;
-
-
-    backupFile = fs.createTmpFileForWrite("output-",
-        LocalDirAllocator.SIZE_UNKNOWN, conf);
-
-    LOG.debug("OutputStream for key '{}' writing to tempfile: {}",
-        key, backupFile);
-
-    this.backupStream = new BufferedOutputStream(
-        new FileOutputStream(backupFile));
-  }
-
-  /**
-   * Check for the filesystem being open.
-   * @throws IOException if the filesystem is closed.
-   */
-  void checkOpen() throws IOException {
-    if (closed.get()) {
-      throw new IOException("Output Stream closed");
-    }
-  }
-
-  @Override
-  public void flush() throws IOException {
-    checkOpen();
-    backupStream.flush();
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (closed.getAndSet(true)) {
-      return;
-    }
-
-    backupStream.close();
-    LOG.debug("OutputStream for key '{}' closed. Now beginning upload", key);
-
-    try {
-      final ObjectMetadata om = fs.newObjectMetadata(backupFile.length());
-      UploadInfo info = fs.putObject(
-          fs.newPutObjectRequest(
-              key,
-              om,
-              backupFile));
-      ProgressableProgressListener listener =
-          new ProgressableProgressListener(fs, key, info.getUpload(), progress);
-      info.getUpload().addProgressListener(listener);
-
-      info.getUpload().waitForUploadResult();
-      listener.uploadCompleted();
-      // This will delete unnecessary fake parent directories, update any
-      // MetadataStore
-      fs.finishedWrite(key, info.getLength());
-    } catch (InterruptedException e) {
-      throw (InterruptedIOException) new InterruptedIOException(e.toString())
-          .initCause(e);
-    } catch (AmazonClientException e) {
-      throw translateException("saving output", key , e);
-    } finally {
-      if (!backupFile.delete()) {
-        LOG.warn("Could not delete temporary s3a file: {}", backupFile);
-      }
-      super.close();
-    }
-    LOG.debug("OutputStream for key '{}' upload complete", key);
-  }
-
-  @Override
-  public void write(int b) throws IOException {
-    checkOpen();
-    backupStream.write(b);
-  }
-
-  @Override
-  public void write(byte[] b, int off, int len) throws IOException {
-    checkOpen();
-    backupStream.write(b, off, len);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/FileMetadata.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/FileMetadata.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/FileMetadata.java
deleted file mode 100644
index 2746af4..0000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/FileMetadata.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.s3native;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * <p>
- * Holds basic metadata for a file stored in a {@link NativeFileSystemStore}.
- * </p>
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-class FileMetadata {
-  private final String key;
-  private final long length;
-  private final long lastModified;
-  
-  public FileMetadata(String key, long length, long lastModified) {
-    this.key = key;
-    this.length = length;
-    this.lastModified = lastModified;
-  }
-  
-  public String getKey() {
-    return key;
-  }
-  
-  public long getLength() {
-    return length;
-  }
-
-  public long getLastModified() {
-    return lastModified;
-  }
-  
-  @Override
-  public String toString() {
-    return "FileMetadata[" + key + ", " + length + ", " + lastModified + "]";
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/Jets3tNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/Jets3tNativeFileSystemStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/Jets3tNativeFileSystemStore.java
deleted file mode 100644
index c9c0f98..0000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/Jets3tNativeFileSystemStore.java
+++ /dev/null
@@ -1,481 +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.s3native;
-
-import static org.apache.hadoop.fs.s3native.NativeS3FileSystem.PATH_DELIMITER;
-
-import java.io.BufferedInputStream;
-import java.io.ByteArrayInputStream;
-import java.io.EOFException;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSExceptionMessages;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.security.AccessControlException;
-import org.jets3t.service.S3Service;
-import org.jets3t.service.S3ServiceException;
-import org.jets3t.service.ServiceException;
-import org.jets3t.service.StorageObjectsChunk;
-import org.jets3t.service.impl.rest.HttpException;
-import org.jets3t.service.impl.rest.httpclient.RestS3Service;
-import org.jets3t.service.model.MultipartPart;
-import org.jets3t.service.model.MultipartUpload;
-import org.jets3t.service.model.S3Bucket;
-import org.jets3t.service.model.S3Object;
-import org.jets3t.service.model.StorageObject;
-import org.jets3t.service.security.AWSCredentials;
-import org.jets3t.service.utils.MultipartUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-class Jets3tNativeFileSystemStore implements NativeFileSystemStore {
-  
-  private S3Service s3Service;
-  private S3Bucket bucket;
-
-  private long multipartBlockSize;
-  private boolean multipartEnabled;
-  private long multipartCopyBlockSize;
-  static final long MAX_PART_SIZE = (long)5 * 1024 * 1024 * 1024;
-
-  private String serverSideEncryptionAlgorithm;
-  
-  public static final Logger LOG =
-      LoggerFactory.getLogger(Jets3tNativeFileSystemStore.class);
-
-  @Override
-  public void initialize(URI uri, Configuration conf) throws IOException {
-    S3Credentials s3Credentials = new S3Credentials();
-    s3Credentials.initialize(uri, conf);
-    try {
-      AWSCredentials awsCredentials =
-        new AWSCredentials(s3Credentials.getAccessKey(),
-            s3Credentials.getSecretAccessKey());
-      this.s3Service = new RestS3Service(awsCredentials);
-    } catch (S3ServiceException e) {
-      handleException(e);
-    }
-    multipartEnabled =
-        conf.getBoolean("fs.s3n.multipart.uploads.enabled", false);
-    multipartBlockSize = Math.min(
-        conf.getLong("fs.s3n.multipart.uploads.block.size", 64 * 1024 * 1024),
-        MAX_PART_SIZE);
-    multipartCopyBlockSize = Math.min(
-        conf.getLong("fs.s3n.multipart.copy.block.size", MAX_PART_SIZE),
-        MAX_PART_SIZE);
-    serverSideEncryptionAlgorithm = conf.get("fs.s3n.server-side-encryption-algorithm");
-
-    bucket = new S3Bucket(uri.getHost());
-  }
-  
-  @Override
-  public void storeFile(String key, File file, byte[] md5Hash)
-    throws IOException {
-
-    if (multipartEnabled && file.length() >= multipartBlockSize) {
-      storeLargeFile(key, file, md5Hash);
-      return;
-    }
-
-    BufferedInputStream in = null;
-    try {
-      in = new BufferedInputStream(new FileInputStream(file));
-      S3Object object = new S3Object(key);
-      object.setDataInputStream(in);
-      object.setContentType("binary/octet-stream");
-      object.setContentLength(file.length());
-      object.setServerSideEncryptionAlgorithm(serverSideEncryptionAlgorithm);
-      if (md5Hash != null) {
-        object.setMd5Hash(md5Hash);
-      }
-      s3Service.putObject(bucket, object);
-    } catch (ServiceException e) {
-      handleException(e, key);
-    } finally {
-      IOUtils.closeStream(in);
-    }
-  }
-
-  public void storeLargeFile(String key, File file, byte[] md5Hash)
-      throws IOException {
-    S3Object object = new S3Object(key);
-    object.setDataInputFile(file);
-    object.setContentType("binary/octet-stream");
-    object.setContentLength(file.length());
-    object.setServerSideEncryptionAlgorithm(serverSideEncryptionAlgorithm);
-    if (md5Hash != null) {
-      object.setMd5Hash(md5Hash);
-    }
-
-    List<StorageObject> objectsToUploadAsMultipart =
-        new ArrayList<StorageObject>();
-    objectsToUploadAsMultipart.add(object);
-    MultipartUtils mpUtils = new MultipartUtils(multipartBlockSize);
-
-    try {
-      mpUtils.uploadObjects(bucket.getName(), s3Service,
-                            objectsToUploadAsMultipart, null);
-    } catch (Exception e) {
-      handleException(e, key);
-    }
-  }
-  
-  @Override
-  public void storeEmptyFile(String key) throws IOException {
-    try {
-      S3Object object = new S3Object(key);
-      object.setDataInputStream(new ByteArrayInputStream(new byte[0]));
-      object.setContentType("binary/octet-stream");
-      object.setContentLength(0);
-      object.setServerSideEncryptionAlgorithm(serverSideEncryptionAlgorithm);
-      s3Service.putObject(bucket, object);
-    } catch (ServiceException e) {
-      handleException(e, key);
-    }
-  }
-
-  @Override
-  public FileMetadata retrieveMetadata(String key) throws IOException {
-    StorageObject object = null;
-    try {
-      LOG.debug("Getting metadata for key: {} from bucket: {}",
-          key, bucket.getName());
-      object = s3Service.getObjectDetails(bucket.getName(), key);
-      return new FileMetadata(key, object.getContentLength(),
-          object.getLastModifiedDate().getTime());
-
-    } catch (ServiceException e) {
-      try {
-        // process
-        handleException(e, key);
-        return null;
-      } catch (FileNotFoundException fnfe) {
-        // and downgrade missing files
-        return null;
-      }
-    } finally {
-      if (object != null) {
-        object.closeDataInputStream();
-      }
-    }
-  }
-
-  /**
-   * @param key
-   * The key is the object name that is being retrieved from the S3 bucket
-   * @return
-   * This method returns null if the key is not found
-   * @throws IOException
-   */
-
-  @Override
-  public InputStream retrieve(String key) throws IOException {
-    try {
-      LOG.debug("Getting key: {} from bucket: {}",
-          key, bucket.getName());
-      S3Object object = s3Service.getObject(bucket.getName(), key);
-      return object.getDataInputStream();
-    } catch (ServiceException e) {
-      handleException(e, key);
-      return null; //return null if key not found
-    }
-  }
-
-  /**
-   *
-   * @param key
-   * The key is the object name that is being retrieved from the S3 bucket
-   * @return
-   * This method returns null if the key is not found
-   * @throws IOException
-   */
-
-  @Override
-  public InputStream retrieve(String key, long byteRangeStart)
-          throws IOException {
-    try {
-      LOG.debug("Getting key: {} from bucket: {} with byteRangeStart: {}",
-          key, bucket.getName(), byteRangeStart);
-      S3Object object = s3Service.getObject(bucket, key, null, null, null,
-                                            null, byteRangeStart, null);
-      return object.getDataInputStream();
-    } catch (ServiceException e) {
-      handleException(e, key);
-      return null;
-    }
-  }
-
-  @Override
-  public PartialListing list(String prefix, int maxListingLength)
-          throws IOException {
-    return list(prefix, maxListingLength, null, false);
-  }
-  
-  @Override
-  public PartialListing list(String prefix, int maxListingLength, String priorLastKey,
-      boolean recurse) throws IOException {
-
-    return list(prefix, recurse ? null : PATH_DELIMITER, maxListingLength, priorLastKey);
-  }
-
-  /**
-   * list objects
-   * @param prefix prefix
-   * @param delimiter delimiter
-   * @param maxListingLength max no. of entries
-   * @param priorLastKey last key in any previous search
-   * @return a list of matches
-   * @throws IOException on any reported failure
-   */
-
-  private PartialListing list(String prefix, String delimiter,
-      int maxListingLength, String priorLastKey) throws IOException {
-    try {
-      if (!prefix.isEmpty() && !prefix.endsWith(PATH_DELIMITER)) {
-        prefix += PATH_DELIMITER;
-      }
-      StorageObjectsChunk chunk = s3Service.listObjectsChunked(bucket.getName(),
-          prefix, delimiter, maxListingLength, priorLastKey);
-      
-      FileMetadata[] fileMetadata =
-        new FileMetadata[chunk.getObjects().length];
-      for (int i = 0; i < fileMetadata.length; i++) {
-        StorageObject object = chunk.getObjects()[i];
-        fileMetadata[i] = new FileMetadata(object.getKey(),
-            object.getContentLength(), object.getLastModifiedDate().getTime());
-      }
-      return new PartialListing(chunk.getPriorLastKey(), fileMetadata,
-          chunk.getCommonPrefixes());
-    } catch (ServiceException e) {
-      handleException(e, prefix);
-      return null; // never returned - keep compiler happy
-    }
-  }
-
-  @Override
-  public void delete(String key) throws IOException {
-    try {
-      LOG.debug("Deleting key: {} from bucket: {}",
-          key, bucket.getName());
-      s3Service.deleteObject(bucket, key);
-    } catch (ServiceException e) {
-      handleException(e, key);
-    }
-  }
-
-  public void rename(String srcKey, String dstKey) throws IOException {
-    try {
-      s3Service.renameObject(bucket.getName(), srcKey, new S3Object(dstKey));
-    } catch (ServiceException e) {
-      handleException(e, srcKey);
-    }
-  }
-  
-  @Override
-  public void copy(String srcKey, String dstKey) throws IOException {
-    try {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Copying srcKey: " + srcKey + "to dstKey: " + dstKey + "in bucket: " + bucket.getName());
-      }
-      if (multipartEnabled) {
-        S3Object object = s3Service.getObjectDetails(bucket, srcKey, null,
-                                                     null, null, null);
-        if (multipartCopyBlockSize > 0 &&
-            object.getContentLength() > multipartCopyBlockSize) {
-          copyLargeFile(object, dstKey);
-          return;
-        }
-      }
-
-      S3Object dstObject = new S3Object(dstKey);
-      dstObject.setServerSideEncryptionAlgorithm(serverSideEncryptionAlgorithm);
-      s3Service.copyObject(bucket.getName(), srcKey, bucket.getName(),
-          dstObject, false);
-    } catch (ServiceException e) {
-      handleException(e, srcKey);
-    }
-  }
-
-  public void copyLargeFile(S3Object srcObject, String dstKey) throws IOException {
-    try {
-      long partCount = srcObject.getContentLength() / multipartCopyBlockSize +
-          (srcObject.getContentLength() % multipartCopyBlockSize > 0 ? 1 : 0);
-
-      MultipartUpload multipartUpload = s3Service.multipartStartUpload
-          (bucket.getName(), dstKey, srcObject.getMetadataMap());
-
-      List<MultipartPart> listedParts = new ArrayList<MultipartPart>();
-      for (int i = 0; i < partCount; i++) {
-        long byteRangeStart = i * multipartCopyBlockSize;
-        long byteLength;
-        if (i < partCount - 1) {
-          byteLength = multipartCopyBlockSize;
-        } else {
-          byteLength = srcObject.getContentLength() % multipartCopyBlockSize;
-          if (byteLength == 0) {
-            byteLength = multipartCopyBlockSize;
-          }
-        }
-
-        MultipartPart copiedPart = s3Service.multipartUploadPartCopy
-            (multipartUpload, i + 1, bucket.getName(), srcObject.getKey(),
-             null, null, null, null, byteRangeStart,
-             byteRangeStart + byteLength - 1, null);
-        listedParts.add(copiedPart);
-      }
-      
-      Collections.reverse(listedParts);
-      s3Service.multipartCompleteUpload(multipartUpload, listedParts);
-    } catch (ServiceException e) {
-      handleException(e, srcObject.getKey());
-    }
-  }
-
-  @Override
-  public void purge(String prefix) throws IOException {
-    String key = "";
-    try {
-      S3Object[] objects =
-          s3Service.listObjects(bucket.getName(), prefix, null);
-      for (S3Object object : objects) {
-        key = object.getKey();
-        s3Service.deleteObject(bucket, key);
-      }
-    } catch (S3ServiceException e) {
-      handleException(e, key);
-    }
-  }
-
-  @Override
-  public void dump() throws IOException {
-    StringBuilder sb = new StringBuilder("S3 Native Filesystem, ");
-    sb.append(bucket.getName()).append("\n");
-    try {
-      S3Object[] objects = s3Service.listObjects(bucket.getName());
-      for (S3Object object : objects) {
-        sb.append(object.getKey()).append("\n");
-      }
-    } catch (S3ServiceException e) {
-      handleException(e);
-    }
-    System.out.println(sb);
-  }
-
-  /**
-   * Handle any service exception by translating it into an IOException
-   * @param e exception
-   * @throws IOException exception -always
-   */
-  private void handleException(Exception e) throws IOException {
-    throw processException(e, e, "");
-  }
-  /**
-   * Handle any service exception by translating it into an IOException
-   * @param e exception
-   * @param key key sought from object store
-
-   * @throws IOException exception -always
-   */
-  private void handleException(Exception e, String key) throws IOException {
-    throw processException(e, e, key);
-  }
-
-  /**
-   * Handle any service exception by translating it into an IOException
-   * @param thrown exception
-   * @param original original exception -thrown if no other translation could
-   * be made
-   * @param key key sought from object store or "" for undefined
-   * @return an exception to throw. If isProcessingCause==true this may be null.
-   */
-  private IOException processException(Throwable thrown, Throwable original,
-      String key) {
-    IOException result;
-    if (thrown.getCause() != null) {
-      // recurse down
-      result = processException(thrown.getCause(), original, key);
-    } else if (thrown instanceof HttpException) {
-      // nested HttpException - examine error code and react
-      HttpException httpException = (HttpException) thrown;
-      String responseMessage = httpException.getResponseMessage();
-      int responseCode = httpException.getResponseCode();
-      String bucketName = "s3n://" + bucket.getName();
-      String text = String.format("%s : %03d : %s",
-          bucketName,
-          responseCode,
-          responseMessage);
-      String filename = !key.isEmpty() ? (bucketName + "/" + key) : text;
-      IOException ioe;
-      switch (responseCode) {
-        case 404:
-          result = new FileNotFoundException(filename);
-          break;
-        case 416: // invalid range
-          result = new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF
-                                    +": " + filename);
-          break;
-        case 403: //forbidden
-          result = new AccessControlException("Permission denied"
-                                    +": " + filename);
-          break;
-        default:
-          result = new IOException(text);
-      }
-      result.initCause(thrown);
-    } else if (thrown instanceof S3ServiceException) {
-      S3ServiceException se = (S3ServiceException) thrown;
-      LOG.debug(
-          "S3ServiceException: {}: {} : {}",
-          se.getS3ErrorCode(), se.getS3ErrorMessage(), se, se);
-      if ("InvalidRange".equals(se.getS3ErrorCode())) {
-        result = new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
-      } else {
-        result = new S3Exception(se);
-      }
-    } else if (thrown instanceof ServiceException) {
-      ServiceException se = (ServiceException) thrown;
-      LOG.debug("S3ServiceException: {}: {} : {}",
-          se.getErrorCode(), se.toString(), se, se);
-      result = new S3Exception(se);
-    } else if (thrown instanceof IOException) {
-      result = (IOException) thrown;
-    } else {
-      // here there is no exception derived yet.
-      // this means no inner cause, and no translation made yet.
-      // convert the original to an IOException -rather than just the
-      // exception at the base of the tree
-      result = new S3Exception(original);
-    }
-
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeFileSystemStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeFileSystemStore.java
deleted file mode 100644
index f26cdac..0000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeFileSystemStore.java
+++ /dev/null
@@ -1,67 +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.s3native;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URI;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * <p>
- * An abstraction for a key-based {@link File} store.
- * </p>
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-interface NativeFileSystemStore {
-  
-  void initialize(URI uri, Configuration conf) throws IOException;
-  
-  void storeFile(String key, File file, byte[] md5Hash) throws IOException;
-  void storeEmptyFile(String key) throws IOException;
-  
-  FileMetadata retrieveMetadata(String key) throws IOException;
-  InputStream retrieve(String key) throws IOException;
-  InputStream retrieve(String key, long byteRangeStart) throws IOException;
-  
-  PartialListing list(String prefix, int maxListingLength) throws IOException;
-  PartialListing list(String prefix, int maxListingLength, String priorLastKey, boolean recursive)
-    throws IOException;
-  
-  void delete(String key) throws IOException;
-
-  void copy(String srcKey, String dstKey) throws IOException;
-  
-  /**
-   * Delete all keys with the given prefix. Used for testing.
-   * @throws IOException
-   */
-  void purge(String prefix) throws IOException;
-  
-  /**
-   * Diagnostic method to dump state to the console.
-   * @throws IOException
-   */
-  void dump() throws IOException;
-}


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


[27/50] [abbrv] hadoop git commit: HADOOP-14553. Add (parallelized) integration tests to hadoop-azure Contributed by Steve Loughran

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationsWithThreads.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationsWithThreads.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationsWithThreads.java
new file mode 100644
index 0000000..4389fda
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationsWithThreads.java
@@ -0,0 +1,821 @@
+/**
+ * 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.azure;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azure.NativeAzureFileSystem.FolderRenamePending;
+import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Tests the Native Azure file system (WASB) using parallel threads for rename and delete operations.
+ */
+public class ITestFileSystemOperationsWithThreads extends AbstractWasbTestBase {
+
+  private final int renameThreads = 10;
+  private final int deleteThreads = 20;
+  private int iterations = 1;
+  private LogCapturer logs = null;
+
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    Configuration conf = fs.getConf();
+
+    // By default enable parallel threads for rename and delete operations.
+    // Also enable flat listing of blobs for these operations.
+    conf.setInt(NativeAzureFileSystem.AZURE_RENAME_THREADS, renameThreads);
+    conf.setInt(NativeAzureFileSystem.AZURE_DELETE_THREADS, deleteThreads);
+    conf.setBoolean(AzureNativeFileSystemStore.KEY_ENABLE_FLAT_LISTING, true);
+
+    URI uri = fs.getUri();
+    fs.initialize(uri, conf);
+
+    // Capture logs
+    logs = LogCapturer.captureLogs(new Log4JLogger(org.apache.log4j.Logger
+        .getRootLogger()));
+  }
+
+  /*
+   * Helper method to create sub directory and different types of files
+   * for multiple iterations.
+   */
+  private void createFolder(FileSystem fs, String root) throws Exception {
+    fs.mkdirs(new Path(root));
+    for (int i = 0; i < this.iterations; i++) {
+      fs.mkdirs(new Path(root + "/" + i));
+      fs.createNewFile(new Path(root + "/" + i + "/fileToRename"));
+      fs.createNewFile(new Path(root + "/" + i + "/file/to/rename"));
+      fs.createNewFile(new Path(root + "/" + i + "/file+to%rename"));
+      fs.createNewFile(new Path(root + "/fileToRename" + i));
+    }
+  }
+
+  /*
+   * Helper method to do rename operation and validate all files in source folder
+   * doesn't exists and similar files exists in new folder.
+   */
+  private void validateRenameFolder(FileSystem fs, String source, String dest) throws Exception {
+    // Create source folder with files.
+    createFolder(fs, source);
+    Path sourceFolder = new Path(source);
+    Path destFolder = new Path(dest);
+
+    // rename operation
+    assertTrue(fs.rename(sourceFolder, destFolder));
+    assertTrue(fs.exists(destFolder));
+
+    for (int i = 0; i < this.iterations; i++) {
+      // Check destination folder and files exists.
+      assertTrue(fs.exists(new Path(dest + "/" + i)));
+      assertTrue(fs.exists(new Path(dest + "/" + i + "/fileToRename")));
+      assertTrue(fs.exists(new Path(dest + "/" + i + "/file/to/rename")));
+      assertTrue(fs.exists(new Path(dest + "/" + i + "/file+to%rename")));
+      assertTrue(fs.exists(new Path(dest + "/fileToRename" + i)));
+
+      // Check source folder and files doesn't exists.
+      assertFalse(fs.exists(new Path(source + "/" + i)));
+      assertFalse(fs.exists(new Path(source + "/" + i + "/fileToRename")));
+      assertFalse(fs.exists(new Path(source + "/" + i + "/file/to/rename")));
+      assertFalse(fs.exists(new Path(source + "/" + i + "/file+to%rename")));
+      assertFalse(fs.exists(new Path(source + "/fileToRename" + i)));
+    }
+  }
+
+  /*
+   * Test case for rename operation with multiple threads and flat listing enabled.
+   */
+  @Test
+  public void testRenameSmallFolderWithThreads() throws Exception {
+
+    validateRenameFolder(fs, "root", "rootnew");
+
+    // With single iteration, we would have created 7 blobs.
+    int expectedThreadsCreated = Math.min(7, renameThreads);
+
+    // Validate from logs that threads are created.
+    String content = logs.getOutput();
+    assertInLog(content, "ms with threads: " + expectedThreadsCreated);
+
+    // Validate thread executions
+    for (int i = 0; i < expectedThreadsCreated; i++) {
+      assertInLog(content,
+          "AzureBlobRenameThread-" + Thread.currentThread().getName() + "-" + i);
+    }
+
+    // Also ensure that we haven't spawned extra threads.
+    if (expectedThreadsCreated < renameThreads) {
+      for (int i = expectedThreadsCreated; i < renameThreads; i++) {
+        assertNotInLog(content,
+            "AzureBlobRenameThread-" + Thread.currentThread().getName() + "-" + i);
+      }
+    }
+  }
+
+  /*
+   * Test case for rename operation with multiple threads and flat listing enabled.
+   */
+  @Test
+  public void testRenameLargeFolderWithThreads() throws Exception {
+
+    // Populate source folder with large number of files and directories.
+    this.iterations = 10;
+    validateRenameFolder(fs, "root", "rootnew");
+
+    // Validate from logs that threads are created.
+    String content = logs.getOutput();
+    assertInLog(content, "ms with threads: " + renameThreads);
+
+    // Validate thread executions
+    for (int i = 0; i < renameThreads; i++) {
+      assertInLog(content,
+          "AzureBlobRenameThread-" + Thread.currentThread().getName() + "-" + i);
+    }
+  }
+
+  /*
+   * Test case for rename operation with threads disabled and flat listing enabled.
+   */
+  @Test
+  public void testRenameLargeFolderDisableThreads() throws Exception {
+    Configuration conf = fs.getConf();
+
+    // Number of threads set to 0 or 1 disables threads.
+    conf.setInt(NativeAzureFileSystem.AZURE_RENAME_THREADS, 0);
+    URI uri = fs.getUri();
+    fs.initialize(uri, conf);
+
+    // Populate source folder with large number of files and directories.
+    this.iterations = 10;
+    validateRenameFolder(fs, "root", "rootnew");
+
+    // Validate from logs that threads are disabled.
+    String content = logs.getOutput();
+    assertInLog(content,
+        "Disabling threads for Rename operation as thread count 0");
+
+    // Validate no thread executions
+    for (int i = 0; i < renameThreads; i++) {
+      String term = "AzureBlobRenameThread-"
+          + Thread.currentThread().getName()
+          + "-" + i;
+      assertNotInLog(content, term);
+    }
+  }
+
+  /**
+   * Assert that a log contains the given term.
+   * @param content log output
+   * @param term search term
+   */
+  protected void assertInLog(String content, String term) {
+    assertTrue("Empty log", !content.isEmpty());
+    if (!content.contains(term)) {
+      String message = "No " + term + " found in logs";
+      LOG.error(message);
+      System.err.println(content);
+      fail(message);
+    }
+  }
+
+  /**
+   * Assert that a log does not contain the given term.
+   * @param content log output
+   * @param term search term
+   */
+  protected void assertNotInLog(String content, String term) {
+    assertTrue("Empty log", !content.isEmpty());
+    if (content.contains(term)) {
+      String message = term + " found in logs";
+      LOG.error(message);
+      System.err.println(content);
+      fail(message);
+    }
+  }
+
+  /*
+   * Test case for rename operation with threads and flat listing disabled.
+   */
+  @Test
+  public void testRenameSmallFolderDisableThreadsDisableFlatListing() throws Exception {
+    Configuration conf = fs.getConf();
+    conf = fs.getConf();
+
+    // Number of threads set to 0 or 1 disables threads.
+    conf.setInt(NativeAzureFileSystem.AZURE_RENAME_THREADS, 1);
+    conf.setBoolean(AzureNativeFileSystemStore.KEY_ENABLE_FLAT_LISTING, false);
+    URI uri = fs.getUri();
+    fs.initialize(uri, conf);
+
+    validateRenameFolder(fs, "root", "rootnew");
+
+    // Validate from logs that threads are disabled.
+    String content = logs.getOutput();
+    assertInLog(content,
+        "Disabling threads for Rename operation as thread count 1");
+
+    // Validate no thread executions
+    for (int i = 0; i < renameThreads; i++) {
+      assertNotInLog(content,
+          "AzureBlobRenameThread-" + Thread.currentThread().getName() + "-" + i);
+    }
+  }
+
+  /*
+   * Helper method to do delete operation and validate all files in source folder
+   * doesn't exists after delete operation.
+   */
+  private void validateDeleteFolder(FileSystem fs, String source)  throws Exception {
+    // Create folder with files.
+    createFolder(fs, "root");
+    Path sourceFolder = new Path(source);
+
+    // Delete operation
+    assertTrue(fs.delete(sourceFolder, true));
+    assertFalse(fs.exists(sourceFolder));
+
+    for (int i = 0; i < this.iterations; i++) {
+      // check that source folder and files doesn't exists
+      assertFalse(fs.exists(new Path(source + "/" + i)));
+      assertFalse(fs.exists(new Path(source + "/" + i + "/fileToRename")));
+      assertFalse(fs.exists(new Path(source + "/" + i + "/file/to/rename")));
+      assertFalse(fs.exists(new Path(source + "/" + i + "/file+to%rename")));
+      assertFalse(fs.exists(new Path(source + "/fileToRename" + i)));
+    }
+  }
+
+  /*
+   * Test case for delete operation with multiple threads and flat listing enabled.
+   */
+  @Test
+  public void testDeleteSmallFolderWithThreads() throws Exception {
+
+    validateDeleteFolder(fs, "root");
+
+    // With single iteration, we would have created 7 blobs.
+    int expectedThreadsCreated = Math.min(7, deleteThreads);
+
+    // Validate from logs that threads are enabled.
+    String content = logs.getOutput();
+    assertInLog(content, "ms with threads: " + expectedThreadsCreated);
+
+    // Validate thread executions
+    for (int i = 0; i < expectedThreadsCreated; i++) {
+      assertInLog(content,
+          "AzureBlobDeleteThread-" + Thread.currentThread().getName() + "-" + i);
+    }
+
+    // Also ensure that we haven't spawned extra threads.
+    if (expectedThreadsCreated < deleteThreads) {
+      for (int i = expectedThreadsCreated; i < deleteThreads; i++) {
+        assertNotInLog(content,
+            "AzureBlobDeleteThread-" + Thread.currentThread().getName() + "-" + i);
+      }
+    }
+  }
+
+  /*
+   * Test case for delete operation with multiple threads and flat listing enabled.
+   */
+  @Test
+  public void testDeleteLargeFolderWithThreads() throws Exception {
+    // Populate source folder with large number of files and directories.
+    this.iterations = 10;
+    validateDeleteFolder(fs, "root");
+
+    // Validate from logs that threads are enabled.
+    String content = logs.getOutput();
+    assertInLog(content, "ms with threads: " + deleteThreads);
+
+    // Validate thread executions
+    for (int i = 0; i < deleteThreads; i++) {
+      assertInLog(content,
+          "AzureBlobDeleteThread-" + Thread.currentThread().getName() + "-" + i);
+    }
+  }
+
+  /*
+   * Test case for delete operation with threads disabled and flat listing enabled.
+   */
+  @Test
+  public void testDeleteLargeFolderDisableThreads() throws Exception {
+    Configuration conf = fs.getConf();
+    conf.setInt(NativeAzureFileSystem.AZURE_DELETE_THREADS, 0);
+    URI uri = fs.getUri();
+    fs.initialize(uri, conf);
+
+    // Populate source folder with large number of files and directories.
+    this.iterations = 10;
+    validateDeleteFolder(fs, "root");
+
+    // Validate from logs that threads are disabled.
+    String content = logs.getOutput();
+    assertInLog(content,
+        "Disabling threads for Delete operation as thread count 0");
+
+    // Validate no thread executions
+    for (int i = 0; i < deleteThreads; i++) {
+      assertNotInLog(content,
+          "AzureBlobDeleteThread-" + Thread.currentThread().getName() + "-" + i);
+    }
+  }
+
+  /*
+   * Test case for rename operation with threads and flat listing disabled.
+   */
+  @Test
+  public void testDeleteSmallFolderDisableThreadsDisableFlatListing() throws Exception {
+    Configuration conf = fs.getConf();
+
+    // Number of threads set to 0 or 1 disables threads.
+    conf.setInt(NativeAzureFileSystem.AZURE_DELETE_THREADS, 1);
+    conf.setBoolean(AzureNativeFileSystemStore.KEY_ENABLE_FLAT_LISTING, false);
+    URI uri = fs.getUri();
+    fs.initialize(uri, conf);
+
+    validateDeleteFolder(fs, "root");
+
+    // Validate from logs that threads are disabled.
+    String content = logs.getOutput();
+    assertInLog(content,
+        "Disabling threads for Delete operation as thread count 1");
+
+    // Validate no thread executions
+    for (int i = 0; i < deleteThreads; i++) {
+      assertNotInLog(content,
+          "AzureBlobDeleteThread-" + Thread.currentThread().getName() + "-" + i);
+    }
+  }
+
+  /*
+   * Test case for delete operation with multiple threads and flat listing enabled.
+   */
+  @Test
+  public void testDeleteThreadPoolExceptionFailure() throws Exception {
+
+    // Spy azure file system object and raise exception for new thread pool
+    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
+
+    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root")));
+
+    AzureFileSystemThreadPoolExecutor mockThreadPoolExecutor = Mockito.spy(
+        mockFs.getThreadPoolExecutor(deleteThreads, "AzureBlobDeleteThread", "Delete",
+            path, NativeAzureFileSystem.AZURE_DELETE_THREADS));
+    Mockito.when(mockThreadPoolExecutor.getThreadPool(7)).thenThrow(new Exception());
+
+    // With single iteration, we would have created 7 blobs resulting 7 threads.
+    Mockito.when(mockFs.getThreadPoolExecutor(deleteThreads, "AzureBlobDeleteThread", "Delete",
+        path, NativeAzureFileSystem.AZURE_DELETE_THREADS)).thenReturn(mockThreadPoolExecutor);
+
+    validateDeleteFolder(mockFs, "root");
+
+    // Validate from logs that threads are disabled.
+    String content = logs.getOutput();
+    assertInLog(content, "Failed to create thread pool with threads");
+    assertInLog(content, "Serializing the Delete operation");
+  }
+
+  /*
+   * Test case for delete operation with multiple threads and flat listing enabled.
+   */
+  @Test
+  public void testDeleteThreadPoolExecuteFailure() throws Exception {
+
+    // Mock thread pool executor to throw exception for all requests.
+    ThreadPoolExecutor mockThreadExecutor = Mockito.mock(ThreadPoolExecutor.class);
+    Mockito.doThrow(new RejectedExecutionException()).when(mockThreadExecutor).execute(Mockito.any(Runnable.class));
+
+    // Spy azure file system object and return mocked thread pool
+    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
+
+    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root")));
+
+    AzureFileSystemThreadPoolExecutor mockThreadPoolExecutor = Mockito.spy(
+        mockFs.getThreadPoolExecutor(deleteThreads, "AzureBlobDeleteThread", "Delete",
+            path, NativeAzureFileSystem.AZURE_DELETE_THREADS));
+    Mockito.when(mockThreadPoolExecutor.getThreadPool(7)).thenReturn(mockThreadExecutor);
+
+    // With single iteration, we would have created 7 blobs resulting 7 threads.
+    Mockito.when(mockFs.getThreadPoolExecutor(deleteThreads, "AzureBlobDeleteThread", "Delete",
+        path, NativeAzureFileSystem.AZURE_DELETE_THREADS)).thenReturn(mockThreadPoolExecutor);
+
+    validateDeleteFolder(mockFs, "root");
+
+    // Validate from logs that threads are disabled.
+    String content = logs.getOutput();
+    assertInLog(content,
+        "Rejected execution of thread for Delete operation on blob");
+    assertInLog(content, "Serializing the Delete operation");
+  }
+
+  /*
+   * Test case for delete operation with multiple threads and flat listing enabled.
+   */
+  @Test
+  public void testDeleteThreadPoolExecuteSingleThreadFailure() throws Exception {
+
+    // Spy azure file system object and return mocked thread pool
+    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
+
+    // Spy a thread pool executor and link it to azure file system object.
+    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root")));
+    AzureFileSystemThreadPoolExecutor mockThreadPoolExecutor = Mockito.spy(
+        mockFs.getThreadPoolExecutor(deleteThreads, "AzureBlobDeleteThread", "Delete",
+            path, NativeAzureFileSystem.AZURE_DELETE_THREADS));
+
+    // With single iteration, we would have created 7 blobs resulting 7 threads.
+    Mockito.when(mockFs.getThreadPoolExecutor(deleteThreads, "AzureBlobDeleteThread", "Delete",
+        path, NativeAzureFileSystem.AZURE_DELETE_THREADS)).thenReturn(mockThreadPoolExecutor);
+
+    // Create a thread executor and link it to mocked thread pool executor object.
+    ThreadPoolExecutor mockThreadExecutor = Mockito.spy(mockThreadPoolExecutor.getThreadPool(7));
+    Mockito.when(mockThreadPoolExecutor.getThreadPool(7)).thenReturn(mockThreadExecutor);
+
+    // Mock thread executor to throw exception for all requests.
+    Mockito.doCallRealMethod().doThrow(new RejectedExecutionException()).when(mockThreadExecutor).execute(Mockito.any(Runnable.class));
+
+    validateDeleteFolder(mockFs, "root");
+
+    // Validate from logs that threads are enabled and unused threads.
+    String content = logs.getOutput();
+    assertInLog(content,
+        "Using thread pool for Delete operation with threads 7");
+    assertInLog(content,
+        "6 threads not used for Delete operation on blob");
+  }
+
+  /*
+   * Test case for delete operation with multiple threads and flat listing enabled.
+   */
+  @Test
+  public void testDeleteThreadPoolTerminationFailure() throws Exception {
+
+    // Spy azure file system object and return mocked thread pool
+    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
+
+    // Spy a thread pool executor and link it to azure file system object.
+    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root")));
+    AzureFileSystemThreadPoolExecutor mockThreadPoolExecutor = Mockito.spy(
+        ((NativeAzureFileSystem) fs).getThreadPoolExecutor(deleteThreads, "AzureBlobDeleteThread", "Delete",
+            path, NativeAzureFileSystem.AZURE_DELETE_THREADS));
+
+    // Create a thread executor and link it to mocked thread pool executor object.
+    // Mock thread executor to throw exception for terminating threads.
+    ThreadPoolExecutor mockThreadExecutor = Mockito.mock(ThreadPoolExecutor.class);
+    Mockito.doNothing().when(mockThreadExecutor).execute(Mockito.any(Runnable.class));
+    Mockito.when(mockThreadExecutor.awaitTermination(Long.MAX_VALUE, TimeUnit.DAYS)).thenThrow(new InterruptedException());
+
+    Mockito.when(mockThreadPoolExecutor.getThreadPool(7)).thenReturn(mockThreadExecutor);
+
+    // With single iteration, we would have created 7 blobs resulting 7 threads.
+    Mockito.when(mockFs.getThreadPoolExecutor(deleteThreads, "AzureBlobDeleteThread", "Delete",
+        path, NativeAzureFileSystem.AZURE_DELETE_THREADS)).thenReturn(mockThreadPoolExecutor);
+
+    createFolder(mockFs, "root");
+    Path sourceFolder = new Path("root");
+    boolean exception = false;
+    try {
+      mockFs.delete(sourceFolder, true);
+    } catch (IOException e){
+      exception = true;
+    }
+
+    assertTrue(exception);
+    assertTrue(mockFs.exists(sourceFolder));
+
+    // Validate from logs that threads are enabled and delete operation is failed.
+    String content = logs.getOutput();
+    assertInLog(content,
+        "Using thread pool for Delete operation with threads");
+    assertInLog(content, "Threads got interrupted Delete blob operation");
+    assertInLog(content,
+        "Delete failed as operation on subfolders and files failed.");
+  }
+
+  /*
+   * Validate that when a directory is deleted recursively, the operation succeeds
+   * even if a child directory delete fails because the directory does not exist.
+   * This can happen if a child directory is deleted by an external agent while
+   * the parent is in progress of being deleted recursively.
+   */
+  @Test
+  public void testRecursiveDirectoryDeleteWhenChildDirectoryDeleted()
+      throws Exception {
+    testRecusiveDirectoryDelete(true);
+  }
+
+  /*
+   * Validate that when a directory is deleted recursively, the operation succeeds
+   * even if a file delete fails because it does not exist.
+   * This can happen if a file is deleted by an external agent while
+   * the parent directory is in progress of being deleted.
+   */
+  @Test
+  public void testRecursiveDirectoryDeleteWhenDeletingChildFileReturnsFalse()
+      throws Exception {
+    testRecusiveDirectoryDelete(false);
+  }
+
+  private void testRecusiveDirectoryDelete(boolean useDir) throws Exception {
+    String childPathToBeDeletedByExternalAgent = (useDir)
+        ? "root/0"
+        : "root/0/fileToRename";
+    // Spy azure file system object and return false for deleting one file
+    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
+    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path(
+        childPathToBeDeletedByExternalAgent)));
+
+    Answer<Boolean> answer = new Answer<Boolean>() {
+      public Boolean answer(InvocationOnMock invocation) throws Throwable {
+        String path = (String) invocation.getArguments()[0];
+        boolean isDir = (boolean) invocation.getArguments()[1];
+        boolean realResult = fs.deleteFile(path, isDir);
+        assertTrue(realResult);
+        boolean fakeResult = false;
+        return fakeResult;
+      }
+    };
+
+    Mockito.when(mockFs.deleteFile(path, useDir)).thenAnswer(answer);
+
+    createFolder(mockFs, "root");
+    Path sourceFolder = new Path("root");
+
+    assertTrue(mockFs.delete(sourceFolder, true));
+    assertFalse(mockFs.exists(sourceFolder));
+
+    // Validate from logs that threads are enabled, that a child directory was
+    // deleted by an external caller, and the parent delete operation still
+    // succeeds.
+    String content = logs.getOutput();
+    assertInLog(content,
+        "Using thread pool for Delete operation with threads");
+    assertInLog(content, String.format("Attempt to delete non-existent %s %s",
+        useDir ? "directory" : "file", path));
+  }
+
+  /*
+   * Test case for delete operation with multiple threads and flat listing enabled.
+   */
+  @Test
+  public void testDeleteSingleDeleteException() throws Exception {
+
+    // Spy azure file system object and raise exception for deleting one file
+    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
+    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root/0")));
+    Mockito.doThrow(new IOException()).when(mockFs).deleteFile(path, true);
+
+    createFolder(mockFs, "root");
+    Path sourceFolder = new Path("root");
+
+    boolean exception = false;
+    try {
+      mockFs.delete(sourceFolder, true);
+    } catch (IOException e){
+      exception = true;
+    }
+
+    assertTrue(exception);
+    assertTrue(mockFs.exists(sourceFolder));
+
+    // Validate from logs that threads are enabled and delete operation failed.
+    String content = logs.getOutput();
+    assertInLog(content,
+        "Using thread pool for Delete operation with threads");
+    assertInLog(content,
+        "Encountered Exception for Delete operation for file " + path);
+    assertInLog(content,
+        "Terminating execution of Delete operation now as some other thread already got exception or operation failed");
+  }
+
+  /*
+   * Test case for rename operation with multiple threads and flat listing enabled.
+   */
+  @Test
+  public void testRenameThreadPoolExceptionFailure() throws Exception {
+
+    // Spy azure file system object and raise exception for new thread pool
+    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
+
+    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root")));
+    AzureFileSystemThreadPoolExecutor mockThreadPoolExecutor = Mockito.spy(
+        ((NativeAzureFileSystem) fs).getThreadPoolExecutor(renameThreads, "AzureBlobRenameThread", "Rename",
+            path, NativeAzureFileSystem.AZURE_RENAME_THREADS));
+    Mockito.when(mockThreadPoolExecutor.getThreadPool(7)).thenThrow(new Exception());
+
+    // With single iteration, we would have created 7 blobs resulting 7 threads.
+    Mockito.doReturn(mockThreadPoolExecutor).when(mockFs).getThreadPoolExecutor(renameThreads, "AzureBlobRenameThread", "Rename",
+        path, NativeAzureFileSystem.AZURE_RENAME_THREADS);
+
+    validateRenameFolder(mockFs, "root", "rootnew");
+
+    // Validate from logs that threads are disabled.
+    String content = logs.getOutput();
+    assertInLog(content, "Failed to create thread pool with threads");
+    assertInLog(content, "Serializing the Rename operation");
+  }
+
+  /*
+   * Test case for rename operation with multiple threads and flat listing enabled.
+   */
+  @Test
+  public void testRenameThreadPoolExecuteFailure() throws Exception {
+
+    // Mock thread pool executor to throw exception for all requests.
+    ThreadPoolExecutor mockThreadExecutor = Mockito.mock(ThreadPoolExecutor.class);
+    Mockito.doThrow(new RejectedExecutionException()).when(mockThreadExecutor).execute(Mockito.any(Runnable.class));
+
+    // Spy azure file system object and return mocked thread pool
+    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
+
+    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root")));
+    AzureFileSystemThreadPoolExecutor mockThreadPoolExecutor = Mockito.spy(
+        mockFs.getThreadPoolExecutor(renameThreads, "AzureBlobRenameThread", "Rename",
+            path, NativeAzureFileSystem.AZURE_RENAME_THREADS));
+    Mockito.when(mockThreadPoolExecutor.getThreadPool(7)).thenReturn(mockThreadExecutor);
+
+    // With single iteration, we would have created 7 blobs resulting 7 threads.
+    Mockito.when(mockFs.getThreadPoolExecutor(renameThreads, "AzureBlobRenameThread", "Rename",
+        path, NativeAzureFileSystem.AZURE_RENAME_THREADS)).thenReturn(mockThreadPoolExecutor);
+
+    validateRenameFolder(mockFs, "root", "rootnew");
+
+    // Validate from logs that threads are disabled.
+    String content = logs.getOutput();
+    assertInLog(content,
+        "Rejected execution of thread for Rename operation on blob");
+    assertInLog(content, "Serializing the Rename operation");
+  }
+
+  /*
+   * Test case for rename operation with multiple threads and flat listing enabled.
+   */
+  @Test
+  public void testRenameThreadPoolExecuteSingleThreadFailure() throws Exception {
+
+    // Spy azure file system object and return mocked thread pool
+    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
+
+    // Spy a thread pool executor and link it to azure file system object.
+    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root")));
+    AzureFileSystemThreadPoolExecutor mockThreadPoolExecutor = Mockito.spy(
+        mockFs.getThreadPoolExecutor(renameThreads, "AzureBlobRenameThread", "Rename",
+            path, NativeAzureFileSystem.AZURE_RENAME_THREADS));
+
+    // With single iteration, we would have created 7 blobs resulting 7 threads.
+    Mockito.when(mockFs.getThreadPoolExecutor(renameThreads, "AzureBlobRenameThread", "Rename",
+        path, NativeAzureFileSystem.AZURE_RENAME_THREADS)).thenReturn(mockThreadPoolExecutor);
+
+    // Create a thread executor and link it to mocked thread pool executor object.
+    ThreadPoolExecutor mockThreadExecutor = Mockito.spy(mockThreadPoolExecutor.getThreadPool(7));
+    Mockito.when(mockThreadPoolExecutor.getThreadPool(7)).thenReturn(mockThreadExecutor);
+
+    // Mock thread executor to throw exception for all requests.
+    Mockito.doCallRealMethod().doThrow(new RejectedExecutionException()).when(mockThreadExecutor).execute(Mockito.any(Runnable.class));
+
+    validateRenameFolder(mockFs, "root", "rootnew");
+
+    // Validate from logs that threads are enabled and unused threads exists.
+    String content = logs.getOutput();
+    assertInLog(content,
+        "Using thread pool for Rename operation with threads 7");
+    assertInLog(content,
+        "6 threads not used for Rename operation on blob");
+  }
+
+  /*
+   * Test case for rename operation with multiple threads and flat listing enabled.
+   */
+  @Test
+  public void testRenameThreadPoolTerminationFailure() throws Exception {
+
+    // Spy azure file system object and return mocked thread pool
+    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
+
+    // Spy a thread pool executor and link it to azure file system object.
+    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root")));
+    AzureFileSystemThreadPoolExecutor mockThreadPoolExecutor = Mockito.spy(
+        mockFs.getThreadPoolExecutor(renameThreads, "AzureBlobRenameThread", "Rename",
+            path, NativeAzureFileSystem.AZURE_RENAME_THREADS));
+
+    // With single iteration, we would have created 7 blobs resulting 7 threads.
+    Mockito.when(mockFs.getThreadPoolExecutor(renameThreads, "AzureBlobRenameThread", "Rename",
+        path, NativeAzureFileSystem.AZURE_RENAME_THREADS)).thenReturn(mockThreadPoolExecutor);
+
+    // Mock thread executor to throw exception for all requests.
+    ThreadPoolExecutor mockThreadExecutor = Mockito.mock(ThreadPoolExecutor.class);
+    Mockito.doNothing().when(mockThreadExecutor).execute(Mockito.any(Runnable.class));
+    Mockito.when(mockThreadExecutor.awaitTermination(Long.MAX_VALUE, TimeUnit.DAYS)).thenThrow(new InterruptedException());
+    Mockito.when(mockThreadPoolExecutor.getThreadPool(7)).thenReturn(mockThreadExecutor);
+
+
+    createFolder(mockFs, "root");
+    Path sourceFolder = new Path("root");
+    Path destFolder = new Path("rootnew");
+    boolean exception = false;
+    try {
+      mockFs.rename(sourceFolder, destFolder);
+    } catch (IOException e){
+      exception = true;
+    }
+
+    assertTrue(exception);
+    assertTrue(mockFs.exists(sourceFolder));
+
+    // Validate from logs that threads are enabled and rename operation is failed.
+    String content = logs.getOutput();
+    assertInLog(content,
+        "Using thread pool for Rename operation with threads");
+    assertInLog(content, "Threads got interrupted Rename blob operation");
+    assertInLog(content,
+        "Rename failed as operation on subfolders and files failed.");
+  }
+
+  /*
+   * Test case for rename operation with multiple threads and flat listing enabled.
+   */
+  @Test
+  public void testRenameSingleRenameException() throws Exception {
+
+    // Spy azure file system object and raise exception for deleting one file
+    Path sourceFolder = new Path("root");
+    Path destFolder = new Path("rootnew");
+
+    // Spy azure file system object and populate rename pending spy object.
+    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
+
+    // Populate data now only such that rename pending spy object would see this data.
+    createFolder(mockFs, "root");
+
+    String srcKey = mockFs.pathToKey(mockFs.makeAbsolute(sourceFolder));
+    String dstKey = mockFs.pathToKey(mockFs.makeAbsolute(destFolder));
+
+    FolderRenamePending mockRenameFs = Mockito.spy(mockFs.prepareAtomicFolderRename(srcKey, dstKey));
+    Mockito.when(mockFs.prepareAtomicFolderRename(srcKey, dstKey)).thenReturn(mockRenameFs);
+    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root/0")));
+    Mockito.doThrow(new IOException()).when(mockRenameFs).renameFile(Mockito.any(FileMetadata.class));
+
+    boolean exception = false;
+    try {
+      mockFs.rename(sourceFolder, destFolder);
+    } catch (IOException e){
+      exception = true;
+    }
+
+    assertTrue(exception);
+    assertTrue(mockFs.exists(sourceFolder));
+
+    // Validate from logs that threads are enabled and delete operation failed.
+    String content = logs.getOutput();
+    assertInLog(content,
+        "Using thread pool for Rename operation with threads");
+    assertInLog(content,
+        "Encountered Exception for Rename operation for file " + path);
+    assertInLog(content,
+        "Terminating execution of Rename operation now as some other thread already got exception or operation failed");
+  }
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFSAuthWithBlobSpecificKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFSAuthWithBlobSpecificKeys.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFSAuthWithBlobSpecificKeys.java
new file mode 100644
index 0000000..d7e4831
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFSAuthWithBlobSpecificKeys.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure;
+
+import org.apache.hadoop.conf.Configuration;
+
+import static org.apache.hadoop.fs.azure.SecureStorageInterfaceImpl.KEY_USE_CONTAINER_SASKEY_FOR_ALL_ACCESS;
+
+/**
+ * Test class to hold all WASB authorization tests that use blob-specific keys
+ * to access storage.
+ */
+public class ITestNativeAzureFSAuthWithBlobSpecificKeys
+    extends ITestNativeAzureFileSystemAuthorizationWithOwner {
+
+
+  @Override
+  public Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    conf.set(KEY_USE_CONTAINER_SASKEY_FOR_ALL_ACCESS, "false");
+    return conf;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFSAuthorizationCaching.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFSAuthorizationCaching.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFSAuthorizationCaching.java
new file mode 100644
index 0000000..c73b1cc
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFSAuthorizationCaching.java
@@ -0,0 +1,53 @@
+/**
+ * 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.azure;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+import static org.apache.hadoop.fs.azure.CachingAuthorizer.KEY_AUTH_SERVICE_CACHING_ENABLE;
+
+/**
+ * Test class to hold all WASB authorization caching related tests.
+ */
+public class ITestNativeAzureFSAuthorizationCaching
+    extends ITestNativeAzureFileSystemAuthorizationWithOwner {
+
+  private static final int DUMMY_TTL_VALUE = 5000;
+
+  @Override
+  public Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    conf.set(KEY_AUTH_SERVICE_CACHING_ENABLE, "true");
+    return conf;
+  }
+
+  /**
+   * Test to verify cache behavior -- assert that PUT overwrites value if present
+   */
+  @Test
+  public void testCachePut() throws Throwable {
+    CachingAuthorizer<String, Integer> cache = new CachingAuthorizer<>(DUMMY_TTL_VALUE, "TEST");
+    cache.init(createConfiguration());
+    cache.put("TEST", 1);
+    cache.put("TEST", 3);
+    int result = cache.get("TEST");
+    assertEquals("Cache returned unexpected result", 3, result);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFSPageBlobLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFSPageBlobLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFSPageBlobLive.java
new file mode 100644
index 0000000..a4d8729
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFSPageBlobLive.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.azure;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Run the base Azure file system tests strictly on page blobs to make sure fundamental
+ * operations on page blob files and folders work as expected.
+ * These operations include create, delete, rename, list, and so on.
+ */
+public class ITestNativeAzureFSPageBlobLive extends
+    NativeAzureFileSystemBaseTest {
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount()
+      throws Exception {
+    Configuration conf = new Configuration();
+
+    // Configure the page blob directories key so every file created is a page blob.
+    conf.set(AzureNativeFileSystemStore.KEY_PAGE_BLOB_DIRECTORIES, "/");
+
+    // Configure the atomic rename directories key so every folder will have
+    // atomic rename applied.
+    conf.set(AzureNativeFileSystemStore.KEY_ATOMIC_RENAME_DIRECTORIES, "/");
+    return AzureBlobStorageTestAccount.create(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemAppend.java
new file mode 100644
index 0000000..29611bf
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemAppend.java
@@ -0,0 +1,350 @@
+/**
+ * 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.azure;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Arrays;
+
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.test.GenericTestUtils;
+
+import org.junit.Test;
+
+/**
+ * Test append operations.
+ */
+public class ITestNativeAzureFileSystemAppend extends AbstractWasbTestBase {
+
+  private Path testPath;
+
+  @Override
+  public Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    conf.setBoolean(NativeAzureFileSystem.APPEND_SUPPORT_ENABLE_PROPERTY_NAME,
+        true);
+    return conf;
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    testPath = methodPath();
+  }
+
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create(createConfiguration());
+  }
+
+  /*
+   * Helper method that creates test data of size provided by the
+   * "size" parameter.
+   */
+  private static byte[] getTestData(int size) {
+    byte[] testData = new byte[size];
+    System.arraycopy(RandomStringUtils.randomAlphabetic(size).getBytes(), 0, testData, 0, size);
+    return testData;
+  }
+
+  // Helper method to create file and write fileSize bytes of data on it.
+  private byte[] createBaseFileWithData(int fileSize, Path testPath) throws Throwable {
+
+    try(FSDataOutputStream createStream = fs.create(testPath)) {
+      byte[] fileData = null;
+
+      if (fileSize != 0) {
+        fileData = getTestData(fileSize);
+        createStream.write(fileData);
+      }
+      return fileData;
+    }
+  }
+
+  /*
+   * Helper method to verify a file data equal to "dataLength" parameter
+   */
+  private boolean verifyFileData(int dataLength, byte[] testData, int testDataIndex,
+      FSDataInputStream srcStream) {
+
+    try {
+
+      byte[] fileBuffer = new byte[dataLength];
+      byte[] testDataBuffer = new byte[dataLength];
+
+      int fileBytesRead = srcStream.read(fileBuffer);
+
+      if (fileBytesRead < dataLength) {
+        return false;
+      }
+
+      System.arraycopy(testData, testDataIndex, testDataBuffer, 0, dataLength);
+
+      if (!Arrays.equals(fileBuffer, testDataBuffer)) {
+        return false;
+      }
+
+      return true;
+
+    } catch (Exception ex) {
+      return false;
+    }
+
+  }
+
+  /*
+   * Helper method to verify Append on a testFile.
+   */
+  private boolean verifyAppend(byte[] testData, Path testFile) {
+
+    try(FSDataInputStream srcStream = fs.open(testFile)) {
+
+      int baseBufferSize = 2048;
+      int testDataSize = testData.length;
+      int testDataIndex = 0;
+
+      while (testDataSize > baseBufferSize) {
+
+        if (!verifyFileData(baseBufferSize, testData, testDataIndex, srcStream)) {
+          return false;
+        }
+        testDataIndex += baseBufferSize;
+        testDataSize -= baseBufferSize;
+      }
+
+      if (!verifyFileData(testDataSize, testData, testDataIndex, srcStream)) {
+        return false;
+      }
+
+      return true;
+    } catch(Exception ex) {
+      return false;
+    }
+  }
+
+  /*
+   * Test case to verify if an append on small size data works. This tests
+   * append E2E
+   */
+  @Test
+  public void testSingleAppend() throws Throwable{
+
+    FSDataOutputStream appendStream = null;
+    try {
+      int baseDataSize = 50;
+      byte[] baseDataBuffer = createBaseFileWithData(baseDataSize, testPath);
+
+      int appendDataSize = 20;
+      byte[] appendDataBuffer = getTestData(appendDataSize);
+      appendStream = fs.append(testPath, 10);
+      appendStream.write(appendDataBuffer);
+      appendStream.close();
+      byte[] testData = new byte[baseDataSize + appendDataSize];
+      System.arraycopy(baseDataBuffer, 0, testData, 0, baseDataSize);
+      System.arraycopy(appendDataBuffer, 0, testData, baseDataSize, appendDataSize);
+
+      assertTrue(verifyAppend(testData, testPath));
+    } finally {
+      if (appendStream != null) {
+        appendStream.close();
+      }
+    }
+  }
+
+  /*
+   * Test case to verify append to an empty file.
+   */
+  @Test
+  public void testSingleAppendOnEmptyFile() throws Throwable {
+
+    FSDataOutputStream appendStream = null;
+
+    try {
+      createBaseFileWithData(0, testPath);
+
+      int appendDataSize = 20;
+      byte[] appendDataBuffer = getTestData(appendDataSize);
+      appendStream = fs.append(testPath, 10);
+      appendStream.write(appendDataBuffer);
+      appendStream.close();
+
+      assertTrue(verifyAppend(appendDataBuffer, testPath));
+    } finally {
+      if (appendStream != null) {
+        appendStream.close();
+      }
+    }
+  }
+
+  /*
+   * Test to verify that we can open only one Append stream on a File.
+   */
+  @Test
+  public void testSingleAppenderScenario() throws Throwable {
+
+    FSDataOutputStream appendStream1 = null;
+    FSDataOutputStream appendStream2 = null;
+    IOException ioe = null;
+    try {
+      createBaseFileWithData(0, testPath);
+      appendStream1 = fs.append(testPath, 10);
+      boolean encounteredException = false;
+      try {
+        appendStream2 = fs.append(testPath, 10);
+      } catch(IOException ex) {
+        encounteredException = true;
+        ioe = ex;
+      }
+
+      appendStream1.close();
+
+      assertTrue(encounteredException);
+      GenericTestUtils.assertExceptionContains("Unable to set Append lease on the Blob", ioe);
+    } finally {
+      if (appendStream1 != null) {
+        appendStream1.close();
+      }
+
+      if (appendStream2 != null) {
+        appendStream2.close();
+      }
+    }
+  }
+
+  /*
+   * Tests to verify multiple appends on a Blob.
+   */
+  @Test
+  public void testMultipleAppends() throws Throwable {
+
+    int baseDataSize = 50;
+    byte[] baseDataBuffer = createBaseFileWithData(baseDataSize, testPath);
+
+    int appendDataSize = 100;
+    int targetAppendCount = 50;
+    byte[] testData = new byte[baseDataSize + (appendDataSize*targetAppendCount)];
+    int testDataIndex = 0;
+    System.arraycopy(baseDataBuffer, 0, testData, testDataIndex, baseDataSize);
+    testDataIndex += baseDataSize;
+
+    int appendCount = 0;
+
+    FSDataOutputStream appendStream = null;
+
+    try {
+      while (appendCount < targetAppendCount) {
+
+        byte[] appendDataBuffer = getTestData(appendDataSize);
+        appendStream = fs.append(testPath, 30);
+        appendStream.write(appendDataBuffer);
+        appendStream.close();
+
+        System.arraycopy(appendDataBuffer, 0, testData, testDataIndex, appendDataSize);
+        testDataIndex += appendDataSize;
+        appendCount++;
+      }
+
+      assertTrue(verifyAppend(testData, testPath));
+
+    } finally {
+      if (appendStream != null) {
+        appendStream.close();
+      }
+    }
+  }
+
+  /*
+   * Test to verify we multiple appends on the same stream.
+   */
+  @Test
+  public void testMultipleAppendsOnSameStream() throws Throwable {
+
+    int baseDataSize = 50;
+    byte[] baseDataBuffer = createBaseFileWithData(baseDataSize, testPath);
+    int appendDataSize = 100;
+    int targetAppendCount = 50;
+    byte[] testData = new byte[baseDataSize + (appendDataSize*targetAppendCount)];
+    int testDataIndex = 0;
+    System.arraycopy(baseDataBuffer, 0, testData, testDataIndex, baseDataSize);
+    testDataIndex += baseDataSize;
+    int appendCount = 0;
+
+    FSDataOutputStream appendStream = null;
+
+    try {
+
+      while (appendCount < targetAppendCount) {
+
+        appendStream = fs.append(testPath, 50);
+
+        int singleAppendChunkSize = 20;
+        int appendRunSize = 0;
+        while (appendRunSize < appendDataSize) {
+
+          byte[] appendDataBuffer = getTestData(singleAppendChunkSize);
+          appendStream.write(appendDataBuffer);
+          System.arraycopy(appendDataBuffer, 0, testData,
+              testDataIndex + appendRunSize, singleAppendChunkSize);
+
+          appendRunSize += singleAppendChunkSize;
+        }
+
+        appendStream.close();
+        testDataIndex += appendDataSize;
+        appendCount++;
+      }
+
+      assertTrue(verifyAppend(testData, testPath));
+    } finally {
+      if (appendStream != null) {
+        appendStream.close();
+      }
+    }
+  }
+
+  @Test(expected=UnsupportedOperationException.class)
+  /*
+   * Test to verify the behavior when Append Support configuration flag is set to false
+   */
+  public void testFalseConfigurationFlagBehavior() throws Throwable {
+
+    fs = testAccount.getFileSystem();
+    Configuration conf = fs.getConf();
+    conf.setBoolean(NativeAzureFileSystem.APPEND_SUPPORT_ENABLE_PROPERTY_NAME, false);
+    URI uri = fs.getUri();
+    fs.initialize(uri, conf);
+
+    FSDataOutputStream appendStream = null;
+
+    try {
+      createBaseFileWithData(0, testPath);
+      appendStream = fs.append(testPath, 10);
+    } finally {
+      if (appendStream != null) {
+        appendStream.close();
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemAtomicRenameDirList.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemAtomicRenameDirList.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemAtomicRenameDirList.java
new file mode 100644
index 0000000..869a31c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemAtomicRenameDirList.java
@@ -0,0 +1,55 @@
+/**
+ * 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.azure;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.junit.Test;
+
+/**
+ * Test atomic renaming.
+ */
+public class ITestNativeAzureFileSystemAtomicRenameDirList
+    extends AbstractWasbTestBase {
+
+  // HBase-site config controlling HBase root dir
+  private static final String HBASE_ROOT_DIR_CONF_STRING = "hbase.rootdir";
+  private static final String HBASE_ROOT_DIR_VALUE_ON_DIFFERENT_FS =
+      "wasb://somedifferentfilesystem.blob.core.windows.net/hbase";
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create();
+  }
+
+  @Test
+  public void testAtomicRenameKeyDoesntNPEOnInitializingWithNonDefaultURI()
+      throws IOException {
+    NativeAzureFileSystem azureFs = fs;
+    AzureNativeFileSystemStore azureStore = azureFs.getStore();
+    Configuration conf = fs.getConf();
+    conf.set(HBASE_ROOT_DIR_CONF_STRING, HBASE_ROOT_DIR_VALUE_ON_DIFFERENT_FS);
+    URI uri = fs.getUri();
+    fs.initialize(uri, conf);
+    azureStore.isAtomicRenameKey("anyrandomkey");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemAuthorizationWithOwner.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemAuthorizationWithOwner.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemAuthorizationWithOwner.java
new file mode 100644
index 0000000..3ec42f0
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemAuthorizationWithOwner.java
@@ -0,0 +1,122 @@
+/**
+ * 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.azure;
+
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import java.security.PrivilegedExceptionAction;
+
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test class that runs wasb authorization tests with owner check enabled.
+ */
+public class ITestNativeAzureFileSystemAuthorizationWithOwner
+  extends TestNativeAzureFileSystemAuthorization {
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    authorizer.init(fs.getConf(), true);
+  }
+
+  /**
+   * Test case when owner matches current user.
+   */
+  @Test
+  public void testOwnerPermissionPositive() throws Throwable {
+
+    Path parentDir = new Path("/testOwnerPermissionPositive");
+    Path testPath = new Path(parentDir, "test.data");
+
+    authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
+    authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.READ.toString(), true);
+    authorizer.addAuthRule(parentDir.toString(), WasbAuthorizationOperations.WRITE.toString(), true);
+    // additional rule used for assertPathExists
+    authorizer.addAuthRule(parentDir.toString(), WasbAuthorizationOperations.READ.toString(), true);
+    fs.updateWasbAuthorizer(authorizer);
+
+    try {
+      // creates parentDir with owner as current user
+      fs.mkdirs(parentDir);
+      ContractTestUtils.assertPathExists(fs, "parentDir does not exist", parentDir);
+
+      fs.create(testPath);
+      fs.getFileStatus(testPath);
+      ContractTestUtils.assertPathExists(fs, "testPath does not exist", testPath);
+
+    } finally {
+      allowRecursiveDelete(fs, parentDir.toString());
+      fs.delete(parentDir, true);
+    }
+  }
+
+  /**
+   * Negative test case for owner does not match current user.
+   */
+  @Test
+  public void testOwnerPermissionNegative() throws Throwable {
+    expectedEx.expect(WasbAuthorizationException.class);
+
+    Path parentDir = new Path("/testOwnerPermissionNegative");
+    Path childDir = new Path(parentDir, "childDir");
+
+    setExpectedFailureMessage("mkdirs", childDir);
+
+    authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
+    authorizer.addAuthRule(parentDir.toString(), WasbAuthorizationOperations.WRITE.toString(), true);
+
+    fs.updateWasbAuthorizer(authorizer);
+
+    try{
+      fs.mkdirs(parentDir);
+      UserGroupInformation ugiSuperUser = UserGroupInformation.createUserForTesting(
+          "testuser", new String[] {});
+
+      ugiSuperUser.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+          fs.mkdirs(childDir);
+          return null;
+        }
+      });
+
+    } finally {
+       allowRecursiveDelete(fs, parentDir.toString());
+       fs.delete(parentDir, true);
+    }
+  }
+
+  /**
+   * Test to verify that retrieving owner information does not
+   * throw when file/folder does not exist.
+   */
+  @Test
+  public void testRetrievingOwnerDoesNotFailWhenFileDoesNotExist() throws Throwable {
+
+    Path testdirectory = new Path("/testDirectory123454565");
+
+    String owner = fs.getOwnerForPath(testdirectory);
+    assertEquals("", owner);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemClientLogging.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemClientLogging.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemClientLogging.java
new file mode 100644
index 0000000..f73a763
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemClientLogging.java
@@ -0,0 +1,136 @@
+/**
+ * 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.azure;
+
+import java.net.URI;
+import java.util.StringTokenizer;
+
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
+import org.apache.log4j.Logger;
+import org.junit.Test;
+
+/**
+ * Test to validate Azure storage client side logging. Tests works only when
+ * testing with Live Azure storage because Emulator does not have support for
+ * client-side logging.
+ *
+ * <I>Important: </I> Do not attempt to move off commons-logging.
+ * The tests will fail.
+ */
+public class ITestNativeAzureFileSystemClientLogging
+    extends AbstractWasbTestBase {
+
+  // Core-site config controlling Azure Storage Client logging
+  private static final String KEY_LOGGING_CONF_STRING = "fs.azure.storage.client.logging";
+
+  // Temporary directory created using WASB.
+  private static final String TEMP_DIR = "tempDir";
+
+  /*
+   * Helper method to verify the client logging is working. This check primarily
+   * checks to make sure we see a line in the logs corresponding to the entity
+   * that is created during test run.
+   */
+  private boolean verifyStorageClientLogs(String capturedLogs, String entity)
+      throws Exception {
+
+    URI uri = testAccount.getRealAccount().getBlobEndpoint();
+    String container = testAccount.getRealContainer().getName();
+    String validateString = uri + Path.SEPARATOR + container + Path.SEPARATOR
+        + entity;
+    boolean entityFound = false;
+
+    StringTokenizer tokenizer = new StringTokenizer(capturedLogs, "\n");
+
+    while (tokenizer.hasMoreTokens()) {
+      String token = tokenizer.nextToken();
+      if (token.contains(validateString)) {
+        entityFound = true;
+        break;
+      }
+    }
+    return entityFound;
+  }
+
+  /*
+   * Helper method that updates the core-site config to enable/disable logging.
+   */
+  private void updateFileSystemConfiguration(Boolean loggingFlag)
+      throws Exception {
+
+    Configuration conf = fs.getConf();
+    conf.set(KEY_LOGGING_CONF_STRING, loggingFlag.toString());
+    URI uri = fs.getUri();
+    fs.initialize(uri, conf);
+  }
+
+  // Using WASB code to communicate with Azure Storage.
+  private void performWASBOperations() throws Exception {
+
+    Path tempDir = new Path(Path.SEPARATOR + TEMP_DIR);
+    fs.mkdirs(tempDir);
+    fs.delete(tempDir, true);
+  }
+
+  @Test
+  public void testLoggingEnabled() throws Exception {
+
+    LogCapturer logs = LogCapturer.captureLogs(new Log4JLogger(Logger
+        .getRootLogger()));
+
+    // Update configuration based on the Test.
+    updateFileSystemConfiguration(true);
+
+    performWASBOperations();
+
+    String output = getLogOutput(logs);
+    assertTrue("Log entry " + TEMP_DIR + " not found  in " + output,
+        verifyStorageClientLogs(output, TEMP_DIR));
+  }
+
+  protected String getLogOutput(LogCapturer logs) {
+    String output = logs.getOutput();
+    assertTrue("No log created/captured", !output.isEmpty());
+    return output;
+  }
+
+  @Test
+  public void testLoggingDisabled() throws Exception {
+
+    LogCapturer logs = LogCapturer.captureLogs(new Log4JLogger(Logger
+        .getRootLogger()));
+
+    // Update configuration based on the Test.
+    updateFileSystemConfiguration(false);
+
+    performWASBOperations();
+    String output = getLogOutput(logs);
+
+    assertFalse("Log entry " + TEMP_DIR + " found  in " + output,
+        verifyStorageClientLogs(output, TEMP_DIR));
+  }
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemConcurrencyLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemConcurrencyLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemConcurrencyLive.java
new file mode 100644
index 0000000..87cac15
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemConcurrencyLive.java
@@ -0,0 +1,185 @@
+/**
+ * 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.azure;
+
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+/***
+ * Test class to hold all Live Azure storage concurrency tests.
+ */
+public class ITestNativeAzureFileSystemConcurrencyLive
+    extends AbstractWasbTestBase {
+
+  private static final int THREAD_COUNT = 102;
+  private static final int TEST_EXECUTION_TIMEOUT = 5000;
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create();
+  }
+
+  /**
+   * Validate contract for FileSystem.create when overwrite is true and there
+   * are concurrent callers of FileSystem.delete.  An existing file should be
+   * overwritten, even if the original destination exists but is deleted by an
+   * external agent during the create operation.
+   */
+  @Test(timeout = TEST_EXECUTION_TIMEOUT)
+  public void testConcurrentCreateDeleteFile() throws Exception {
+    Path testFile = methodPath();
+
+    List<CreateFileTask> tasks = new ArrayList<>(THREAD_COUNT);
+
+    for (int i = 0; i < THREAD_COUNT; i++) {
+      tasks.add(new CreateFileTask(fs, testFile));
+    }
+
+    ExecutorService es = null;
+
+    try {
+      es = Executors.newFixedThreadPool(THREAD_COUNT);
+
+      List<Future<Void>> futures = es.invokeAll(tasks);
+
+      for (Future<Void> future : futures) {
+        Assert.assertTrue(future.isDone());
+
+        // we are using Callable<V>, so if an exception
+        // occurred during the operation, it will be thrown
+        // when we call get
+        Assert.assertEquals(null, future.get());
+      }
+    } finally {
+      if (es != null) {
+        es.shutdownNow();
+      }
+    }
+  }
+
+  /**
+   * Validate contract for FileSystem.delete when invoked concurrently.
+   * One of the threads should successfully delete the file and return true;
+   * all other threads should return false.
+   */
+  @Test(timeout = TEST_EXECUTION_TIMEOUT)
+  public void testConcurrentDeleteFile() throws Exception {
+    Path testFile = new Path("test.dat");
+    fs.create(testFile).close();
+
+    List<DeleteFileTask> tasks = new ArrayList<>(THREAD_COUNT);
+
+    for (int i = 0; i < THREAD_COUNT; i++) {
+      tasks.add(new DeleteFileTask(fs, testFile));
+    }
+
+    ExecutorService es = null;
+    try {
+      es = Executors.newFixedThreadPool(THREAD_COUNT);
+
+      List<Future<Boolean>> futures = es.invokeAll(tasks);
+
+      int successCount = 0;
+      for (Future<Boolean> future : futures) {
+        Assert.assertTrue(future.isDone());
+
+        // we are using Callable<V>, so if an exception
+        // occurred during the operation, it will be thrown
+        // when we call get
+        Boolean success = future.get();
+        if (success) {
+          successCount++;
+        }
+      }
+
+      Assert.assertEquals(
+          "Exactly one delete operation should return true.",
+          1,
+          successCount);
+    } finally {
+      if (es != null) {
+        es.shutdownNow();
+      }
+    }
+  }
+
+  abstract class FileSystemTask<V> implements Callable<V> {
+    private final FileSystem fileSystem;
+    private final Path path;
+
+    protected FileSystem getFileSystem() {
+      return this.fileSystem;
+    }
+
+    protected Path getFilePath() {
+      return this.path;
+    }
+
+    FileSystemTask(FileSystem fs, Path p) {
+      this.fileSystem = fs;
+      this.path = p;
+    }
+
+    public abstract V call() throws Exception;
+  }
+
+  class DeleteFileTask extends FileSystemTask<Boolean> {
+
+    DeleteFileTask(FileSystem fs, Path p) {
+      super(fs, p);
+    }
+
+    @Override
+    public Boolean call() throws Exception {
+      return this.getFileSystem().delete(this.getFilePath(), false);
+    }
+  }
+
+  class CreateFileTask extends FileSystemTask<Void> {
+    CreateFileTask(FileSystem fs, Path p) {
+      super(fs, p);
+    }
+
+    public Void call() throws Exception {
+      FileSystem fs = getFileSystem();
+      Path p = getFilePath();
+
+      // Create an empty file and close the stream.
+      FSDataOutputStream stream = fs.create(p, true);
+      stream.close();
+
+      // Delete the file.  We don't care if delete returns true or false.
+      // We just want to ensure the file does not exist.
+      this.getFileSystem().delete(this.getFilePath(), false);
+
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemContractEmulator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemContractEmulator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemContractEmulator.java
new file mode 100644
index 0000000..4836fc4
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemContractEmulator.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.azure;
+
+import static org.junit.Assume.assumeNotNull;
+
+import org.apache.hadoop.fs.FileSystemContractBaseTest;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azure.integration.AzureTestUtils;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+
+/**
+ * Run the {@code FileSystemContractBaseTest} tests against the emulator
+ */
+public class ITestNativeAzureFileSystemContractEmulator extends
+    FileSystemContractBaseTest {
+  private AzureBlobStorageTestAccount testAccount;
+  private Path basePath;
+
+  @Rule
+  public TestName methodName = new TestName();
+
+  private void nameThread() {
+    Thread.currentThread().setName("JUnit-" + methodName.getMethodName());
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    nameThread();
+    testAccount = AzureBlobStorageTestAccount.createForEmulator();
+    if (testAccount != null) {
+      fs = testAccount.getFileSystem();
+    }
+    assumeNotNull(fs);
+    basePath = fs.makeQualified(
+        AzureTestUtils.createTestPath(
+            new Path("ITestNativeAzureFileSystemContractEmulator")));
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    super.tearDown();
+    testAccount = AzureTestUtils.cleanup(testAccount);
+    fs = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemContractLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemContractLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemContractLive.java
new file mode 100644
index 0000000..d3d1bd8
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemContractLive.java
@@ -0,0 +1,108 @@
+/**
+ * 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.azure;
+
+import static org.junit.Assume.assumeNotNull;
+
+import org.apache.hadoop.fs.FileSystemContractBaseTest;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azure.integration.AzureTestConstants;
+import org.apache.hadoop.fs.azure.integration.AzureTestUtils;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * Run the {@link FileSystemContractBaseTest} test suite against azure storage.
+ */
+public class ITestNativeAzureFileSystemContractLive extends
+    FileSystemContractBaseTest {
+  private AzureBlobStorageTestAccount testAccount;
+  private Path basePath;
+
+  @Rule
+  public TestName methodName = new TestName();
+
+  private void nameThread() {
+    Thread.currentThread().setName("JUnit-" + methodName.getMethodName());
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    nameThread();
+    testAccount = AzureBlobStorageTestAccount.create();
+    if (testAccount != null) {
+      fs = testAccount.getFileSystem();
+    }
+    assumeNotNull(fs);
+    basePath = fs.makeQualified(
+        AzureTestUtils.createTestPath(
+            new Path("NativeAzureFileSystemContractLive")));
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    super.tearDown();
+    testAccount = AzureTestUtils.cleanup(testAccount);
+    fs = null;
+  }
+
+  @Override
+  public Path getTestBaseDir() {
+    return basePath;
+  }
+
+  protected int getGlobalTimeout() {
+    return AzureTestConstants.AZURE_TEST_TIMEOUT;
+  }
+
+  /**
+   * The following tests are failing on Azure and the Azure 
+   * file system code needs to be modified to make them pass.
+   * A separate work item has been opened for this.
+   */
+  @Ignore
+  @Test
+  public void testMoveFileUnderParent() throws Throwable {
+  }
+
+  @Ignore
+  @Test
+  public void testRenameFileToSelf() throws Throwable {
+  }
+
+  @Ignore
+  @Test
+  public void testRenameChildDirForbidden() throws Exception {
+  }
+
+  @Ignore
+  @Test
+  public void testMoveDirUnderParent() throws Throwable {
+  }
+
+  @Ignore
+  @Test
+  public void testRenameDirToSelf() throws Throwable {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemContractPageBlobLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemContractPageBlobLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemContractPageBlobLive.java
new file mode 100644
index 0000000..03e90aa
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemContractPageBlobLive.java
@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystemContractBaseTest;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azure.integration.AzureTestConstants;
+import org.apache.hadoop.fs.azure.integration.AzureTestUtils;
+
+import static org.junit.Assume.assumeNotNull;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * Run the {@link FileSystemContractBaseTest} test suite against azure
+ * storage, after switching the FS using page blobs everywhere.
+ */
+public class ITestNativeAzureFileSystemContractPageBlobLive extends
+    FileSystemContractBaseTest {
+  private AzureBlobStorageTestAccount testAccount;
+  private Path basePath;
+  @Rule
+  public TestName methodName = new TestName();
+
+  private void nameThread() {
+    Thread.currentThread().setName("JUnit-" + methodName.getMethodName());
+  }
+
+  private AzureBlobStorageTestAccount createTestAccount()
+      throws Exception {
+    Configuration conf = new Configuration();
+
+    // Configure the page blob directories key so every file created is a page blob.
+    conf.set(AzureNativeFileSystemStore.KEY_PAGE_BLOB_DIRECTORIES, "/");
+
+    // Configure the atomic rename directories key so every folder will have
+    // atomic rename applied.
+    conf.set(AzureNativeFileSystemStore.KEY_ATOMIC_RENAME_DIRECTORIES, "/");
+    return AzureBlobStorageTestAccount.create(conf);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    testAccount = createTestAccount();
+    assumeNotNull(testAccount);
+    fs = testAccount.getFileSystem();
+    basePath = AzureTestUtils.pathForTests(fs, "filesystemcontractpageblob");
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    testAccount = AzureTestUtils.cleanup(testAccount);
+    fs = null;
+  }
+
+  protected int getGlobalTimeout() {
+    return AzureTestConstants.AZURE_TEST_TIMEOUT;
+  }
+
+  @Override
+  public Path getTestBaseDir() {
+    return basePath;
+  }
+
+  /**
+   * The following tests are failing on Azure and the Azure 
+   * file system code needs to be modified to make them pass.
+   * A separate work item has been opened for this.
+   */
+  @Ignore
+  @Test
+  public void testMoveFileUnderParent() throws Throwable {
+  }
+
+  @Ignore
+  @Test
+  public void testRenameFileToSelf() throws Throwable {
+  }
+  
+  @Ignore
+  @Test
+  public void testRenameChildDirForbidden() throws Exception {
+  }
+  
+  @Ignore
+  @Test
+  public void testMoveDirUnderParent() throws Throwable {
+  }
+  
+  @Ignore
+  @Test
+  public void testRenameDirToSelf() throws Throwable {
+  }
+}


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


[11/50] [abbrv] hadoop git commit: Revert "HDFS-6874. Add GETFILEBLOCKLOCATIONS operation to HttpFS. Contributed by Weiwei Yang"

Posted by ae...@apache.org.
Revert "HDFS-6874. Add GETFILEBLOCKLOCATIONS operation to HttpFS.  Contributed by Weiwei Yang"

This reverts commit 931a49800ef05ee0a6fdc143be1799abb228735d.

 Conflicts:
	hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
	hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
	hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2f867115
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2f867115
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2f867115

Branch: refs/heads/HDFS-7240
Commit: 2f867115a819327a50cf9c8caa56e0e5a85cabb9
Parents: 4946716
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Sep 14 15:02:27 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Sep 14 15:02:27 2017 -0700

----------------------------------------------------------------------
 .../hadoop/fs/http/client/HttpFSFileSystem.java | 42 ---------
 .../hadoop/fs/http/server/FSOperations.java     | 38 ---------
 .../http/server/HttpFSParametersProvider.java   |  3 +-
 .../hadoop/fs/http/server/HttpFSServer.java     | 21 +----
 .../fs/http/client/BaseTestHttpFSWith.java      | 89 +-------------------
 5 files changed, 3 insertions(+), 190 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f867115/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
index 1059a02..b5880e9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
@@ -23,12 +23,9 @@ import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.type.MapType;
 import com.google.common.base.Charsets;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.DelegationTokenRenewer;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -122,8 +119,6 @@ public class HttpFSFileSystem extends FileSystem
   public static final String NEW_LENGTH_PARAM = "newlength";
   public static final String START_AFTER_PARAM = "startAfter";
   public static final String POLICY_NAME_PARAM = "storagepolicy";
-  public static final String OFFSET_PARAM = "offset";
-  public static final String LENGTH_PARAM = "length";
   public static final String SNAPSHOT_NAME_PARAM = "snapshotname";
   public static final String OLD_SNAPSHOT_NAME_PARAM = "oldsnapshotname";
 
@@ -210,7 +205,6 @@ public class HttpFSFileSystem extends FileSystem
 
   public static final String STORAGE_POLICIES_JSON = "BlockStoragePolicies";
   public static final String STORAGE_POLICY_JSON = "BlockStoragePolicy";
-  public static final String BLOCK_LOCATIONS_JSON = "BlockLocations";
 
   public static final int HTTP_TEMPORARY_REDIRECT = 307;
 
@@ -1359,42 +1353,6 @@ public class HttpFSFileSystem extends FileSystem
     return createStoragePolicy((JSONObject) json.get(STORAGE_POLICY_JSON));
   }
 
-  @Override
-  public BlockLocation[] getFileBlockLocations(FileStatus file, long start,
-      long len) throws IOException {
-    Map<String, String> params = new HashMap<String, String>();
-    params.put(OP_PARAM, Operation.GETFILEBLOCKLOCATIONS.toString());
-    params.put(OFFSET_PARAM, Long.toString(start));
-    params.put(LENGTH_PARAM, Long.toString(len));
-    HttpURLConnection conn =
-        getConnection(Operation.GETFILEBLOCKLOCATIONS.getMethod(), params,
-            file.getPath(), true);
-    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
-    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
-    return toBlockLocations(json);
-  }
-
-  private BlockLocation[] toBlockLocations(JSONObject json)
-      throws IOException {
-    ObjectMapper mapper = new ObjectMapper();
-    MapType subType = mapper.getTypeFactory().constructMapType(
-        Map.class,
-        String.class,
-        BlockLocation[].class);
-    MapType rootType = mapper.getTypeFactory().constructMapType(
-        Map.class,
-        mapper.constructType(String.class),
-        mapper.constructType(subType));
-
-    Map<String, Map<String, BlockLocation[]>> jsonMap = mapper
-        .readValue(json.toJSONString(), rootType);
-    Map<String, BlockLocation[]> locationMap = jsonMap
-        .get(BLOCK_LOCATIONS_JSON);
-    BlockLocation[] locationArray = locationMap.get(
-        BlockLocation.class.getSimpleName());
-    return locationArray;
-  }
-
   private BlockStoragePolicy createStoragePolicy(JSONObject policyJson)
       throws IOException {
     byte id = ((Number) policyJson.get("id")).byteValue();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f867115/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
index 4b5918a..a08bc54 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.fs.http.server;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockStoragePolicySpi;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileChecksum;
@@ -36,7 +35,6 @@ import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.web.JsonUtil;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.lib.service.FileSystemAccess;
 import org.apache.hadoop.util.StringUtils;
@@ -1459,41 +1457,6 @@ public class FSOperations {
   }
 
   /**
-   * Executor that performs a getFileBlockLocations FileSystemAccess
-   * file system operation.
-   */
-  @InterfaceAudience.Private
-  @SuppressWarnings("rawtypes")
-  public static class FSFileBlockLocations implements
-      FileSystemAccess.FileSystemExecutor<Map> {
-    private Path path;
-    private long offsetValue;
-    private long lengthValue;
-
-    /**
-     * Creates a file-block-locations executor.
-     *
-     * @param path the path to retrieve the location
-     * @param offsetValue offset into the given file
-     * @param lengthValue length for which to get locations for
-     */
-    public FSFileBlockLocations(String path, long offsetValue,
-        long lengthValue) {
-      this.path = new Path(path);
-      this.offsetValue = offsetValue;
-      this.lengthValue = lengthValue;
-    }
-
-    @Override
-    public Map execute(FileSystem fs) throws IOException {
-      BlockLocation[] locations =
-          fs.getFileBlockLocations(this.path, this.offsetValue,
-              this.lengthValue);
-      return JsonUtil.toJsonMap(locations);
-    }
-  }
-
-  /**
    *  Executor that performs a createSnapshot FileSystemAccess operation.
    */
   @InterfaceAudience.Private
@@ -1596,5 +1559,4 @@ public class FSOperations {
       return null;
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f867115/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
index 5f265c0..3e6a5ad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
@@ -58,8 +58,7 @@ public class HttpFSParametersProvider extends ParametersProvider {
     PARAMS_DEF.put(Operation.GETHOMEDIRECTORY, new Class[]{});
     PARAMS_DEF.put(Operation.GETCONTENTSUMMARY, new Class[]{});
     PARAMS_DEF.put(Operation.GETFILECHECKSUM, new Class[]{});
-    PARAMS_DEF.put(Operation.GETFILEBLOCKLOCATIONS,
-        new Class[] {OffsetParam.class, LenParam.class});
+    PARAMS_DEF.put(Operation.GETFILEBLOCKLOCATIONS, new Class[]{});
     PARAMS_DEF.put(Operation.GETACLSTATUS, new Class[]{});
     PARAMS_DEF.put(Operation.GETTRASHROOT, new Class[]{});
     PARAMS_DEF.put(Operation.INSTRUMENTATION, new Class[]{});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f867115/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
index 03ccb4c..bcc1182 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
@@ -51,7 +51,6 @@ import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrEncodingPa
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrNameParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrSetFlagParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.XAttrValueParam;
-import org.apache.hadoop.hdfs.web.JsonUtil;
 import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.lib.service.FileSystemAccess;
 import org.apache.hadoop.lib.service.FileSystemAccessException;
@@ -299,25 +298,7 @@ public class HttpFSServer {
       break;
     }
     case GETFILEBLOCKLOCATIONS: {
-      long offset = 0;
-      // In case length is not given, reset to max long
-      // in order to retrieve all file block locations
-      long len = Long.MAX_VALUE;
-      Long offsetParam = params.get(OffsetParam.NAME, OffsetParam.class);
-      Long lenParam = params.get(LenParam.NAME, LenParam.class);
-      AUDIT_LOG.info("[{}] offset [{}] len [{}]",
-          new Object[] {path, offsetParam, lenParam});
-      if (offsetParam != null && offsetParam.longValue() > 0) {
-        offset = offsetParam.longValue();
-      }
-      if (lenParam != null && lenParam.longValue() > 0) {
-        len = lenParam.longValue();
-      }
-      FSOperations.FSFileBlockLocations command =
-          new FSOperations.FSFileBlockLocations(path, offset, len);
-      @SuppressWarnings("rawtypes") Map locations = fsExecute(user, command);
-      final String json = JsonUtil.toJsonString("BlockLocations", locations);
-      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
+      response = Response.status(Response.Status.BAD_REQUEST).build();
       break;
     }
     case GETACLSTATUS: {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f867115/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
index 2cd8934..a6dce4d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.fs.http.client;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockStoragePolicySpi;
-import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileChecksum;
@@ -1041,7 +1040,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     WORKING_DIRECTORY, MKDIRS, SET_TIMES, SET_PERMISSION, SET_OWNER,
     SET_REPLICATION, CHECKSUM, CONTENT_SUMMARY, FILEACLS, DIRACLS, SET_XATTR,
     GET_XATTRS, REMOVE_XATTR, LIST_XATTRS, ENCRYPTION, LIST_STATUS_BATCH,
-    GETTRASHROOT, STORAGEPOLICY, ERASURE_CODING, GETFILEBLOCKLOCATIONS,
+    GETTRASHROOT, STORAGEPOLICY, ERASURE_CODING,
     CREATE_SNAPSHOT, RENAME_SNAPSHOT, DELETE_SNAPSHOT
   }
 
@@ -1131,9 +1130,6 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     case ERASURE_CODING:
       testErasureCoding();
       break;
-    case GETFILEBLOCKLOCATIONS:
-      testGetFileBlockLocations();
-      break;
     case CREATE_SNAPSHOT:
       testCreateSnapshot();
       break;
@@ -1189,88 +1185,6 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     });
   }
 
-  private void testGetFileBlockLocations() throws Exception {
-    BlockLocation[] locations1, locations2, locations11, locations21 = null;
-    Path testFile = null;
-
-    // Test single block file block locations.
-    try (FileSystem fs = FileSystem.get(getProxiedFSConf())) {
-      testFile = new Path(getProxiedFSTestDir(), "singleBlock.txt");
-      DFSTestUtil.createFile(fs, testFile, (long) 1, (short) 1, 0L);
-      locations1 = fs.getFileBlockLocations(testFile, 0, 1);
-      Assert.assertNotNull(locations1);
-    }
-
-    try (FileSystem fs = getHttpFSFileSystem()) {
-      locations2 = fs.getFileBlockLocations(testFile, 0, 1);
-      Assert.assertNotNull(locations2);
-    }
-
-    verifyBlockLocations(locations1, locations2);
-
-    // Test multi-block single replica file block locations.
-    try (FileSystem fs = FileSystem.get(getProxiedFSConf())) {
-      testFile = new Path(getProxiedFSTestDir(), "multipleBlocks.txt");
-      DFSTestUtil.createFile(fs, testFile, 512, (short) 2048,
-          (long) 512, (short) 1,  0L);
-      locations1 = fs.getFileBlockLocations(testFile, 0, 1024);
-      locations11 = fs.getFileBlockLocations(testFile, 1024, 2048);
-      Assert.assertNotNull(locations1);
-      Assert.assertNotNull(locations11);
-    }
-
-    try (FileSystem fs = getHttpFSFileSystem()) {
-      locations2 = fs.getFileBlockLocations(testFile, 0, 1024);
-      locations21 = fs.getFileBlockLocations(testFile, 1024, 2048);
-      Assert.assertNotNull(locations2);
-      Assert.assertNotNull(locations21);
-    }
-
-    verifyBlockLocations(locations1, locations2);
-    verifyBlockLocations(locations11, locations21);
-
-    // Test multi-block multi-replica file block locations.
-    try (FileSystem fs = FileSystem.get(getProxiedFSConf())) {
-      testFile = new Path(getProxiedFSTestDir(), "multipleBlocks.txt");
-      DFSTestUtil.createFile(fs, testFile, 512, (short) 2048,
-          (long) 512, (short) 3,  0L);
-      locations1 = fs.getFileBlockLocations(testFile, 0, 2048);
-      Assert.assertNotNull(locations1);
-    }
-
-    try (FileSystem fs = getHttpFSFileSystem()) {
-      locations2 = fs.getFileBlockLocations(testFile, 0, 2048);
-      Assert.assertNotNull(locations2);
-    }
-
-    verifyBlockLocations(locations1, locations2);
-  }
-
-  private void verifyBlockLocations(BlockLocation[] locations1,
-      BlockLocation[] locations2) throws IOException {
-    Assert.assertEquals(locations1.length, locations2.length);
-    for (int i = 0; i < locations1.length; i++) {
-      BlockLocation location1 = locations1[i];
-      BlockLocation location2 = locations2[i];
-
-      Assert.assertEquals(location1.isCorrupt(), location2.isCorrupt());
-      Assert.assertEquals(location1.getOffset(), location2.getOffset());
-      Assert.assertEquals(location1.getLength(), location2.getLength());
-
-      Arrays.sort(location1.getHosts());
-      Arrays.sort(location2.getHosts());
-      Arrays.sort(location1.getNames());
-      Arrays.sort(location2.getNames());
-      Arrays.sort(location1.getTopologyPaths());
-      Arrays.sort(location2.getTopologyPaths());
-
-      Assert.assertArrayEquals(location1.getHosts(), location2.getHosts());
-      Assert.assertArrayEquals(location1.getNames(), location2.getNames());
-      Assert.assertArrayEquals(location1.getTopologyPaths(),
-          location2.getTopologyPaths());
-    }
-  }
-
   private void testCreateSnapshot(String snapshotName) throws Exception {
     if (!this.isLocalFS()) {
       Path snapshottablePath = new Path("/tmp/tmp-snap-test");
@@ -1363,5 +1277,4 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
       fs.delete(snapshottablePath, true);
     }
   }
-
 }


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


[50/50] [abbrv] hadoop git commit: Merge branch 'trunk' into HDFS-7240

Posted by ae...@apache.org.
Merge branch 'trunk' into HDFS-7240

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


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/19f7f875
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/19f7f875
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/19f7f875

Branch: refs/heads/HDFS-7240
Commit: 19f7f87517d89cfe095f688163b867aaee24f04b
Parents: 44d0867 1ee2527
Author: Anu Engineer <ae...@apache.org>
Authored: Mon Sep 18 15:11:06 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Mon Sep 18 15:11:06 2017 -0700

----------------------------------------------------------------------
 LICENSE.txt                                     |    2 +-
 NOTICE.txt                                      |   65 +-
 hadoop-client-modules/hadoop-client-api/pom.xml |   27 +
 .../hadoop-client-check-invariants/pom.xml      |   74 +-
 .../ensure-jars-have-correct-contents.sh        |   82 +
 .../hadoop-client-check-test-invariants/pom.xml |   76 +-
 .../ensure-jars-have-correct-contents.sh        |   70 +
 .../apache/hadoop/example/ITUseMiniCluster.java |    4 +-
 .../hadoop-client-minicluster/pom.xml           |  105 +-
 .../hadoop-client-runtime/pom.xml               |   50 +
 hadoop-client-modules/hadoop-client/pom.xml     |   80 +-
 .../JWTRedirectAuthenticationHandler.java       |    4 +-
 .../TestJWTRedirectAuthenticationHandler.java   |  476 +++++
 .../TestJWTRedirectAuthentictionHandler.java    |  481 -----
 .../src/main/conf/log4j.properties              |    2 -
 .../crypto/key/KeyProviderCryptoExtension.java  |   10 +
 .../crypto/key/kms/KMSClientProvider.java       |   39 +-
 .../key/kms/LoadBalancingKMSClientProvider.java |   78 +-
 .../apache/hadoop/fs/AbstractFileSystem.java    |    5 +
 .../org/apache/hadoop/fs/BlockLocation.java     |   28 +
 .../fs/CommonConfigurationKeysPublic.java       |   29 -
 .../org/apache/hadoop/fs/FSDataInputStream.java |    4 +-
 .../java/org/apache/hadoop/fs/FileContext.java  |   33 +-
 .../java/org/apache/hadoop/fs/FileSystem.java   |   32 +-
 .../org/apache/hadoop/fs/LocatedFileStatus.java |    7 +
 .../org/apache/hadoop/http/HttpServer2.java     |   31 +-
 .../file/tfile/BoundedRangeFileInputStream.java |    2 +-
 .../hadoop/io/file/tfile/Compression.java       |    6 +-
 .../file/tfile/SimpleBufferedOutputStream.java  |    2 +-
 .../org/apache/hadoop/util/CpuTimeTracker.java  |    4 +-
 .../src/main/resources/core-default.xml         |  302 +--
 .../src/site/markdown/Compatibility.md          |  645 +++++--
 .../site/markdown/InterfaceClassification.md    |  227 ++-
 .../src/site/markdown/filesystem/filesystem.md  |    6 +-
 .../site/markdown/filesystem/introduction.md    |    7 +-
 .../src/site/markdown/filesystem/testing.md     |   10 +-
 .../conf/TestCommonConfigurationFields.java     |    6 +-
 .../apache/hadoop/conf/TestConfigRedactor.java  |    1 -
 .../kms/TestLoadBalancingKMSClientProvider.java |  315 +---
 .../hadoop/fs/FileSystemContractBaseTest.java   |   52 +-
 .../fs/contract/AbstractContractOpenTest.java   |    4 +-
 .../fs/contract/AbstractContractSeekTest.java   |    2 +-
 .../hadoop/fs/permission/TestFsPermission.java  |   57 +
 .../hadoop/fs/sftp/TestSFTPFileSystem.java      |   16 +-
 .../org/apache/hadoop/test/LambdaTestUtils.java |   68 +-
 .../apache/hadoop/test/TestLambdaTestUtils.java |  127 +-
 .../apache/hadoop/util/TestCpuTimeTracker.java  |   52 +
 .../src/test/resources/core-site.xml            |    6 -
 .../src/test/resources/jets3t.properties        |   16 -
 .../main/java/org/apache/hadoop/fs/Hdfs.java    |   15 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |    5 +
 .../hadoop/hdfs/DFSStripedOutputStream.java     |   17 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |   14 +
 .../hadoop/hdfs/protocol/BlocksStats.java       |   90 -
 .../hadoop/hdfs/protocol/ClientProtocol.java    |    4 +-
 .../hadoop/hdfs/protocol/ECBlockGroupStats.java |   83 +
 .../hdfs/protocol/ECBlockGroupsStats.java       |   83 -
 .../hdfs/protocol/HdfsLocatedFileStatus.java    |   12 +-
 .../hdfs/protocol/ReplicatedBlockStats.java     |   90 +
 .../ClientNamenodeProtocolTranslatorPB.java     |   32 +-
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |   58 +-
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |   53 -
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |   62 +-
 .../hadoop/hdfs/web/resources/GetOpParam.java   |   12 +-
 .../src/main/proto/ClientNamenodeProtocol.proto |   16 +-
 .../hadoop/fs/http/client/HttpFSFileSystem.java |   42 -
 .../hadoop/fs/http/server/FSOperations.java     |   38 -
 .../http/server/HttpFSParametersProvider.java   |    3 +-
 .../hadoop/fs/http/server/HttpFSServer.java     |   21 +-
 .../fs/http/client/BaseTestHttpFSWith.java      |   89 +-
 .../org/apache/hadoop/test/TestHdfsHelper.java  |    3 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |    8 +-
 ...tNamenodeProtocolServerSideTranslatorPB.java |   20 +-
 .../hadoop/hdfs/qjournal/client/QuorumCall.java |   65 +-
 .../hdfs/qjournal/server/JournalNode.java       |   47 +-
 .../server/blockmanagement/BlockManager.java    |   48 +-
 .../blockmanagement/CorruptReplicasMap.java     |    6 +-
 .../blockmanagement/InvalidateBlocks.java       |   52 +-
 .../blockmanagement/LowRedundancyBlocks.java    |   16 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |   12 +-
 .../erasurecode/ErasureCodingWorker.java        |   18 +-
 .../erasurecode/StripedBlockReconstructor.java  |   11 +
 .../datanode/metrics/DataNodeMetrics.java       |   18 +
 .../namenode/ErasureCodingPolicyManager.java    |   64 +-
 .../server/namenode/FSDirEncryptionZoneOp.java  |    5 +-
 .../server/namenode/FSDirErasureCodingOp.java   |   61 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |   24 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |   37 +
 .../hdfs/server/namenode/FSEditLogLoader.java   |   43 +
 .../hdfs/server/namenode/FSEditLogOp.java       |  325 ++++
 .../hdfs/server/namenode/FSEditLogOpCodes.java  |    5 +
 .../server/namenode/FSImageFormatPBINode.java   |    6 +-
 .../server/namenode/FSImageFormatProtobuf.java  |   48 +-
 .../server/namenode/FSImageSerialization.java   |   45 +
 .../hdfs/server/namenode/FSNamesystem.java      |   67 +-
 .../hdfs/server/namenode/LeaseManager.java      |    6 +
 .../hdfs/server/namenode/NameNodeRpcServer.java |   70 +-
 .../namenode/metrics/ECBlockGroupsMBean.java    |    4 +-
 .../namenode/startupprogress/StepType.java      |    7 +-
 .../web/resources/NamenodeWebHdfsMethods.java   |   16 -
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |   33 +-
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    |   33 -
 .../hadoop-hdfs/src/main/proto/fsimage.proto    |    4 +
 .../src/main/resources/hdfs-default.xml         |   25 +-
 .../src/main/webapps/hdfs/dfshealth.js          |   12 +-
 .../hadoop-hdfs/src/site/markdown/Federation.md |    4 +-
 .../src/site/markdown/HDFSErasureCoding.md      |   12 +-
 .../src/site/markdown/HdfsEditsViewer.md        |    4 +
 .../hadoop-hdfs/src/site/markdown/WebHDFS.md    |  188 +-
 .../apache/hadoop/cli/TestErasureCodingCLI.java |   11 +-
 .../java/org/apache/hadoop/fs/TestUnbuffer.java |   21 +
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   80 +-
 .../hdfs/ReadStripedFileWithDecodingHelper.java |    4 +-
 .../hdfs/TestDFSInotifyEventInputStream.java    |    2 +-
 .../hadoop/hdfs/TestDFSStripedInputStream.java  |    3 +-
 .../hadoop/hdfs/TestDFSStripedOutputStream.java |    5 +-
 .../TestDFSStripedOutputStreamWithFailure.java  |   37 +-
 .../hdfs/TestDecommissionWithStriped.java       |    4 +-
 .../hadoop/hdfs/TestDistributedFileSystem.java  |    1 -
 .../TestDistributedFileSystemWithECFile.java    |  186 ++
 .../hadoop/hdfs/TestEncryptionZonesWithKMS.java |   19 +-
 .../TestErasureCodeBenchmarkThroughput.java     |    4 +-
 .../hadoop/hdfs/TestErasureCodingPolicies.java  |   11 +-
 .../TestErasureCodingPolicyWithSnapshot.java    |    3 +-
 .../apache/hadoop/hdfs/TestFileChecksum.java    |    5 +-
 .../hadoop/hdfs/TestFileStatusWithECPolicy.java |    4 +-
 .../hadoop/hdfs/TestLeaseRecoveryStriped.java   |    3 +-
 .../TestReadStripedFileWithMissingBlocks.java   |    3 +-
 .../hadoop/hdfs/TestReconstructStripedFile.java |    6 +-
 .../hdfs/TestSafeModeWithStripedFile.java       |    4 +-
 .../hadoop/hdfs/TestSetrepIncreasing.java       |    4 +-
 .../TestUnsetAndChangeDirectoryEcPolicy.java    |    2 +-
 .../hadoop/hdfs/TestWriteReadStripedFile.java   |    4 +-
 .../datatransfer/sasl/TestSaslDataTransfer.java |    4 +-
 .../hdfs/qjournal/client/TestQuorumCall.java    |   31 +-
 .../hdfs/qjournal/server/TestJournalNode.java   |   21 +
 .../hdfs/server/balancer/TestBalancer.java      |    4 +-
 .../blockmanagement/TestBlockManager.java       |    7 +-
 .../blockmanagement/TestBlockStatsMXBean.java   |    7 +-
 .../TestBlockTokenWithDFSStriped.java           |    6 +-
 .../TestComputeInvalidateWork.java              |    8 +-
 .../blockmanagement/TestCorruptReplicaInfo.java |    8 +-
 .../TestLowRedundancyBlockQueues.java           |   10 +-
 ...constructStripedBlocksWithRackAwareness.java |   13 +-
 .../TestSequentialBlockGroupId.java             |    8 +-
 .../TestDataNodeErasureCodingMetrics.java       |   11 +-
 .../command/TestDiskBalancerCommand.java        |    5 +-
 .../hadoop/hdfs/server/mover/TestMover.java     |    4 +-
 .../TestAddOverReplicatedStripedBlocks.java     |    3 +-
 .../namenode/TestAddStripedBlockInFBR.java      |    5 +-
 .../server/namenode/TestAddStripedBlocks.java   |    4 +-
 .../hdfs/server/namenode/TestDeadDatanode.java  |    4 +-
 .../hdfs/server/namenode/TestDeleteRace.java    |   84 +
 .../server/namenode/TestEnabledECPolicies.java  |   23 +-
 .../server/namenode/TestFSEditLogLoader.java    |   89 +-
 .../hdfs/server/namenode/TestFSImage.java       |  159 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |   17 +-
 .../server/namenode/TestNameNodeMXBean.java     |    4 +-
 .../server/namenode/TestNamenodeRetryCache.java |    4 +-
 .../namenode/TestQuotaWithStripedBlocks.java    |    3 +-
 .../namenode/TestReconstructStripedBlocks.java  |   17 +-
 .../hdfs/server/namenode/TestReencryption.java  |  103 +-
 .../namenode/TestReencryptionWithKMS.java       |    5 +
 .../hdfs/server/namenode/TestStartup.java       |    2 +-
 .../server/namenode/TestStripedINodeFile.java   |    8 +-
 .../namenode/ha/TestRetryCacheWithHA.java       |    4 +-
 .../namenode/metrics/TestNameNodeMetrics.java   |    8 +-
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  |    7 +-
 .../TestOfflineEditsViewer.java                 |    2 +-
 .../TestOfflineImageViewer.java                 |    3 +-
 ...TestOfflineImageViewerWithStripedBlocks.java |    4 +-
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java |  281 +--
 .../hadoop-hdfs/src/test/resources/editsStored  |  Bin 5850 -> 6293 bytes
 .../src/test/resources/editsStored.xml          |   90 +-
 .../hadoop-mapreduce-client-core/pom.xml        |    4 +
 .../java/org/apache/hadoop/mapred/Task.java     |    2 +-
 .../hadoop/mapred/lib/MultipleOutputs.java      |    4 +-
 .../hadoop/mapreduce/JobResourceUploader.java   |   17 +
 .../apache/hadoop/mapreduce/MRJobConfig.java    |    5 +
 .../lib/output/FileOutputCommitter.java         |   25 +-
 .../mapreduce/lib/output/FileOutputFormat.java  |   57 +-
 .../lib/output/PathOutputCommitter.java         |   91 +
 .../hadoop/mapreduce/task/JobContextImpl.java   |   10 +-
 .../mapreduce/task/TaskAttemptContextImpl.java  |   13 +-
 .../hadoop/mapreduce/task/reduce/Fetcher.java   |   18 +-
 .../task/reduce/ShuffleSchedulerImpl.java       |    3 +
 .../src/main/resources/mapred-default.xml       |    9 +
 .../mapreduce/TestJobResourceUploader.java      |   46 +
 .../lib/output/TestPathOutputCommitter.java     |  377 ++++
 .../mapreduce/task/reduce/TestFetcher.java      |    9 +-
 .../task/reduce/TestShuffleScheduler.java       |   80 +
 .../hadoop/mapred/ResourceMgrDelegate.java      |   20 +
 .../hadoop/mapred/TestClientRedirect.java       |   26 +
 .../apache/hadoop/mapreduce/v2/TestMRJobs.java  |    7 +-
 .../hadoop/mapred/nativetask/NativeRuntime.java |    2 +-
 .../hadoop-mapreduce-client-shuffle/pom.xml     |    7 +
 .../apache/hadoop/examples/QuasiMonteCarlo.java |    2 +-
 hadoop-project/pom.xml                          |   23 +-
 hadoop-project/src/site/markdown/index.md.vm    |   10 +
 hadoop-project/src/site/site.xml                |    1 +
 .../hadoop-aws/dev-support/findbugs-exclude.xml |    4 -
 hadoop-tools/hadoop-aws/pom.xml                 |    7 -
 .../org/apache/hadoop/fs/s3a/Constants.java     |   12 +
 .../fs/s3a/InconsistentAmazonS3Client.java      |  143 +-
 .../java/org/apache/hadoop/fs/s3a/Listing.java  |   22 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |  162 +-
 .../apache/hadoop/fs/s3a/S3AOutputStream.java   |  143 --
 .../org/apache/hadoop/fs/s3a/S3ListRequest.java |   69 +
 .../org/apache/hadoop/fs/s3a/S3ListResult.java  |   97 +
 .../apache/hadoop/fs/s3native/FileMetadata.java |   59 -
 .../s3native/Jets3tNativeFileSystemStore.java   |  481 -----
 .../fs/s3native/NativeFileSystemStore.java      |   67 -
 .../hadoop/fs/s3native/NativeS3FileSystem.java  |  799 +-------
 .../hadoop/fs/s3native/PartialListing.java      |   64 -
 .../hadoop/fs/s3native/S3Credentials.java       |  100 -
 .../apache/hadoop/fs/s3native/S3Exception.java  |   39 -
 .../s3native/S3NativeFileSystemConfigKeys.java  |   66 -
 .../org/apache/hadoop/fs/s3native/package.html  |    5 +-
 .../markdown/tools/hadoop-aws/encryption.md     |  427 +++++
 .../src/site/markdown/tools/hadoop-aws/index.md | 1746 ++++++------------
 .../site/markdown/tools/hadoop-aws/s3guard.md   |   19 +-
 .../src/site/markdown/tools/hadoop-aws/s3n.md   |   52 +
 .../site/markdown/tools/hadoop-aws/testing.md   |   91 +-
 .../tools/hadoop-aws/troubleshooting_s3a.md     |  701 ++++++-
 .../fs/contract/s3a/ITestS3AContractDistCp.java |    1 -
 .../fs/contract/s3n/ITestS3NContractCreate.java |   41 -
 .../fs/contract/s3n/ITestS3NContractDelete.java |   34 -
 .../fs/contract/s3n/ITestS3NContractMkdir.java  |   34 -
 .../fs/contract/s3n/ITestS3NContractOpen.java   |   34 -
 .../fs/contract/s3n/ITestS3NContractRename.java |   35 -
 .../contract/s3n/ITestS3NContractRootDir.java   |   35 -
 .../fs/contract/s3n/ITestS3NContractSeek.java   |   34 -
 .../fs/contract/s3n/NativeS3Contract.java       |   50 -
 .../hadoop/fs/s3a/ITestS3ABlockOutputArray.java |    1 -
 .../ITestS3AContractGetFileStatusV1List.java    |   59 +
 ...ITestS3AEncryptionSSECBlockOutputStream.java |    1 -
 ...onSSEKMSUserDefinedKeyBlockOutputStream.java |    4 +-
 ...TestS3AEncryptionSSES3BlockOutputStream.java |    1 -
 .../fs/s3a/ITestS3GuardListConsistency.java     |   22 +-
 .../hadoop/fs/s3a/TestS3AGetFileStatus.java     |   41 +-
 .../fs/s3a/scale/AbstractSTestS3AHugeFiles.java |    1 -
 .../scale/ITestS3AHugeFilesClassicOutput.java   |   41 -
 ...ITestInMemoryNativeS3FileSystemContract.java |   33 -
 .../ITestJets3tNativeFileSystemStore.java       |  133 --
 .../ITestJets3tNativeS3FileSystemContract.java  |   33 -
 .../s3native/InMemoryNativeFileSystemStore.java |  213 ---
 .../NativeS3FileSystemContractBaseTest.java     |  266 ---
 .../fs/s3native/S3NInMemoryFileSystem.java      |   32 -
 .../hadoop/fs/s3native/TestS3Credentials.java   |  129 --
 .../fs/s3native/TestS3NInMemoryFileSystem.java  |   69 -
 .../src/test/resources/contract/s3n.xml         |  110 --
 .../src/test/resources/log4j.properties         |    3 +
 hadoop-tools/hadoop-azure-datalake/pom.xml      |    2 +-
 .../org/apache/hadoop/fs/adl/AdlConfKeys.java   |    1 -
 .../org/apache/hadoop/fs/adl/AdlFileSystem.java |    4 +-
 .../src/site/markdown/index.md                  |   21 +-
 .../hadoop/fs/adl/TestAzureADTokenProvider.java |    4 -
 hadoop-tools/hadoop-azure/pom.xml               |  251 +++
 .../fs/azure/AzureNativeFileSystemStore.java    |    2 +-
 .../hadoop-azure/src/site/markdown/index.md     |   94 +-
 .../src/site/markdown/testing_azure.md          |  576 ++++++
 .../hadoop/fs/azure/AbstractWasbTestBase.java   |  136 +-
 .../fs/azure/AbstractWasbTestWithTimeout.java   |   73 +
 .../fs/azure/AzureBlobStorageTestAccount.java   |   42 +-
 .../azure/ITestAzureConcurrentOutOfBandIo.java  |  179 ++
 ...zureConcurrentOutOfBandIoWithSecureMode.java |   33 +
 .../ITestAzureFileSystemErrorConditions.java    |  243 +++
 .../fs/azure/ITestBlobDataValidation.java       |  244 +++
 .../fs/azure/ITestBlobTypeSpeedDifference.java  |  163 ++
 .../fs/azure/ITestBlockBlobInputStream.java     |  874 +++++++++
 .../hadoop/fs/azure/ITestContainerChecks.java   |  194 ++
 ...estFileSystemOperationExceptionHandling.java |  283 +++
 ...TestFileSystemOperationExceptionMessage.java |   79 +
 ...perationsExceptionHandlingMultiThreaded.java |  366 ++++
 .../ITestFileSystemOperationsWithThreads.java   |  821 ++++++++
 ...stNativeAzureFSAuthWithBlobSpecificKeys.java |   40 +
 .../ITestNativeAzureFSAuthorizationCaching.java |   53 +
 .../azure/ITestNativeAzureFSPageBlobLive.java   |   43 +
 .../azure/ITestNativeAzureFileSystemAppend.java |  350 ++++
 ...ativeAzureFileSystemAtomicRenameDirList.java |   55 +
 ...veAzureFileSystemAuthorizationWithOwner.java |  122 ++
 ...ITestNativeAzureFileSystemClientLogging.java |  136 ++
 ...estNativeAzureFileSystemConcurrencyLive.java |  185 ++
 ...stNativeAzureFileSystemContractEmulator.java |   65 +
 .../ITestNativeAzureFileSystemContractLive.java |  108 ++
 ...tiveAzureFileSystemContractPageBlobLive.java |  114 ++
 .../azure/ITestNativeAzureFileSystemLive.java   |  236 +++
 .../ITestOutOfBandAzureBlobOperationsLive.java  |  185 ++
 .../ITestReadAndSeekPageBlobAfterWrite.java     |  341 ++++
 .../fs/azure/ITestWasbRemoteCallHelper.java     |  568 ++++++
 .../fs/azure/ITestWasbUriAndConfiguration.java  |  610 ++++++
 .../hadoop/fs/azure/MockWasbAuthorizerImpl.java |  103 +-
 .../fs/azure/NativeAzureFileSystemBaseTest.java |  115 +-
 .../hadoop/fs/azure/RunningLiveWasbTests.txt    |   22 -
 .../azure/TestAzureConcurrentOutOfBandIo.java   |  195 --
 ...zureConcurrentOutOfBandIoWithSecureMode.java |   50 -
 .../TestAzureFileSystemErrorConditions.java     |  244 ---
 .../hadoop/fs/azure/TestBlobDataValidation.java |  237 ---
 .../hadoop/fs/azure/TestBlobMetadata.java       |    7 +-
 .../fs/azure/TestBlobOperationDescriptor.java   |    3 -
 .../fs/azure/TestBlobTypeSpeedDifference.java   |  160 --
 .../fs/azure/TestBlockBlobInputStream.java      |  875 ---------
 .../fs/azure/TestClientThrottlingAnalyzer.java  |    5 +-
 .../hadoop/fs/azure/TestContainerChecks.java    |  185 --
 ...estFileSystemOperationExceptionHandling.java |  269 ---
 ...TestFileSystemOperationExceptionMessage.java |   79 -
 ...perationsExceptionHandlingMultiThreaded.java |  330 ----
 .../TestFileSystemOperationsWithThreads.java    |  821 --------
 ...stNativeAzureFSAuthWithBlobSpecificKeys.java |   44 -
 .../TestNativeAzureFSAuthorizationCaching.java  |   60 -
 .../fs/azure/TestNativeAzureFSPageBlobLive.java |   43 -
 .../azure/TestNativeAzureFileSystemAppend.java  |  362 ----
 ...ativeAzureFileSystemAtomicRenameDirList.java |   50 -
 .../TestNativeAzureFileSystemAuthorization.java |   53 +-
 ...veAzureFileSystemAuthorizationWithOwner.java |  122 --
 ...TestNativeAzureFileSystemBlockLocations.java |    8 +-
 .../TestNativeAzureFileSystemClientLogging.java |  140 --
 .../TestNativeAzureFileSystemConcurrency.java   |   29 +-
 ...estNativeAzureFileSystemConcurrencyLive.java |  184 --
 ...stNativeAzureFileSystemContractEmulator.java |   48 -
 .../TestNativeAzureFileSystemContractLive.java  |   80 -
 ...TestNativeAzureFileSystemContractMocked.java |    3 +
 ...tiveAzureFileSystemContractPageBlobLive.java |   93 -
 .../TestNativeAzureFileSystemFileNameCheck.java |   28 +-
 .../fs/azure/TestNativeAzureFileSystemLive.java |  242 ---
 .../azure/TestNativeAzureFileSystemMocked.java  |    4 +
 .../TestNativeAzureFileSystemUploadLogic.java   |   78 +-
 .../azure/TestOutOfBandAzureBlobOperations.java |    8 +-
 .../TestOutOfBandAzureBlobOperationsLive.java   |  203 --
 .../TestReadAndSeekPageBlobAfterWrite.java      |  355 ----
 .../azure/TestShellDecryptionKeyProvider.java   |   15 +-
 .../apache/hadoop/fs/azure/TestWasbFsck.java    |    9 +-
 .../fs/azure/TestWasbRemoteCallHelper.java      |  569 ------
 .../fs/azure/TestWasbUriAndConfiguration.java   |  617 -------
 .../ITestAzureNativeContractAppend.java         |   41 +
 .../ITestAzureNativeContractCreate.java         |   34 +
 .../ITestAzureNativeContractDelete.java         |   33 +
 .../ITestAzureNativeContractDistCp.java         |   47 +
 .../ITestAzureNativeContractGetFileStatus.java  |   35 +
 .../contract/ITestAzureNativeContractMkdir.java |   33 +
 .../contract/ITestAzureNativeContractOpen.java  |   34 +
 .../ITestAzureNativeContractRename.java         |   34 +
 .../contract/ITestAzureNativeContractSeek.java  |   34 +
 .../contract/NativeAzureFileSystemContract.java |   19 +-
 .../contract/TestAzureNativeContractAppend.java |   37 -
 .../contract/TestAzureNativeContractCreate.java |   30 -
 .../contract/TestAzureNativeContractDelete.java |   30 -
 .../contract/TestAzureNativeContractDistCp.java |   33 -
 .../TestAzureNativeContractGetFileStatus.java   |   30 -
 .../contract/TestAzureNativeContractMkdir.java  |   30 -
 .../contract/TestAzureNativeContractOpen.java   |   30 -
 .../contract/TestAzureNativeContractRename.java |   30 -
 .../contract/TestAzureNativeContractSeek.java   |   30 -
 .../integration/AbstractAzureScaleTest.java     |   66 +
 .../azure/integration/AzureTestConstants.java   |  180 ++
 .../fs/azure/integration/AzureTestUtils.java    |  479 +++++
 .../integration/CleanupTestContainers.java      |   87 +
 .../azure/integration/ITestAzureHugeFiles.java  |  456 +++++
 .../hadoop/fs/azure/integration/Sizes.java      |   43 +
 .../ITestAzureFileSystemInstrumentation.java    |  586 ++++++
 .../TestAzureFileSystemInstrumentation.java     |  579 ------
 .../hadoop/tools/util/TestDistCpUtils.java      |    6 +-
 .../dev-support/findbugs-exclude.xml            |   22 +
 .../ams/ApplicationMasterServiceProcessor.java  |    8 +-
 .../yarn/api/ApplicationClientProtocol.java     |   55 +-
 .../GetAllResourceProfilesRequest.java          |   35 +
 .../GetAllResourceProfilesResponse.java         |   60 +
 .../GetAllResourceTypeInfoRequest.java          |   35 +
 .../GetAllResourceTypeInfoResponse.java         |   60 +
 .../GetResourceProfileRequest.java              |   59 +
 .../GetResourceProfileResponse.java             |   68 +
 .../RegisterApplicationMasterResponse.java      |    8 +
 .../yarn/api/protocolrecords/ResourceTypes.java |   27 +
 .../UpdateApplicationTimeoutsResponse.java      |   19 +
 .../records/ApplicationResourceUsageReport.java |   57 +-
 .../hadoop/yarn/api/records/CollectorInfo.java  |    8 +-
 .../hadoop/yarn/api/records/ContainerState.java |    7 +-
 .../yarn/api/records/ProfileCapability.java     |  174 ++
 .../hadoop/yarn/api/records/Resource.java       |  402 +++-
 .../yarn/api/records/ResourceInformation.java   |  291 +++
 .../yarn/api/records/ResourceRequest.java       |   44 +-
 .../yarn/api/records/ResourceTypeInfo.java      |  197 ++
 .../api/records/impl/LightWeightResource.java   |  163 ++
 .../yarn/api/records/impl/package-info.java     |   26 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |   92 +-
 .../exceptions/ResourceNotFoundException.java   |   45 +
 .../ResourceProfilesNotEnabledException.java    |   43 +
 .../YARNFeatureNotEnabledException.java         |   45 +
 .../hadoop/yarn/util/UnitsConversionUtil.java   |  221 +++
 .../yarn/util/resource/ResourceUtils.java       |  576 ++++++
 .../hadoop/yarn/util/resource/package-info.java |   22 +
 .../main/proto/applicationclient_protocol.proto |    3 +
 .../src/main/proto/yarn_protos.proto            |   41 +
 .../src/main/proto/yarn_service_protos.proto    |   23 +
 .../yarn/conf/TestResourceInformation.java      |   73 +
 .../yarn/conf/TestYarnConfigurationFields.java  |    4 +
 .../yarn/util/TestUnitsConversionUtil.java      |  139 ++
 .../distributedshell/ApplicationMaster.java     |   61 +-
 .../applications/distributedshell/Client.java   |  176 +-
 .../distributedshell/TestDistributedShell.java  |   29 +
 .../hadoop-yarn/hadoop-yarn-client/pom.xml      |    1 +
 .../hadoop/yarn/client/api/AMRMClient.java      |  132 +-
 .../hadoop/yarn/client/api/YarnClient.java      |   45 +
 .../yarn/client/api/async/AMRMClientAsync.java  |    8 +-
 .../yarn/client/api/impl/AMRMClientImpl.java    |  153 +-
 .../client/api/impl/RemoteRequestsTable.java    |  109 +-
 .../yarn/client/api/impl/YarnClientImpl.java    |   29 +
 .../hadoop/yarn/client/cli/ApplicationCLI.java  |   51 +-
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  |  200 +-
 .../api/async/impl/TestAMRMClientAsync.java     |    2 +-
 .../yarn/client/api/impl/TestAMRMClient.java    |  145 +-
 .../impl/TestAMRMClientContainerRequest.java    |    8 +-
 .../api/impl/TestDistributedScheduling.java     |   12 +-
 .../yarn/client/api/impl/TestNMClient.java      |    5 +-
 .../TestOpportunisticContainerAllocation.java   |   31 +-
 .../yarn/client/api/impl/TestYarnClient.java    |   10 +-
 .../hadoop/yarn/client/cli/TestLogsCLI.java     |  205 +-
 .../hadoop/yarn/client/cli/TestYarnCLI.java     |   38 +-
 .../src/test/resources/resource-profiles.json   |   18 +
 .../hadoop-yarn/hadoop-yarn-common/pom.xml      |    8 +
 .../FileSystemBasedConfigurationProvider.java   |    3 +-
 .../hadoop/yarn/LocalConfigurationProvider.java |    3 +-
 .../ApplicationClientProtocolPBClientImpl.java  |   54 +
 .../ApplicationClientProtocolPBServiceImpl.java |   63 +
 .../pb/GetAllResourceProfilesRequestPBImpl.java |   55 +
 .../GetAllResourceProfilesResponsePBImpl.java   |  137 ++
 .../pb/GetAllResourceTypeInfoRequestPBImpl.java |   71 +
 .../GetAllResourceTypeInfoResponsePBImpl.java   |  184 ++
 .../pb/GetResourceProfileRequestPBImpl.java     |   96 +
 .../pb/GetResourceProfileResponsePBImpl.java    |  107 ++
 ...RegisterApplicationMasterResponsePBImpl.java |   58 +
 ...UpdateApplicationTimeoutsResponsePBImpl.java |  108 ++
 .../ApplicationResourceUsageReportPBImpl.java   |  150 +-
 .../impl/pb/ProfileCapabilityPBImpl.java        |  129 ++
 .../yarn/api/records/impl/pb/ProtoUtils.java    |   46 +
 .../api/records/impl/pb/ResourcePBImpl.java     |  152 +-
 .../records/impl/pb/ResourceRequestPBImpl.java  |   41 +-
 .../records/impl/pb/ResourceTypeInfoPBImpl.java |  152 ++
 .../logaggregation/ContainerLogFileInfo.java    |   93 +
 .../yarn/logaggregation/ContainerLogMeta.java   |    8 +-
 .../logaggregation/LogAggregationUtils.java     |   27 +
 .../yarn/logaggregation/LogCLIHelpers.java      |   20 +-
 .../yarn/logaggregation/LogToolUtils.java       |   26 +
 .../logaggregation/PerContainerLogFileInfo.java |   93 -
 .../LogAggregationFileController.java           |   45 +-
 .../ifile/IndexedFileAggregatedLogsBlock.java   |  275 +++
 .../LogAggregationIndexedFileController.java    | 1057 +++++++++++
 .../filecontroller/ifile/package-info.java      |   21 +
 .../tfile/LogAggregationTFileController.java    |   10 +-
 .../state/MultiStateTransitionListener.java     |   61 +
 .../hadoop/yarn/state/StateMachineFactory.java  |   40 +
 .../yarn/state/StateTransitionListener.java     |   50 +
 .../apache/hadoop/yarn/util/StringHelper.java   |   36 +
 .../resource/DefaultResourceCalculator.java     |    3 +-
 .../resource/DominantResourceCalculator.java    |  577 ++++--
 .../yarn/util/resource/ResourceCalculator.java  |    3 +-
 .../hadoop/yarn/util/resource/Resources.java    |  285 ++-
 .../src/main/resources/yarn-default.xml         |   59 +
 .../hadoop/yarn/api/BasePBImplRecordsTest.java  |   12 +
 .../hadoop/yarn/api/TestPBImplRecords.java      |   64 +
 .../hadoop/yarn/api/TestResourcePBImpl.java     |   61 +
 .../TestLogAggregationIndexFileController.java  |  314 ++++
 .../util/resource/TestResourceCalculator.java   |  225 ++-
 .../yarn/util/resource/TestResourceUtils.java   |  306 +++
 .../yarn/util/resource/TestResources.java       |  220 ++-
 .../resource-types/node-resources-1.xml         |   29 +
 .../resource-types/node-resources-2.xml         |   39 +
 .../resource-types/resource-types-1.xml         |   18 +
 .../resource-types/resource-types-2.xml         |   29 +
 .../resource-types/resource-types-3.xml         |   24 +
 .../resource-types/resource-types-4.xml         |   34 +
 .../resource-types/resource-types-error-1.xml   |   29 +
 .../resource-types/resource-types-error-2.xml   |   33 +
 .../resource-types/resource-types-error-3.xml   |   29 +
 .../resource-types/resource-types-error-4.xml   |   24 +
 ...pplicationHistoryManagerOnTimelineStore.java |   18 +-
 .../webapp/TestAHSWebServices.java              |    8 +-
 .../api/protocolrecords/NMContainerStatus.java  |   17 +-
 .../protocolrecords/NodeHeartbeatResponse.java  |    5 +
 .../impl/pb/NMContainerStatusPBImpl.java        |   30 +
 .../impl/pb/NodeHeartbeatResponsePBImpl.java    |   65 +
 .../hadoop/yarn/server/utils/BuilderUtils.java  |   16 +-
 .../yarn/server/webapp/AppAttemptBlock.java     |   26 +-
 .../hadoop/yarn/server/webapp/AppBlock.java     |   34 +-
 .../hadoop/yarn/server/webapp/AppsBlock.java    |   11 +-
 .../yarn/server/webapp/ContainerBlock.java      |   14 +-
 .../hadoop/yarn/server/webapp/WebServices.java  |   64 +-
 .../server/webapp/dao/ContainerLogsInfo.java    |   10 +-
 .../yarn_server_common_service_protos.proto     |    1 +
 .../yarn/server/MockResourceManagerFacade.java  |   24 +
 .../server/nodemanager/ContainerExecutor.java   |   26 +-
 .../ContainerStateTransitionListener.java       |   48 +
 .../hadoop/yarn/server/nodemanager/Context.java |    2 +
 .../nodemanager/LinuxContainerExecutor.java     |    3 +-
 .../yarn/server/nodemanager/NodeManager.java    |   48 +-
 .../nodemanager/NodeStatusUpdaterImpl.java      |   13 +-
 .../containermanager/container/Container.java   |    9 +
 .../container/ContainerEventType.java           |    6 +-
 .../container/ContainerImpl.java                |  236 ++-
 .../container/ContainerPauseEvent.java          |   40 +
 .../container/ContainerResumeEvent.java         |   39 +
 .../container/ContainerState.java               |    3 +-
 .../container/ResourceMappings.java             |  124 ++
 .../launcher/ContainerLaunch.java               |  111 +-
 .../launcher/ContainersLauncher.java            |   42 +
 .../launcher/ContainersLauncherEventType.java   |    4 +
 .../launcher/RecoverPausedContainerLaunch.java  |  124 ++
 .../launcher/RecoveredContainerLaunch.java      |    3 +-
 .../linux/resources/CGroupsHandlerImpl.java     |   11 -
 .../DelegatingLinuxContainerRuntime.java        |   79 +-
 .../runtime/LinuxContainerRuntimeConstants.java |    9 +
 .../monitor/ContainersMonitorImpl.java          |    4 +-
 .../runtime/ContainerRuntime.java               |    6 +-
 .../scheduler/ContainerScheduler.java           |  129 +-
 .../scheduler/ContainerSchedulerEventType.java  |    1 +
 .../recovery/NMLeveldbStateStoreService.java    |   84 +-
 .../recovery/NMNullStateStoreService.java       |   16 +
 .../recovery/NMStateStoreService.java           |   49 +-
 .../util/NodeManagerHardwareUtils.java          |   84 +-
 .../webapp/dao/NMContainerLogsInfo.java         |    8 +-
 .../yarn/server/nodemanager/TestEventFlow.java  |    3 +-
 .../server/nodemanager/TestNodeManager.java     |   68 +
 .../amrmproxy/BaseAMRMProxyTest.java            |    8 +
 .../containermanager/TestContainerManager.java  |    2 +-
 .../TestContainerManagerRecovery.java           |  160 +-
 .../container/TestContainer.java                |  107 ++
 .../launcher/TestContainerLaunch.java           |   32 +-
 .../TestDelegatingLinuxContainerRuntime.java    |  137 ++
 ...ContainerSchedulerBehaviorCompatibility.java |  100 +
 .../TestContainerSchedulerQueuing.java          |  103 ++
 .../recovery/NMMemoryStateStoreService.java     |   27 +
 .../TestNMLeveldbStateStoreService.java         |  138 +-
 .../util/TestNodeManagerHardwareUtils.java      |    2 +-
 .../nodemanager/webapp/MockContainer.java       |   11 +
 .../nodemanager/webapp/TestNMWebServices.java   |    8 +-
 .../conf/capacity-scheduler.xml                 |   35 +
 .../hadoop-yarn-server-resourcemanager/pom.xml  |    5 +
 .../resourcemanager/AMSProcessingChain.java     |    2 +-
 .../server/resourcemanager/ClientRMService.java |   62 +-
 .../resourcemanager/DefaultAMSProcessor.java    |   12 +-
 ...pportunisticContainerAllocatorAMService.java |    3 +-
 .../server/resourcemanager/RMAppManager.java    |   38 +-
 .../yarn/server/resourcemanager/RMContext.java  |    5 +
 .../server/resourcemanager/RMContextImpl.java   |   12 +
 .../server/resourcemanager/RMServerUtils.java   |   49 +-
 .../server/resourcemanager/ResourceManager.java |    9 +
 .../resourcemanager/ResourceTrackerService.java |    9 +-
 .../CapacitySchedulerPreemptionUtils.java       |    4 +-
 ...QueuePriorityContainerCandidateSelector.java |    5 +-
 .../ReservedContainerCandidatesSelector.java    |   34 +-
 .../resourcemanager/recovery/RMStateStore.java  |    7 +-
 .../records/ApplicationAttemptStateData.java    |   89 +-
 .../pb/ApplicationAttemptStateDataPBImpl.java   |   50 +
 .../reservation/CapacityOverTimePolicy.java     |   32 +-
 .../reservation/NoOverCommitPolicy.java         |    8 +-
 .../resource/ResourceProfilesManager.java       |   86 +
 .../resource/ResourceProfilesManagerImpl.java   |  257 +++
 .../resource/ResourceWeights.java               |   72 -
 .../server/resourcemanager/rmapp/RMAppImpl.java |   45 +-
 .../resourcemanager/rmapp/RMAppMetrics.java     |   41 +-
 .../attempt/AggregateAppResourceUsage.java      |   34 +-
 .../rmapp/attempt/RMAppAttemptImpl.java         |   32 +-
 .../rmapp/attempt/RMAppAttemptMetrics.java      |  105 +-
 .../rmcontainer/RMContainerImpl.java            |   20 +-
 .../resourcemanager/rmnode/RMNodeImpl.java      |   23 +-
 .../rmnode/RMNodeUpdateContainerEvent.java      |    9 +-
 .../scheduler/AbstractYarnScheduler.java        |  160 +-
 .../scheduler/AppSchedulingInfo.java            |    4 +
 .../scheduler/ClusterNodeTracker.java           |   13 +-
 .../scheduler/SchedulerApplicationAttempt.java  |   57 +-
 .../scheduler/SchedulerNode.java                |    8 -
 .../scheduler/SchedulerUtils.java               |   10 +
 .../scheduler/YarnScheduler.java                |   20 +
 .../scheduler/capacity/AbstractCSQueue.java     |    2 +-
 .../scheduler/capacity/CapacityScheduler.java   |   47 +-
 .../CapacitySchedulerConfiguration.java         |   26 +
 .../scheduler/capacity/LeafQueue.java           |   34 +-
 .../scheduler/capacity/ParentQueue.java         |    4 +
 .../scheduler/capacity/UsersManager.java        |    4 +-
 .../allocator/RegularContainerAllocator.java    |    8 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java |   21 +-
 .../common/fica/FiCaSchedulerNode.java          |   13 +
 .../scheduler/fair/AllocationConfiguration.java |   17 +-
 .../fair/AllocationFileLoaderService.java       |    7 +-
 .../scheduler/fair/FSAppAttempt.java            |   28 +-
 .../scheduler/fair/FSLeafQueue.java             |    3 +-
 .../resourcemanager/scheduler/fair/FSQueue.java |    9 +-
 .../scheduler/fair/FSSchedulerNode.java         |   10 +
 .../scheduler/fair/FairScheduler.java           |   75 +-
 .../scheduler/fair/Schedulable.java             |   12 +-
 .../fair/policies/ComputeFairShares.java        |   81 +-
 .../DominantResourceFairnessPolicy.java         |  240 ++-
 .../fair/policies/FairSharePolicy.java          |   15 +-
 .../scheduler/fifo/FifoScheduler.java           |   13 +-
 .../RMDelegationTokenSecretManager.java         |   16 +-
 .../resourcemanager/webapp/ContainerPage.java   |    3 +-
 .../webapp/MetricsOverviewTable.java            |   11 +-
 .../webapp/RMAppAttemptBlock.java               |   20 +-
 .../resourcemanager/webapp/RMAppBlock.java      |   49 +-
 .../resourcemanager/webapp/RMAppsBlock.java     |   17 +-
 .../webapp/RMContainerBlock.java                |   47 +
 .../server/resourcemanager/webapp/RMWebApp.java |    2 -
 .../resourcemanager/webapp/RMWebServices.java   |   52 +-
 .../resourcemanager/webapp/dao/AppInfo.java     |   28 +
 .../webapp/dao/ResourceInfo.java                |   36 +-
 .../webapp/dao/SchedulerInfo.java               |    3 +-
 .../yarn_server_resourcemanager_recovery.proto  |    2 +
 .../yarn/server/resourcemanager/MockRM.java     |   11 +
 .../ParameterizedSchedulerTestBase.java         |   63 +-
 .../server/resourcemanager/RMHATestBase.java    |    2 +
 .../server/resourcemanager/TestAppManager.java  |    9 +-
 .../TestApplicationMasterService.java           |   46 +-
 .../resourcemanager/TestClientRMService.java    |  120 ++
 .../resourcemanager/TestClientRMTokens.java     |    3 +
 .../TestContainerResourceUsage.java             |    7 +-
 .../TestNodeBlacklistingOnAMFailures.java       |   12 +
 .../yarn/server/resourcemanager/TestRM.java     |    6 +
 .../resourcemanager/TestRMAdminService.java     |    5 +
 .../server/resourcemanager/TestRMRestart.java   |    9 +-
 .../TestResourceTrackerService.java             |  108 ++
 .../TestTokenClientRMService.java               |    3 +
 .../TestWorkPreservingRMRestart.java            |    4 +
 .../applicationsmanager/MockAsm.java            |    9 +-
 .../metrics/TestSystemMetricsPublisher.java     |   15 +-
 .../TestSystemMetricsPublisherForV2.java        |   22 +-
 .../recovery/RMStateStoreTestBase.java          |    8 +-
 .../recovery/TestZKRMStateStore.java            |   23 +-
 .../reservation/BaseSharingPolicyTest.java      |  189 ++
 .../reservation/ReservationSystemTestUtil.java  |   28 +-
 .../reservation/TestCapacityOverTimePolicy.java |  339 +---
 .../TestFairSchedulerPlanFollower.java          |    6 +-
 .../reservation/TestNoOverCommitPolicy.java     |  185 +-
 .../reservation/TestReservationSystem.java      |    4 +
 .../resource/TestResourceProfiles.java          |  152 ++
 .../resource/TestResourceWeights.java           |   55 -
 .../resourcemanager/resource/TestResources.java |   43 -
 .../resourcetracker/TestNMReconnect.java        |    5 +
 .../rmapp/TestApplicationLifetimeMonitor.java   |   56 +-
 .../scheduler/TestAbstractYarnScheduler.java    |   42 +-
 .../TestSchedulingWithAllocationRequestId.java  |  155 +-
 .../capacity/TestCapacityScheduler.java         |  227 +--
 .../capacity/TestCapacitySchedulerPerf.java     |  265 +++
 .../capacity/TestContainerAllocation.java       |   50 +
 .../scheduler/capacity/TestLeafQueue.java       |    8 +-
 .../scheduler/capacity/TestReservations.java    |   20 +-
 .../scheduler/fair/FakeSchedulable.java         |   29 +-
 .../scheduler/fair/TestComputeFairShares.java   |   58 +-
 .../fair/TestContinuousScheduling.java          |   11 +-
 .../scheduler/fair/TestFairScheduler.java       |   18 +-
 .../scheduler/fair/TestSchedulingPolicy.java    |   13 +-
 .../TestDominantResourceFairnessPolicy.java     |  304 ++-
 .../policy/TestFairOrderingPolicy.java          |    6 +-
 .../security/TestClientToAMTokens.java          |    4 +
 .../resourcemanager/webapp/TestAppPage.java     |   13 +-
 .../resourcemanager/webapp/TestRMWebApp.java    |    3 -
 .../webapp/TestRMWebAppFairScheduler.java       |   11 +-
 .../webapp/TestRMWebServicesApps.java           |    2 +-
 .../webapp/TestRedirectionErrorPage.java        |    3 -
 .../resources/profiles/illegal-profiles-1.json  |   10 +
 .../resources/profiles/illegal-profiles-2.json  |   10 +
 .../resources/profiles/illegal-profiles-3.json  |   10 +
 .../resources/profiles/sample-profiles-1.json   |   14 +
 .../resources/profiles/sample-profiles-2.json   |   26 +
 .../DefaultClientRequestInterceptor.java        |   24 +
 .../clientrm/FederationClientInterceptor.java   |   23 +
 .../router/clientrm/RouterClientRMService.java  |   27 +
 .../PassThroughClientRequestInterceptor.java    |   24 +
 .../hadoop/yarn/server/MiniYARNCluster.java     |    7 +
 .../storage/TimelineSchemaCreator.java          |    2 +-
 .../src/site/markdown/CapacityScheduler.md      |   10 +
 .../src/site/markdown/DockerContainers.md       |  145 +-
 .../src/site/markdown/ResourceProfiles.md       |  116 ++
 .../src/site/markdown/TimelineServiceV2.md      |    4 +-
 .../app/components/base-chart-component.js      |    2 +-
 .../main/webapp/app/components/nodes-heatmap.js |   30 +-
 .../src/main/webapp/app/styles/app.css          |   19 +-
 pom.xml                                         |    2 +-
 677 files changed, 33887 insertions(+), 18480 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f7f875/LICENSE.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f7f875/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f7f875/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --cc hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 8c8507c,a11e7c3..538df97
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@@ -1428,20 -1344,16 +1344,16 @@@
    <description>The implementation class of the S3A AbstractFileSystem.</description>
  </property>
  
- <!-- Ozone file system properties -->
- <property>
-   <name>fs.ozfs.impl</name>
-   <value>org.apache.hadoop.fs.ozone.OzoneFileSystem</value>
-   <description>The implementation class of the Ozone FileSystem.</description>
- </property>
- 
--<property>
-   <name>fs.AbstractFileSystem.ozfs.impl</name>
-   <value>org.apache.hadoop.fs.ozone.OzFs</value>
-   <description>The implementation class of the OzFs AbstractFileSystem.</description>
 -  <name>fs.s3a.list.version</name>
 -  <value>2</value>
 -  <description>
 -    Select which version of the S3 SDK's List Objects API to use.  Currently
 -    support 2 (default) and 1 (older API).
 -  </description>
--</property>
++  <property>
++    <name>fs.s3a.list.version</name>
++    <value>2</value>
++    <description>
++      Select which version of the S3 SDK's List Objects API to use.  Currently
++      support 2 (default) and 1 (older API).
++    </description>
++  </property>
  
--<!-- Azure file system properties -->
++  <!-- Azure file system properties -->
  <property>
    <name>fs.wasb.impl</name>
    <value>org.apache.hadoop.fs.azure.NativeAzureFileSystem</value>
@@@ -1547,7 -1459,7 +1459,21 @@@
  </property>
  
  
--<!-- ipc properties -->
++  <!-- Ozone file system properties -->
++  <property>
++    <name>fs.ozfs.impl</name>
++    <value>org.apache.hadoop.fs.ozone.OzoneFileSystem</value>
++    <description>The implementation class of the Ozone FileSystem.</description>
++  </property>
++
++  <property>
++    <name>fs.AbstractFileSystem.ozfs.impl</name>
++    <value>org.apache.hadoop.fs.ozone.OzFs</value>
++    <description>The implementation class of the OzFs AbstractFileSystem.</description>
++  </property>
++
++
++  <!-- ipc properties -->
  
  <property>
    <name>ipc.client.idlethreshold</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f7f875/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f7f875/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f7f875/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f7f875/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f7f875/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f7f875/hadoop-project/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f7f875/hadoop-project/src/site/site.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19f7f875/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
----------------------------------------------------------------------
diff --cc hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
index 51806c9,742a43a..aa75ce8
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
@@@ -130,15 -130,17 +130,17 @@@ public class TestAMRMClient 
    @Before
    public void setup() throws Exception {
      conf = new YarnConfiguration();
-     createClusterAndStartApplication();
+     createClusterAndStartApplication(conf);
    }
  
-   private void createClusterAndStartApplication() throws Exception {
+   private void createClusterAndStartApplication(Configuration conf)
+       throws Exception {
      // start minicluster
+     this.conf = conf;
      conf.set(YarnConfiguration.RM_SCHEDULER, schedulerName);
      conf.setLong(
 -      YarnConfiguration.RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS,
 -      rolling_interval_sec);
 +        YarnConfiguration.RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS,
 +        rolling_interval_sec);
      conf.setLong(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, am_expire_ms);
      conf.setInt(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS, 100);
      // set the minimum allocation so that resource decrease can go under 1024
@@@ -527,16 -529,17 +529,17 @@@
        }
      }
    }
 -  
 +
    private void verifyMatches(
 -                  List<? extends Collection<ContainerRequest>> matches,
 -                  int matchSize) {
 +      List<? extends Collection<ContainerRequest>> matches,
 +      int matchSize) {
      assertEquals(1, matches.size());
-     assertEquals(matches.get(0).size(), matchSize);
+     assertEquals(matchSize, matches.get(0).size());
    }
 -  
 +
    @Test (timeout=60000)
-   public void testAMRMClientMatchingFitInferredRack() throws YarnException, IOException {
+   public void testAMRMClientMatchingFitInferredRack()
+       throws YarnException, IOException {
      AMRMClientImpl<ContainerRequest> amClient = null;
      try {
        // start am rm client
@@@ -605,6 -609,9 +609,9 @@@
        amClient.addContainerRequest(storedContainer2);
        amClient.addContainerRequest(storedContainer3);
  
+       ProfileCapability profileCapability =
+           ProfileCapability.newInstance(capability);
 -      
++
        // test addition and storage
        RemoteRequestsTable<ContainerRequest> remoteRequestsTable =
            amClient.getTable(0);
@@@ -613,10 -620,10 +620,10 @@@
            .remoteRequest.getNumContainers();
        assertEquals(2, containersRequestedAny);
        containersRequestedAny = remoteRequestsTable.get(priority1,
-           ResourceRequest.ANY, ExecutionType.GUARANTEED, capability)
+           ResourceRequest.ANY, ExecutionType.GUARANTEED, profileCapability)
            .remoteRequest.getNumContainers();
 -         assertEquals(1, containersRequestedAny);
 -      List<? extends Collection<ContainerRequest>> matches = 
 +      assertEquals(1, containersRequestedAny);
 +      List<? extends Collection<ContainerRequest>> matches =
            amClient.getMatchingRequests(priority, node, capability);
        verifyMatches(matches, 2);
        matches = amClient.getMatchingRequests(priority, rack, capability);
@@@ -1183,9 -1190,11 +1190,11 @@@
              true, null, ExecutionTypeRequest
              .newInstance(ExecutionType.OPPORTUNISTIC, true)));
  
+     ProfileCapability profileCapability =
 -          ProfileCapability.newInstance(capability);
++        ProfileCapability.newInstance(capability);
      int oppContainersRequestedAny =
          amClient.getTable(0).get(priority2, ResourceRequest.ANY,
-             ExecutionType.OPPORTUNISTIC, capability).remoteRequest
+             ExecutionType.OPPORTUNISTIC, profileCapability).remoteRequest
              .getNumContainers();
  
      assertEquals(1, oppContainersRequestedAny);


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


[49/50] [abbrv] hadoop git commit: HADOOP-14771. hadoop-client does not include hadoop-yarn-client. (Ajay Kumar via Haibo Chen)

Posted by ae...@apache.org.
HADOOP-14771. hadoop-client does not include hadoop-yarn-client. (Ajay Kumar via Haibo Chen)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1ee25278
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1ee25278
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1ee25278

Branch: refs/heads/HDFS-7240
Commit: 1ee25278c891e95ba2ab142e5b78aebd752ea163
Parents: 7c73292
Author: Haibo Chen <ha...@apache.org>
Authored: Mon Sep 18 14:25:35 2017 -0700
Committer: Haibo Chen <ha...@apache.org>
Committed: Mon Sep 18 14:25:35 2017 -0700

----------------------------------------------------------------------
 hadoop-client-modules/hadoop-client/pom.xml | 31 ++++++++++++++++++++++++
 1 file changed, 31 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ee25278/hadoop-client-modules/hadoop-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client/pom.xml b/hadoop-client-modules/hadoop-client/pom.xml
index bed3f5c..6500ebf 100644
--- a/hadoop-client-modules/hadoop-client/pom.xml
+++ b/hadoop-client-modules/hadoop-client/pom.xml
@@ -179,6 +179,37 @@
 
     <dependency>
       <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-client</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <!--Excluding hadoop-yarn-api & hadoop-annotations as they are already
+        included as direct dependencies. Guava,commons-cli and log4j are
+        transitive dependencies -->
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-annotations</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-cli</groupId>
+          <artifactId>commons-cli</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>log4j</groupId>
+          <artifactId>log4j</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-mapreduce-client-core</artifactId>
       <scope>compile</scope>
       <exclusions>


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


[17/50] [abbrv] hadoop git commit: HADOOP-14869. Upgrade Apache Kerby version to v1.0.1. Contributed by Wei Zhou

Posted by ae...@apache.org.
HADOOP-14869. Upgrade Apache Kerby version to v1.0.1. Contributed by Wei Zhou


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/08b98149
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/08b98149
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/08b98149

Branch: refs/heads/HDFS-7240
Commit: 08b98149e112044edfe72c216258e90e485f342b
Parents: 50764ef
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Sep 15 19:49:58 2017 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Fri Sep 15 19:49:58 2017 +0800

----------------------------------------------------------------------
 hadoop-project/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/08b98149/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 220bb62..b8fb961 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -1279,7 +1279,7 @@
         <dependency>
           <groupId>org.apache.kerby</groupId>
           <artifactId>kerb-simplekdc</artifactId>
-          <version>1.0.0</version>
+          <version>1.0.1</version>
         </dependency>
         <dependency>
           <groupId>org.apache.geronimo.specs</groupId>


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


[33/50] [abbrv] hadoop git commit: HADOOP-14853. hadoop-mapreduce-client-app is not a client module (haibochen via rkanter)

Posted by ae...@apache.org.
HADOOP-14853. hadoop-mapreduce-client-app is not a client module (haibochen via rkanter)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b9b607da
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b9b607da
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b9b607da

Branch: refs/heads/HDFS-7240
Commit: b9b607daa74322a2928f7671a985a60388b9b9c2
Parents: fbe06b5
Author: Robert Kanter <rk...@apache.org>
Authored: Fri Sep 15 13:43:39 2017 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Fri Sep 15 13:53:11 2017 -0700

----------------------------------------------------------------------
 hadoop-client-modules/hadoop-client/pom.xml | 49 ------------------------
 1 file changed, 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9b607da/hadoop-client-modules/hadoop-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client/pom.xml b/hadoop-client-modules/hadoop-client/pom.xml
index b802416..bed3f5c 100644
--- a/hadoop-client-modules/hadoop-client/pom.xml
+++ b/hadoop-client-modules/hadoop-client/pom.xml
@@ -131,55 +131,6 @@
 
     <dependency>
       <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-mapreduce-client-app</artifactId>
-      <scope>compile</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>javax.servlet</groupId>
-          <artifactId>javax.servlet-api</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-yarn-server-nodemanager</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-yarn-server-web-proxy</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-annotations</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.google.inject.extensions</groupId>
-          <artifactId>guice-servlet</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>junit</groupId>
-          <artifactId>junit</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.avro</groupId>
-          <artifactId>avro</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>io.netty</groupId>
-          <artifactId>netty</artifactId>
-        </exclusion>
-        <!-- No slf4j backends for downstream clients -->
-        <exclusion>
-          <groupId>org.slf4j</groupId>
-          <artifactId>slf4j-log4j12</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.zookeeper</groupId>
-          <artifactId>zookeeper</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-api</artifactId>
       <scope>compile</scope>
       <exclusions>


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


[04/50] [abbrv] hadoop git commit: YARN-6612. Update fair scheduler policies to be aware of resource types. (Contributed by Daniel Templeton via Yufei Gu)

Posted by ae...@apache.org.
YARN-6612. Update fair scheduler policies to be aware of resource types. (Contributed by Daniel Templeton via Yufei Gu)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/09b476e6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/09b476e6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/09b476e6

Branch: refs/heads/HDFS-7240
Commit: 09b476e6dabe8039a41dde7930c8a9c0d14bb750
Parents: 65a9410
Author: Yufei Gu <yu...@apache.org>
Authored: Thu Sep 14 11:22:08 2017 -0700
Committer: Yufei Gu <yu...@apache.org>
Committed: Thu Sep 14 11:23:37 2017 -0700

----------------------------------------------------------------------
 .../yarn/util/resource/ResourceUtils.java       |   5 +-
 .../resource/ResourceWeights.java               |  72 -----
 .../scheduler/fair/AllocationConfiguration.java |  17 +-
 .../fair/AllocationFileLoaderService.java       |   7 +-
 .../scheduler/fair/FSAppAttempt.java            |   9 +-
 .../scheduler/fair/FSLeafQueue.java             |   3 +-
 .../resourcemanager/scheduler/fair/FSQueue.java |   9 +-
 .../scheduler/fair/FairScheduler.java           |   9 +-
 .../scheduler/fair/Schedulable.java             |  12 +-
 .../fair/policies/ComputeFairShares.java        |  81 ++---
 .../DominantResourceFairnessPolicy.java         | 240 +++++++++++----
 .../fair/policies/FairSharePolicy.java          |  15 +-
 .../TestFairSchedulerPlanFollower.java          |   6 +-
 .../resource/TestResourceWeights.java           |  55 ----
 .../scheduler/fair/FakeSchedulable.java         |  29 +-
 .../scheduler/fair/TestComputeFairShares.java   |  58 ++--
 .../scheduler/fair/TestFairScheduler.java       |  14 +-
 .../scheduler/fair/TestSchedulingPolicy.java    |  13 +-
 .../TestDominantResourceFairnessPolicy.java     | 304 +++++++++++++++----
 19 files changed, 547 insertions(+), 411 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
index 1da5d6a..0564d74 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
@@ -344,11 +344,10 @@ public class ResourceUtils {
             addResourcesFileToConf(resourceFile, conf);
             LOG.debug("Found " + resourceFile + ", adding to configuration");
           } catch (FileNotFoundException fe) {
-            LOG.info("Unable to find '" + resourceFile
-                + "'. Falling back to memory and vcores as resources.");
+            LOG.debug("Unable to find '" + resourceFile + "'.");
           }
-          initializeResourcesMap(conf);
 
+          initializeResourcesMap(conf);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/ResourceWeights.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/ResourceWeights.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/ResourceWeights.java
deleted file mode 100644
index b66a5d0..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/ResourceWeights.java
+++ /dev/null
@@ -1,72 +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.yarn.server.resourcemanager.resource;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Evolving;
-import org.apache.hadoop.util.StringUtils;
-
-@Private
-@Evolving
-public class ResourceWeights {
-  public static final ResourceWeights NEUTRAL = new ResourceWeights(1.0f);
-
-  private final float[] weights = new float[ResourceType.values().length];
-
-  public ResourceWeights(float memoryWeight, float cpuWeight) {
-    weights[ResourceType.MEMORY.ordinal()] = memoryWeight;
-    weights[ResourceType.CPU.ordinal()] = cpuWeight;
-  }
-
-  public ResourceWeights(float weight) {
-    setWeight(weight);
-  }
-
-  public ResourceWeights() { }
-
-  public final void setWeight(float weight) {
-    for (int i = 0; i < weights.length; i++) {
-      weights[i] = weight;
-    }
-  }
-
-  public void setWeight(ResourceType resourceType, float weight) {
-    weights[resourceType.ordinal()] = weight;
-  }
-  
-  public float getWeight(ResourceType resourceType) {
-    return weights[resourceType.ordinal()];
-  }
-  
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("<");
-    for (int i = 0; i < ResourceType.values().length; i++) {
-      if (i != 0) {
-        sb.append(", ");
-      }
-      ResourceType resourceType = ResourceType.values()[i];
-      sb.append(StringUtils.toLowerCase(resourceType.name()));
-      sb.append(StringUtils.format(" weight=%.1f", getWeight(resourceType)));
-    }
-    sb.append(">");
-    return sb.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
index 71e6f7f..7bd6959 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.api.records.ReservationACL;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.security.AccessType;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSchedulerConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@@ -51,7 +50,7 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
   // Maximum amount of resources for each queue's ad hoc children
   private final Map<String, Resource> maxChildQueueResources;
   // Sharing weights for each queue
-  private final Map<String, ResourceWeights> queueWeights;
+  private final Map<String, Float> queueWeights;
   
   // Max concurrent running applications for each queue and for each user; in addition,
   // for users that have no max specified, we use the userMaxJobsDefault.
@@ -112,10 +111,12 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
   public AllocationConfiguration(Map<String, Resource> minQueueResources,
       Map<String, Resource> maxQueueResources,
       Map<String, Resource> maxChildQueueResources,
-      Map<String, Integer> queueMaxApps, Map<String, Integer> userMaxApps,
-      Map<String, ResourceWeights> queueWeights,
+      Map<String, Integer> queueMaxApps,
+      Map<String, Integer> userMaxApps,
+      Map<String, Float> queueWeights,
       Map<String, Float> queueMaxAMShares, int userMaxAppsDefault,
-      int queueMaxAppsDefault, Resource queueMaxResourcesDefault,
+      int queueMaxAppsDefault,
+      Resource queueMaxResourcesDefault,
       float queueMaxAMShareDefault,
       Map<String, SchedulingPolicy> schedulingPolicies,
       SchedulingPolicy defaultSchedulingPolicy,
@@ -253,9 +254,9 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
     return !nonPreemptableQueues.contains(queueName);
   }
 
-  private ResourceWeights getQueueWeight(String queue) {
-    ResourceWeights weight = queueWeights.get(queue);
-    return (weight == null) ? ResourceWeights.NEUTRAL : weight;
+  private float getQueueWeight(String queue) {
+    Float weight = queueWeights.get(queue);
+    return (weight == null) ? 1.0f : weight;
   }
 
   public int getUserMaxApps(String user) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
index 313a27a..4d918c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
@@ -45,7 +45,6 @@ import org.apache.hadoop.yarn.security.AccessType;
 import org.apache.hadoop.yarn.security.Permission;
 import org.apache.hadoop.yarn.security.PrivilegedEntity;
 import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.util.Clock;
@@ -232,7 +231,7 @@ public class AllocationFileLoaderService extends AbstractService {
     Map<String, Integer> queueMaxApps = new HashMap<>();
     Map<String, Integer> userMaxApps = new HashMap<>();
     Map<String, Float> queueMaxAMShares = new HashMap<>();
-    Map<String, ResourceWeights> queueWeights = new HashMap<>();
+    Map<String, Float> queueWeights = new HashMap<>();
     Map<String, SchedulingPolicy> queuePolicies = new HashMap<>();
     Map<String, Long> minSharePreemptionTimeouts = new HashMap<>();
     Map<String, Long> fairSharePreemptionTimeouts = new HashMap<>();
@@ -454,7 +453,7 @@ public class AllocationFileLoaderService extends AbstractService {
       Map<String, Integer> queueMaxApps,
       Map<String, Integer> userMaxApps,
       Map<String, Float> queueMaxAMShares,
-      Map<String, ResourceWeights> queueWeights,
+      Map<String, Float> queueWeights,
       Map<String, SchedulingPolicy> queuePolicies,
       Map<String, Long> minSharePreemptionTimeouts,
       Map<String, Long> fairSharePreemptionTimeouts,
@@ -522,7 +521,7 @@ public class AllocationFileLoaderService extends AbstractService {
       } else if ("weight".equals(field.getTagName())) {
         String text = ((Text)field.getFirstChild()).getData().trim();
         double val = Double.parseDouble(text);
-        queueWeights.put(queueName, new ResourceWeights((float)val));
+        queueWeights.put(queueName, (float)val);
       } else if ("minSharePreemptionTimeout".equals(field.getTagName())) {
         String text = ((Text)field.getFirstChild()).getData().trim();
         long val = Long.parseLong(text) * 1000L;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
index 309dff4..3024558 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
@@ -75,7 +74,6 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
 
   private final long startTime;
   private final Priority appPriority;
-  private final ResourceWeights resourceWeights;
   private Resource demand = Resources.createResource(0);
   private final FairScheduler scheduler;
   private Resource fairShare = Resources.createResource(0, 0);
@@ -120,11 +118,6 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     this.startTime = scheduler.getClock().getTime();
     this.lastTimeAtFairShare = this.startTime;
     this.appPriority = Priority.newInstance(1);
-    this.resourceWeights = new ResourceWeights();
-  }
-
-  ResourceWeights getResourceWeights() {
-    return resourceWeights;
   }
 
   /**
@@ -1281,7 +1274,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
   }
 
   @Override
-  public ResourceWeights getWeights() {
+  public float getWeight() {
     return scheduler.getAppWeight(this);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
index b911a1a..1dcfffc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
@@ -553,7 +552,7 @@ public class FSLeafQueue extends FSQueue {
    * @param weight queue weight
    */
   public void setWeights(float weight) {
-    this.weights = new ResourceWeights(weight);
+    this.weights = weight;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
index 1016823..8ae3cb6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.yarn.security.AccessRequest;
 import org.apache.hadoop.yarn.security.PrivilegedEntity;
 import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
 import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -70,7 +69,7 @@ public abstract class FSQueue implements Queue, Schedulable {
   
   protected SchedulingPolicy policy = SchedulingPolicy.DEFAULT_POLICY;
 
-  protected ResourceWeights weights;
+  protected float weights;
   protected Resource minShare;
   protected Resource maxShare;
   protected int maxRunningApps;
@@ -140,12 +139,12 @@ public abstract class FSQueue implements Queue, Schedulable {
     this.policy = policy;
   }
 
-  public void setWeights(ResourceWeights weights){
+  public void setWeights(float weights) {
     this.weights = weights;
   }
 
   @Override
-  public ResourceWeights getWeights() {
+  public float getWeight() {
     return weights;
   }
 
@@ -439,7 +438,7 @@ public abstract class FSQueue implements Queue, Schedulable {
   @Override
   public String toString() {
     return String.format("[%s, demand=%s, running=%s, share=%s, w=%s]",
-        getName(), getDemand(), getResourceUsage(), fairShare, getWeights());
+        getName(), getDemand(), getResourceUsage(), fairShare, getWeight());
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index a5afa96..9cf2b2e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -54,7 +54,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMCriticalThreadUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationConstants;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
@@ -369,7 +368,7 @@ public class FairScheduler extends
     return rmContext.getContainerTokenSecretManager();
   }
 
-  public ResourceWeights getAppWeight(FSAppAttempt app) {
+  public float getAppWeight(FSAppAttempt app) {
     try {
       readLock.lock();
       double weight = 1.0;
@@ -377,14 +376,10 @@ public class FairScheduler extends
         // Set weight based on current memory demand
         weight = Math.log1p(app.getDemand().getMemorySize()) / Math.log(2);
       }
-      weight *= app.getPriority().getPriority();
-      ResourceWeights resourceWeights = app.getResourceWeights();
-      resourceWeights.setWeight((float) weight);
-      return resourceWeights;
+      return (float)weight * app.getPriority().getPriority();
     } finally {
       readLock.unlock();
     }
-
   }
 
   public Resource getIncrementResourceCapability() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java
index fcdc056..4d6af98 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
 
 /**
  * A Schedulable represents an entity that can be scheduled such as an
@@ -72,8 +71,15 @@ public interface Schedulable {
   /** Maximum Resource share assigned to the schedulable. */
   Resource getMaxShare();
 
-  /** Job/queue weight in fair sharing. */
-  ResourceWeights getWeights();
+  /**
+   * Job/queue weight in fair sharing. Weights are only meaningful when
+   * compared. A weight of 2.0f has twice the weight of a weight of 1.0f,
+   * which has twice the weight of a weight of 0.5f. A weight of 1.0f is
+   * considered unweighted or a neutral weight. A weight of 0 is no weight.
+   *
+   * @return the weight
+   */
+  float getWeight();
 
   /** Start time for jobs in FIFO queues; meaningless for QueueSchedulables.*/
   long getStartTime();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/ComputeFairShares.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/ComputeFairShares.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/ComputeFairShares.java
index 440c73c..0a21b02 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/ComputeFairShares.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/ComputeFairShares.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.Schedulable;
 
@@ -47,7 +46,7 @@ public class ComputeFairShares {
    */
   public static void computeShares(
       Collection<? extends Schedulable> schedulables, Resource totalResources,
-      ResourceType type) {
+      String type) {
     computeSharesInternal(schedulables, totalResources, type, false);
   }
 
@@ -62,7 +61,7 @@ public class ComputeFairShares {
    */
   public static void computeSteadyShares(
       Collection<? extends FSQueue> queues, Resource totalResources,
-      ResourceType type) {
+      String type) {
     computeSharesInternal(queues, totalResources, type, true);
   }
 
@@ -110,9 +109,9 @@ public class ComputeFairShares {
    */
   private static void computeSharesInternal(
       Collection<? extends Schedulable> allSchedulables,
-      Resource totalResources, ResourceType type, boolean isSteadyShare) {
+      Resource totalResources, String type, boolean isSteadyShare) {
 
-    Collection<Schedulable> schedulables = new ArrayList<Schedulable>();
+    Collection<Schedulable> schedulables = new ArrayList<>();
     int takenResources = handleFixedFairShares(
         allSchedulables, schedulables, isSteadyShare, type);
 
@@ -124,7 +123,7 @@ public class ComputeFairShares {
     // have met all Schedulables' max shares.
     int totalMaxShare = 0;
     for (Schedulable sched : schedulables) {
-      long maxShare = getResourceValue(sched.getMaxShare(), type);
+      long maxShare = sched.getMaxShare().getResourceValue(type);
       totalMaxShare = (int) Math.min(maxShare + (long)totalMaxShare,
           Integer.MAX_VALUE);
       if (totalMaxShare == Integer.MAX_VALUE) {
@@ -132,7 +131,7 @@ public class ComputeFairShares {
       }
     }
 
-    long totalResource = Math.max((getResourceValue(totalResources, type) -
+    long totalResource = Math.max((totalResources.getResourceValue(type) -
         takenResources), 0);
     totalResource = Math.min(totalMaxShare, totalResource);
 
@@ -159,13 +158,15 @@ public class ComputeFairShares {
     }
     // Set the fair shares based on the value of R we've converged to
     for (Schedulable sched : schedulables) {
+      Resource target;
+
       if (isSteadyShare) {
-        setResourceValue(computeShare(sched, right, type),
-            ((FSQueue) sched).getSteadyFairShare(), type);
+        target = ((FSQueue) sched).getSteadyFairShare();
       } else {
-        setResourceValue(
-            computeShare(sched, right, type), sched.getFairShare(), type);
+        target = sched.getFairShare();
       }
+
+      target.setResourceValue(type, (long)computeShare(sched, right, type));
     }
   }
 
@@ -174,7 +175,7 @@ public class ComputeFairShares {
    * w2rRatio, for use in the computeFairShares algorithm as described in #
    */
   private static int resourceUsedWithWeightToResourceRatio(double w2rRatio,
-      Collection<? extends Schedulable> schedulables, ResourceType type) {
+      Collection<? extends Schedulable> schedulables, String type) {
     int resourcesTaken = 0;
     for (Schedulable sched : schedulables) {
       int share = computeShare(sched, w2rRatio, type);
@@ -188,10 +189,10 @@ public class ComputeFairShares {
    * weight-to-resource ratio w2rRatio.
    */
   private static int computeShare(Schedulable sched, double w2rRatio,
-      ResourceType type) {
-    double share = sched.getWeights().getWeight(type) * w2rRatio;
-    share = Math.max(share, getResourceValue(sched.getMinShare(), type));
-    share = Math.min(share, getResourceValue(sched.getMaxShare(), type));
+      String type) {
+    double share = sched.getWeight() * w2rRatio;
+    share = Math.max(share, sched.getMinShare().getResourceValue(type));
+    share = Math.min(share, sched.getMaxShare().getResourceValue(type));
     return (int) share;
   }
 
@@ -203,7 +204,7 @@ public class ComputeFairShares {
   private static int handleFixedFairShares(
       Collection<? extends Schedulable> schedulables,
       Collection<Schedulable> nonFixedSchedulables,
-      boolean isSteadyShare, ResourceType type) {
+      boolean isSteadyShare, String type) {
     int totalResource = 0;
 
     for (Schedulable sched : schedulables) {
@@ -211,11 +212,15 @@ public class ComputeFairShares {
       if (fixedShare < 0) {
         nonFixedSchedulables.add(sched);
       } else {
-        setResourceValue(fixedShare,
-            isSteadyShare
-                ? ((FSQueue)sched).getSteadyFairShare()
-                : sched.getFairShare(),
-            type);
+        Resource target;
+
+        if (isSteadyShare) {
+          target = ((FSQueue)sched).getSteadyFairShare();
+        } else {
+          target = sched.getFairShare();
+        }
+
+        target.setResourceValue(type, fixedShare);
         totalResource = (int) Math.min((long)totalResource + (long)fixedShare,
             Integer.MAX_VALUE);
       }
@@ -230,10 +235,10 @@ public class ComputeFairShares {
    * or the Schedulable is not active for instantaneous fairshare.
    */
   private static long getFairShareIfFixed(Schedulable sched,
-      boolean isSteadyShare, ResourceType type) {
+      boolean isSteadyShare, String type) {
 
     // Check if maxShare is 0
-    if (getResourceValue(sched.getMaxShare(), type) <= 0) {
+    if (sched.getMaxShare().getResourceValue(type) <= 0) {
       return 0;
     }
 
@@ -244,35 +249,11 @@ public class ComputeFairShares {
     }
 
     // Check if weight is 0
-    if (sched.getWeights().getWeight(type) <= 0) {
-      long minShare = getResourceValue(sched.getMinShare(), type);
+    if (sched.getWeight() <= 0) {
+      long minShare = sched.getMinShare().getResourceValue(type);
       return (minShare <= 0) ? 0 : minShare;
     }
 
     return -1;
   }
-
-  private static long getResourceValue(Resource resource, ResourceType type) {
-    switch (type) {
-    case MEMORY:
-      return resource.getMemorySize();
-    case CPU:
-      return resource.getVirtualCores();
-    default:
-      throw new IllegalArgumentException("Invalid resource");
-    }
-  }
-  
-  private static void setResourceValue(long val, Resource resource, ResourceType type) {
-    switch (type) {
-    case MEMORY:
-      resource.setMemorySize(val);
-      break;
-    case CPU:
-      resource.setVirtualCores((int)val);
-      break;
-    default:
-      throw new IllegalArgumentException("Invalid resource");
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
index 72377b0..e58b357 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/DominantResourceFairnessPolicy.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies;
 
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Comparator;
 
@@ -25,18 +26,15 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceType;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.Schedulable;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy;
-
 import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
-
-import static org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceType.*;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 
 /**
  * Makes scheduling decisions by trying to equalize dominant resource usage.
@@ -72,16 +70,18 @@ public class DominantResourceFairnessPolicy extends SchedulingPolicy {
   @Override
   public void computeShares(Collection<? extends Schedulable> schedulables,
       Resource totalResources) {
-    for (ResourceType type : ResourceType.values()) {
-      ComputeFairShares.computeShares(schedulables, totalResources, type);
+    for (ResourceInformation info: ResourceUtils.getResourceTypesArray()) {
+      ComputeFairShares.computeShares(schedulables, totalResources,
+          info.getName());
     }
   }
 
   @Override
   public void computeSteadyShares(Collection<? extends FSQueue> queues,
       Resource totalResources) {
-    for (ResourceType type : ResourceType.values()) {
-      ComputeFairShares.computeSteadyShares(queues, totalResources, type);
+    for (ResourceInformation info: ResourceUtils.getResourceTypesArray()) {
+      ComputeFairShares.computeSteadyShares(queues, totalResources,
+          info.getName());
     }
   }
 
@@ -110,9 +110,13 @@ public class DominantResourceFairnessPolicy extends SchedulingPolicy {
     COMPARATOR.setFSContext(fsContext);
   }
 
-  public static class DominantResourceFairnessComparator implements Comparator<Schedulable> {
-    private static final int NUM_RESOURCES = ResourceType.values().length;
-
+  /**
+   * This class compares two {@link Schedulable} instances according to the
+   * DRF policy. If neither instance is below min share, approximate fair share
+   * ratios are compared.
+   */
+  public static class DominantResourceFairnessComparator
+      implements Comparator<Schedulable> {
     private FSContext fsContext;
 
     public void setFSContext(FSContext fsContext) {
@@ -121,89 +125,199 @@ public class DominantResourceFairnessPolicy extends SchedulingPolicy {
 
     @Override
     public int compare(Schedulable s1, Schedulable s2) {
-      ResourceWeights sharesOfCluster1 = new ResourceWeights();
-      ResourceWeights sharesOfCluster2 = new ResourceWeights();
-      ResourceWeights sharesOfMinShare1 = new ResourceWeights();
-      ResourceWeights sharesOfMinShare2 = new ResourceWeights();
-      ResourceType[] resourceOrder1 = new ResourceType[NUM_RESOURCES];
-      ResourceType[] resourceOrder2 = new ResourceType[NUM_RESOURCES];
+      ResourceInformation[] info = ResourceUtils.getResourceTypesArray();
+      Resource usage1 = s1.getResourceUsage();
+      Resource usage2 = s2.getResourceUsage();
+      Resource minShare1 = s1.getMinShare();
+      Resource minShare2 = s2.getMinShare();
       Resource clusterCapacity = fsContext.getClusterResource();
 
-      // Calculate shares of the cluster for each resource both schedulables.
-      calculateShares(s1.getResourceUsage(),
-          clusterCapacity, sharesOfCluster1, resourceOrder1, s1.getWeights());
-      calculateShares(s1.getResourceUsage(),
-          s1.getMinShare(), sharesOfMinShare1, null, ResourceWeights.NEUTRAL);
-      calculateShares(s2.getResourceUsage(),
-          clusterCapacity, sharesOfCluster2, resourceOrder2, s2.getWeights());
-      calculateShares(s2.getResourceUsage(),
-          s2.getMinShare(), sharesOfMinShare2, null, ResourceWeights.NEUTRAL);
-      
+      // These arrays hold the usage, fair, and min share ratios for each
+      // resource type. ratios[0][x] are the usage ratios, ratios[1][x] are
+      // the fair share ratios, and ratios[2][x] are the min share ratios.
+      float[][] ratios1 = new float[info.length][3];
+      float[][] ratios2 = new float[info.length][3];
+
+      // Calculate cluster shares and approximate fair shares for each
+      // resource type of both schedulables.
+      int dominant1 = calculateClusterAndFairRatios(usage1, clusterCapacity,
+          ratios1, s1.getWeight());
+      int dominant2 = calculateClusterAndFairRatios(usage2, clusterCapacity,
+          ratios2, s2.getWeight());
+
       // A queue is needy for its min share if its dominant resource
-      // (with respect to the cluster capacity) is below its configured min share
-      // for that resource
-      boolean s1Needy = sharesOfMinShare1.getWeight(resourceOrder1[0]) < 1.0f;
-      boolean s2Needy = sharesOfMinShare2.getWeight(resourceOrder2[0]) < 1.0f;
+      // (with respect to the cluster capacity) is below its configured min
+      // share for that resource
+      boolean s1Needy =
+          usage1.getResources()[dominant1].getValue() <
+          minShare1.getResources()[dominant1].getValue();
+      boolean s2Needy =
+          usage2.getResources()[dominant2].getValue() <
+          minShare2.getResources()[dominant2].getValue();
       
       int res = 0;
+
       if (!s2Needy && !s1Needy) {
-        res = compareShares(sharesOfCluster1, sharesOfCluster2,
-            resourceOrder1, resourceOrder2);
+        // Sort shares by usage ratio and compare them by approximate fair share
+        // ratio
+        sortRatios(ratios1, ratios2);
+        res = compareRatios(ratios1, ratios2, 1);
       } else if (s1Needy && !s2Needy) {
         res = -1;
       } else if (s2Needy && !s1Needy) {
         res = 1;
       } else { // both are needy below min share
-        res = compareShares(sharesOfMinShare1, sharesOfMinShare2,
-            resourceOrder1, resourceOrder2);
+        // Calculate the min share ratios, then sort by usage ratio, and compare
+        // by min share ratio
+        calculateMinShareRatios(usage1, minShare1, ratios1);
+        calculateMinShareRatios(usage2, minShare2, ratios2);
+        sortRatios(ratios1, ratios2);
+        res = compareRatios(ratios1, ratios2, 2);
       }
+
       if (res == 0) {
         // Apps are tied in fairness ratio. Break the tie by submit time and job
         // name to get a deterministic ordering, which is useful for unit tests.
         res = (int) Math.signum(s1.getStartTime() - s2.getStartTime());
+
         if (res == 0) {
           res = s1.getName().compareTo(s2.getName());
         }
       }
+
       return res;
     }
-    
+
+    /**
+     * Sort both ratios arrays according to the usage ratios (the
+     * first index of the inner arrays, e.g. {@code ratios1[x][0]}).
+     *
+     * @param ratios1 the first ratios array
+     * @param ratios2 the second ratios array
+     */
+    @VisibleForTesting
+    void sortRatios(float[][] ratios1, float[][]ratios2) {
+      // sort order descending by resource share
+      Arrays.sort(ratios1, (float[] o1, float[] o2) ->
+          (int) Math.signum(o2[0] - o1[0]));
+      Arrays.sort(ratios2, (float[] o1, float[] o2) ->
+          (int) Math.signum(o2[0] - o1[0]));
+    }
+
     /**
-     * Calculates and orders a resource's share of a pool in terms of two vectors.
-     * The shares vector contains, for each resource, the fraction of the pool that
-     * it takes up.  The resourceOrder vector contains an ordering of resources
-     * by largest share.  So if resource=<10 MB, 5 CPU>, and pool=<100 MB, 10 CPU>,
-     * shares will be [.1, .5] and resourceOrder will be [CPU, MEMORY].
+     * Calculate a resource's usage ratio and approximate fair share ratio.
+     * The {@code shares} array will be populated with both the usage ratio
+     * and the approximate fair share ratio for each resource type. The usage
+     * ratio is calculated as {@code resource} divided by {@code cluster}.
+     * The approximate fair share ratio is calculated as the usage ratio
+     * divided by {@code weight}. If the cluster's resources are 100MB and
+     * 10 vcores, and the usage ({@code resource}) is 10 MB and 5 CPU, the
+     * usage ratios will be 0.1 and 0.5. If the weights are 2, the fair
+     * share ratios will be 0.05 and 0.25.
+     *
+     * The approximate fair share ratio is the usage divided by the
+     * approximate fair share, i.e. the cluster resources times the weight.
+     * The approximate fair share is an acceptable proxy for the fair share
+     * because when comparing resources, the resource with the higher weight
+     * will be assigned by the scheduler a proportionally higher fair share.
+     *
+     * The {@code shares} array must be at least <i>n</i> x 2, where <i>n</i>
+     * is the number of resource types. Only the first and second indices of
+     * the inner arrays in the {@code shares} array will be used, e.g.
+     * {@code shares[x][0]} and {@code shares[x][1]}.
+     *
+     * The return value will be the index of the dominant resource type in the
+     * {@code shares} array. The dominant resource is the resource type for
+     * which {@code resource} has the largest usage ratio.
+     *
+     * @param resource the resource for which to calculate ratios
+     * @param cluster the total cluster resources
+     * @param ratios the shares array to populate
+     * @param weight the resource weight
+     * @return the index of the resource type with the largest cluster share
      */
     @VisibleForTesting
-    void calculateShares(Resource resource, Resource pool,
-        ResourceWeights shares, ResourceType[] resourceOrder, ResourceWeights weights) {
-      shares.setWeight(MEMORY, (float)resource.getMemorySize() /
-          (pool.getMemorySize() * weights.getWeight(MEMORY)));
-      shares.setWeight(CPU, (float)resource.getVirtualCores() /
-          (pool.getVirtualCores() * weights.getWeight(CPU)));
-      // sort order vector by resource share
-      if (resourceOrder != null) {
-        if (shares.getWeight(MEMORY) > shares.getWeight(CPU)) {
-          resourceOrder[0] = MEMORY;
-          resourceOrder[1] = CPU;
-        } else  {
-          resourceOrder[0] = CPU;
-          resourceOrder[1] = MEMORY;
+    int calculateClusterAndFairRatios(Resource resource, Resource cluster,
+        float[][] ratios, float weight) {
+      ResourceInformation[] resourceInfo = resource.getResources();
+      ResourceInformation[] clusterInfo = cluster.getResources();
+      int max = 0;
+
+      for (int i = 0; i < clusterInfo.length; i++) {
+        // First calculate the cluster share
+        ratios[i][0] =
+            resourceInfo[i].getValue() / (float) clusterInfo[i].getValue();
+
+        // Use the cluster share to find the dominant resource
+        if (ratios[i][0] > ratios[max][0]) {
+          max = i;
         }
+
+        // Now divide by the weight to get the approximate fair share.
+        // It's OK if the weight is zero, because the floating point division
+        // will yield Infinity, i.e. this Schedulable will lose out to any
+        // other Schedulable with non-zero weight.
+        ratios[i][1] = ratios[i][0] / weight;
       }
+
+      return max;
     }
     
-    private int compareShares(ResourceWeights shares1, ResourceWeights shares2,
-        ResourceType[] resourceOrder1, ResourceType[] resourceOrder2) {
-      for (int i = 0; i < resourceOrder1.length; i++) {
-        int ret = (int)Math.signum(shares1.getWeight(resourceOrder1[i])
-            - shares2.getWeight(resourceOrder2[i]));
+    /**
+     * Calculate a resource's min share ratios. The {@code ratios} array will be
+     * populated with the {@code resource} divided by {@code minShare} for each
+     * resource type. If the min shares are 5 MB and 10 vcores, and the usage
+     * ({@code resource}) is 10 MB and 5 CPU, the ratios will be 2 and 0.5.
+     *
+     * The {@code ratios} array must be <i>n</i> x 3, where <i>n</i> is the
+     * number of resource types. Only the third index of the inner arrays in
+     * the {@code ratios} array will be used, e.g. {@code ratios[x][2]}.
+     *
+     * @param resource the resource for which to calculate min shares
+     * @param minShare the min share
+     * @param ratios the shares array to populate
+     */
+    @VisibleForTesting
+    void calculateMinShareRatios(Resource resource, Resource minShare,
+        float[][] ratios) {
+      ResourceInformation[] resourceInfo = resource.getResources();
+      ResourceInformation[] minShareInfo = minShare.getResources();
+
+      for (int i = 0; i < minShareInfo.length; i++) {
+        ratios[i][2] =
+            resourceInfo[i].getValue() / (float) minShareInfo[i].getValue();
+      }
+    }
+
+    /**
+     * Compare the two ratios arrays and return -1, 0, or 1 if the first array
+     * is less than, equal to, or greater than the second array, respectively.
+     * The {@code index} parameter determines which index of the inner arrays
+     * will be used for the comparisons. 0 is for usage ratios, 1 is for
+     * fair share ratios, and 2 is for the min share ratios. The ratios arrays
+     * are assumed to be sorted in descending order by usage ratio.
+     *
+     * @param ratios1 the first shares array
+     * @param ratios2 the second shares array
+     * @param index the outer index of the ratios arrays to compare. 0 is for
+     * usage ratio, 1 is for approximate fair share ratios, and 1 is for min
+     * share ratios
+     * @return -1, 0, or 1 if the first array is less than, equal to, or
+     * greater than the second array, respectively
+     */
+    @VisibleForTesting
+    int compareRatios(float[][] ratios1, float[][] ratios2, int index) {
+      int ret = 0;
+
+      for (int i = 0; i < ratios1.length; i++) {
+        ret = (int) Math.signum(ratios1[i][index] - ratios2[i][index]);
+
         if (ret != 0) {
-          return ret;
+          break;
         }
       }
-      return 0;
+
+      return ret;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
index 0ef90a1..8179aa7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/FairSharePolicy.java
@@ -26,7 +26,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceType;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.Schedulable;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SchedulingPolicy;
@@ -42,9 +42,10 @@ import com.google.common.annotations.VisibleForTesting;
 @Private
 @Unstable
 public class FairSharePolicy extends SchedulingPolicy {
-  private static final Log LOG = LogFactory.getLog(FairSharePolicy.class);
   @VisibleForTesting
   public static final String NAME = "fair";
+  private static final Log LOG = LogFactory.getLog(FairSharePolicy.class);
+  private static final String MEMORY = ResourceInformation.MEMORY_MB.getName();
   private static final DefaultResourceCalculator RESOURCE_CALCULATOR =
       new DefaultResourceCalculator();
   private static final FairShareComparator COMPARATOR =
@@ -164,10 +165,11 @@ public class FairSharePolicy extends SchedulingPolicy {
      */
     private int compareFairShareUsage(Schedulable s1, Schedulable s2,
         Resource resourceUsage1, Resource resourceUsage2) {
-      double weight1 = s1.getWeights().getWeight(ResourceType.MEMORY);
-      double weight2 = s2.getWeights().getWeight(ResourceType.MEMORY);
+      double weight1 = s1.getWeight();
+      double weight2 = s2.getWeight();
       double useToWeightRatio1;
       double useToWeightRatio2;
+
       if (weight1 > 0.0 && weight2 > 0.0) {
         useToWeightRatio1 = resourceUsage1.getMemorySize() / weight1;
         useToWeightRatio2 = resourceUsage2.getMemorySize() / weight2;
@@ -213,14 +215,13 @@ public class FairSharePolicy extends SchedulingPolicy {
   @Override
   public void computeShares(Collection<? extends Schedulable> schedulables,
       Resource totalResources) {
-    ComputeFairShares.computeShares(schedulables, totalResources, ResourceType.MEMORY);
+    ComputeFairShares.computeShares(schedulables, totalResources, MEMORY);
   }
 
   @Override
   public void computeSteadyShares(Collection<? extends FSQueue> queues,
       Resource totalResources) {
-    ComputeFairShares.computeSteadyShares(queues, totalResources,
-        ResourceType.MEMORY);
+    ComputeFairShares.computeSteadyShares(queues, totalResources, MEMORY);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestFairSchedulerPlanFollower.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestFairSchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestFairSchedulerPlanFollower.java
index 9561234..f0f2b35 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestFairSchedulerPlanFollower.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestFairSchedulerPlanFollower.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -137,7 +136,7 @@ public class TestFairSchedulerPlanFollower extends
   }
   @Override
   protected void verifyCapacity(Queue defQ) {
-    assertTrue(((FSQueue) defQ).getWeights().getWeight(ResourceType.MEMORY) > 0.9);
+    assertTrue(((FSQueue) defQ).getWeight() > 0.9);
   }
 
   @Override
@@ -173,8 +172,7 @@ public class TestFairSchedulerPlanFollower extends
             false);
     assertNotNull(q);
     // For now we are setting both to same weight
-    Assert.assertEquals(expectedCapacity,
-        q.getWeights().getWeight(ResourceType.MEMORY), 0.01);
+    Assert.assertEquals(expectedCapacity, q.getWeight(), 0.01);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resource/TestResourceWeights.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resource/TestResourceWeights.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resource/TestResourceWeights.java
deleted file mode 100644
index f420b9e..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resource/TestResourceWeights.java
+++ /dev/null
@@ -1,55 +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.yarn.server.resourcemanager.resource;
-
-import org.junit.Assert;
-
-import org.junit.Test;
-
-public class TestResourceWeights {
-  
-  @Test(timeout=3000)
-  public void testWeights() {
-    ResourceWeights rw1 = new ResourceWeights();
-    Assert.assertEquals("Default CPU weight should be 0.0f.", 0.0f, 
-        rw1.getWeight(ResourceType.CPU), 0.00001f);
-    Assert.assertEquals("Default memory weight should be 0.0f", 0.0f, 
-        rw1.getWeight(ResourceType.MEMORY), 0.00001f);
-
-    ResourceWeights rw2 = new ResourceWeights(2.0f);
-    Assert.assertEquals("The CPU weight should be 2.0f.", 2.0f, 
-        rw2.getWeight(ResourceType.CPU), 0.00001f);
-    Assert.assertEquals("The memory weight should be 2.0f", 2.0f, 
-        rw2.getWeight(ResourceType.MEMORY), 0.00001f);
-
-    // set each individually
-    ResourceWeights rw3 = new ResourceWeights(1.5f, 2.0f);
-    Assert.assertEquals("The CPU weight should be 2.0f", 2.0f, 
-        rw3.getWeight(ResourceType.CPU), 0.00001f);
-    Assert.assertEquals("The memory weight should be 1.5f", 1.5f, 
-        rw3.getWeight(ResourceType.MEMORY), 0.00001f);
-
-    // reset weights
-    rw3.setWeight(ResourceType.CPU, 2.5f);
-    Assert.assertEquals("The CPU weight should be set to 2.5f.", 2.5f,
-        rw3.getWeight(ResourceType.CPU), 0.00001f);
-    rw3.setWeight(ResourceType.MEMORY, 4.0f);
-    Assert.assertEquals("The memory weight should be set to 4.0f.", 4.0f, 
-        rw3.getWeight(ResourceType.MEMORY), 0.00001f);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FakeSchedulable.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FakeSchedulable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FakeSchedulable.java
index 36ff85e..03332b2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FakeSchedulable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FakeSchedulable.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
@@ -33,7 +31,7 @@ public class FakeSchedulable implements Schedulable {
   private Resource minShare;
   private Resource maxShare;
   private Resource fairShare;
-  private ResourceWeights weights;
+  private float weights;
   private Priority priority;
   private long startTime;
   
@@ -49,28 +47,31 @@ public class FakeSchedulable implements Schedulable {
     this(minShare, maxShare, 1, 0, 0, 0);
   }
   
-  public FakeSchedulable(int minShare, double memoryWeight) {
+  public FakeSchedulable(int minShare, float memoryWeight) {
     this(minShare, Integer.MAX_VALUE, memoryWeight, 0, 0, 0);
   }
   
-  public FakeSchedulable(int minShare, int maxShare, double memoryWeight) {
+  public FakeSchedulable(int minShare, int maxShare, float memoryWeight) {
     this(minShare, maxShare, memoryWeight, 0, 0, 0);
   }
   
-  public FakeSchedulable(int minShare, int maxShare, double weight, int fairShare, int usage,
-      long startTime) {
-    this(Resources.createResource(minShare, 0), Resources.createResource(maxShare, 0),
-        new ResourceWeights((float)weight), Resources.createResource(fairShare, 0),
+  public FakeSchedulable(int minShare, int maxShare, float weight,
+      int fairShare, int usage, long startTime) {
+    this(Resources.createResource(minShare, 0),
+        Resources.createResource(maxShare, 0),
+        weight, Resources.createResource(fairShare, 0),
         Resources.createResource(usage, 0), startTime);
   }
   
-  public FakeSchedulable(Resource minShare, ResourceWeights weights) {
-    this(minShare, Resources.createResource(Integer.MAX_VALUE, Integer.MAX_VALUE),
-        weights, Resources.createResource(0, 0), Resources.createResource(0, 0), 0);
+  public FakeSchedulable(Resource minShare, float weights) {
+    this(minShare,
+        Resources.createResource(Integer.MAX_VALUE, Integer.MAX_VALUE),
+        weights, Resources.createResource(0, 0),
+        Resources.createResource(0, 0), 0);
   }
   
   public FakeSchedulable(Resource minShare, Resource maxShare,
-      ResourceWeights weight, Resource fairShare, Resource usage, long startTime) {
+      float weight, Resource fairShare, Resource usage, long startTime) {
     this.minShare = minShare;
     this.maxShare = maxShare;
     this.weights = weight;
@@ -121,7 +122,7 @@ public class FakeSchedulable implements Schedulable {
   }
   
   @Override
-  public ResourceWeights getWeights() {
+  public float getWeight() {
     return weights;
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestComputeFairShares.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestComputeFairShares.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestComputeFairShares.java
index 4f3ccb2..c3bcb3b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestComputeFairShares.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestComputeFairShares.java
@@ -20,12 +20,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
 import java.util.ArrayList;
 import java.util.List;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
 
 import org.junit.Assert;
 
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceType;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.ComputeFairShares;
 import org.junit.Before;
 import org.junit.Test;
@@ -52,7 +51,7 @@ public class TestComputeFairShares {
     scheds.add(new FakeSchedulable());
     scheds.add(new FakeSchedulable());
     ComputeFairShares.computeShares(scheds,
-        Resources.createResource(40), ResourceType.MEMORY);
+        Resources.createResource(40), ResourceInformation.MEMORY_MB.getName());
     verifyMemoryShares(10, 10, 10, 10);
   }
   
@@ -70,7 +69,7 @@ public class TestComputeFairShares {
     scheds.add(new FakeSchedulable(0, 11));
     scheds.add(new FakeSchedulable(0, 3));
     ComputeFairShares.computeShares(scheds,
-        Resources.createResource(40), ResourceType.MEMORY);
+        Resources.createResource(40), ResourceInformation.MEMORY_MB.getName());
     verifyMemoryShares(13, 13, 11, 3);
   }
 
@@ -90,7 +89,7 @@ public class TestComputeFairShares {
     scheds.add(new FakeSchedulable(0));
     scheds.add(new FakeSchedulable(2));
     ComputeFairShares.computeShares(scheds,
-        Resources.createResource(40), ResourceType.MEMORY);
+        Resources.createResource(40), ResourceInformation.MEMORY_MB.getName());
     verifyMemoryShares(20, 18, 0, 2);
   }
   
@@ -100,12 +99,12 @@ public class TestComputeFairShares {
    */
   @Test
   public void testWeightedSharing() {
-    scheds.add(new FakeSchedulable(0, 2.0));
-    scheds.add(new FakeSchedulable(0, 1.0));
-    scheds.add(new FakeSchedulable(0, 1.0));
-    scheds.add(new FakeSchedulable(0, 0.5));
+    scheds.add(new FakeSchedulable(0, 2.0f));
+    scheds.add(new FakeSchedulable(0, 1.0f));
+    scheds.add(new FakeSchedulable(0, 1.0f));
+    scheds.add(new FakeSchedulable(0, 0.5f));
     ComputeFairShares.computeShares(scheds,
-        Resources.createResource(45), ResourceType.MEMORY);
+        Resources.createResource(45), ResourceInformation.MEMORY_MB.getName());
     verifyMemoryShares(20, 10, 10, 5);
   }
   
@@ -118,12 +117,12 @@ public class TestComputeFairShares {
    */
   @Test
   public void testWeightedSharingWithMaxShares() {
-    scheds.add(new FakeSchedulable(0, 10, 2.0));
-    scheds.add(new FakeSchedulable(0, 11, 1.0));
-    scheds.add(new FakeSchedulable(0, 30, 1.0));
-    scheds.add(new FakeSchedulable(0, 20, 0.5));
+    scheds.add(new FakeSchedulable(0, 10, 2.0f));
+    scheds.add(new FakeSchedulable(0, 11, 1.0f));
+    scheds.add(new FakeSchedulable(0, 30, 1.0f));
+    scheds.add(new FakeSchedulable(0, 20, 0.5f));
     ComputeFairShares.computeShares(scheds,
-        Resources.createResource(45), ResourceType.MEMORY);
+        Resources.createResource(45), ResourceInformation.MEMORY_MB.getName());
     verifyMemoryShares(10, 11, 16, 8);
   }
 
@@ -137,12 +136,12 @@ public class TestComputeFairShares {
    */
   @Test
   public void testWeightedSharingWithMinShares() {
-    scheds.add(new FakeSchedulable(20, 2.0));
-    scheds.add(new FakeSchedulable(0, 1.0));
-    scheds.add(new FakeSchedulable(5, 1.0));
-    scheds.add(new FakeSchedulable(15, 0.5));
+    scheds.add(new FakeSchedulable(20, 2.0f));
+    scheds.add(new FakeSchedulable(0, 1.0f));
+    scheds.add(new FakeSchedulable(5, 1.0f));
+    scheds.add(new FakeSchedulable(15, 0.5f));
     ComputeFairShares.computeShares(scheds,
-        Resources.createResource(45), ResourceType.MEMORY);
+        Resources.createResource(45), ResourceInformation.MEMORY_MB.getName());
     verifyMemoryShares(20, 5, 5, 15);
   }
 
@@ -158,7 +157,8 @@ public class TestComputeFairShares {
     scheds.add(new FakeSchedulable());
     scheds.add(new FakeSchedulable());
     ComputeFairShares.computeShares(scheds,
-        Resources.createResource(40 * million), ResourceType.MEMORY);
+        Resources.createResource(40 * million),
+        ResourceInformation.MEMORY_MB.getName());
     verifyMemoryShares(10 * million, 10 * million, 10 * million, 10 * million);
   }
   
@@ -168,7 +168,7 @@ public class TestComputeFairShares {
   @Test
   public void testEmptyList() {
     ComputeFairShares.computeShares(scheds,
-        Resources.createResource(40), ResourceType.MEMORY);
+        Resources.createResource(40), ResourceInformation.MEMORY_MB.getName());
     verifyMemoryShares();
   }
   
@@ -177,16 +177,12 @@ public class TestComputeFairShares {
    */
   @Test
   public void testCPU() {
-    scheds.add(new FakeSchedulable(Resources.createResource(0, 20),
-        new ResourceWeights(2.0f)));
-    scheds.add(new FakeSchedulable(Resources.createResource(0, 0),
-        new ResourceWeights(1.0f)));
-    scheds.add(new FakeSchedulable(Resources.createResource(0, 5),
-        new ResourceWeights(1.0f)));
-    scheds.add(new FakeSchedulable(Resources.createResource(0, 15),
-        new ResourceWeights(0.5f)));
+    scheds.add(new FakeSchedulable(Resources.createResource(0, 20), 2.0f));
+    scheds.add(new FakeSchedulable(Resources.createResource(0, 0), 1.0f));
+    scheds.add(new FakeSchedulable(Resources.createResource(0, 5), 1.0f));
+    scheds.add(new FakeSchedulable(Resources.createResource(0, 15), 0.5f));
     ComputeFairShares.computeShares(scheds,
-        Resources.createResource(0, 45), ResourceType.CPU);
+        Resources.createResource(0, 45), ResourceInformation.VCORES.getName());
     verifyCPUShares(20, 5, 5, 15);
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index 446b6ee..0ef4d7b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -79,7 +79,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.MockRMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
@@ -1984,7 +1983,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         // assert that the steady fair share is 1/4th node1's capacity
         assertEquals(capacity / 4, leaf.getSteadyFairShare().getMemorySize());
         // assert weights are equal for both the user queues
-        assertEquals(1.0, leaf.getWeights().getWeight(ResourceType.MEMORY), 0);
+        assertEquals(1.0, leaf.getWeight(), 0);
       }
     }
   }
@@ -5275,7 +5274,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     child1.updateDemand();
 
     String childQueueString = "{Name: root.parent.child1,"
-        + " Weight: <memory weight=1.0, cpu weight=1.0>,"
+        + " Weight: 1.0,"
         + " Policy: fair,"
         + " FairShare: <memory:0, vCores:0>,"
         + " SteadyFairShare: <memory:0, vCores:0>,"
@@ -5292,14 +5291,15 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         + " LastTimeAtMinShare: " + clock.getTime()
         + "}";
 
-    assertTrue(child1.dumpState().equals(childQueueString));
+    assertEquals("Unexpected state dump string",
+        childQueueString, child1.dumpState());
     FSParentQueue parent =
         scheduler.getQueueManager().getParentQueue("parent", false);
     parent.setMaxShare(resource);
     parent.updateDemand();
 
     String parentQueueString = "{Name: root.parent,"
-        + " Weight: <memory weight=1.0, cpu weight=1.0>,"
+        + " Weight: 1.0,"
         + " Policy: fair,"
         + " FairShare: <memory:0, vCores:0>,"
         + " SteadyFairShare: <memory:0, vCores:0>,"
@@ -5310,7 +5310,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         + " MaxAMShare: 0.5,"
         + " Runnable: 0}";
 
-    assertTrue(parent.dumpState().equals(
-        parentQueueString + ", " + childQueueString));
+    assertEquals("Unexpected state dump string",
+        parentQueueString + ", " + childQueueString, parent.dumpState());
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java
index 3a16454..b016c1b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestSchedulingPolicy.java
@@ -30,7 +30,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FairSharePolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
@@ -134,11 +133,7 @@ public class TestSchedulingPolicy {
         Resource.newInstance(0, 1), Resource.newInstance(2, 1),
         Resource.newInstance(4, 1) };
 
-    private ResourceWeights[] weightsCollection = {
-        new ResourceWeights(0.0f), new ResourceWeights(1.0f),
-        new ResourceWeights(2.0f) };
-
-
+    private float[] weightsCollection = {0.0f, 1.0f, 2.0f};
 
     public FairShareComparatorTester(
         Comparator<Schedulable> fairShareComparator) {
@@ -225,10 +220,10 @@ public class TestSchedulingPolicy {
       private String name;
       private long startTime;
       private Resource usage;
-      private ResourceWeights weights;
+      private float weights;
 
       public MockSchedulable(Resource minShare, Resource demand, String name,
-          long startTime, Resource usage, ResourceWeights weights) {
+          long startTime, Resource usage, float weights) {
         this.minShare = minShare;
         this.demand = demand;
         this.name = name;
@@ -258,7 +253,7 @@ public class TestSchedulingPolicy {
       }
 
       @Override
-      public ResourceWeights getWeights() {
+      public float getWeight() {
         return weights;
       }
 


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


[37/50] [abbrv] hadoop git commit: YARN-6977. Node information is not provided for non am containers in RM logs. (Suma Shivaprasad via wangda)

Posted by ae...@apache.org.
YARN-6977. Node information is not provided for non am containers in RM logs. (Suma Shivaprasad via wangda)

Change-Id: I0c44d09a560446dee2ba68c2b9ae69fce0ec1d3e
(cherry picked from commit 8a42e922fad613f3cf1cc6cb0f3fa72546a9cc56)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/958e8c0e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/958e8c0e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/958e8c0e

Branch: refs/heads/HDFS-7240
Commit: 958e8c0e257216c82f68fee726e5280a919da94a
Parents: ef8cd5d
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Sep 15 21:24:11 2017 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Sep 15 21:29:31 2017 -0700

----------------------------------------------------------------------
 .../resourcemanager/scheduler/SchedulerNode.java       |  8 --------
 .../scheduler/common/fica/FiCaSchedulerNode.java       | 13 +++++++++++++
 .../scheduler/fair/FSSchedulerNode.java                | 10 ++++++++++
 3 files changed, 23 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/958e8c0e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
index 272537c..90fa3e4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
@@ -170,14 +170,6 @@ public abstract class SchedulerNode {
 
     launchedContainers.put(container.getId(),
         new ContainerInfo(rmContainer, launchedOnNode));
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Assigned container " + container.getId() + " of capacity "
-              + container.getResource() + " on host " + rmNode.getNodeAddress()
-              + ", which has " + numContainers + " containers, "
-              + getAllocatedResource() + " used and " + getUnallocatedResource()
-              + " available after allocation");
-    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/958e8c0e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
index c26a11b..7277779 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
@@ -160,4 +160,17 @@ public class FiCaSchedulerNode extends SchedulerNode {
   public synchronized Map<ContainerId, RMContainer> getKillableContainers() {
     return Collections.unmodifiableMap(killableContainers);
   }
+
+  protected synchronized void allocateContainer(RMContainer rmContainer,
+      boolean launchedOnNode) {
+    super.allocateContainer(rmContainer, launchedOnNode);
+
+    final Container container = rmContainer.getContainer();
+    LOG.info("Assigned container " + container.getId() + " of capacity "
+          + container.getResource() + " on host " + getRMNode().getNodeAddress()
+          + ", which has " + getNumContainers() + " containers, "
+          + getAllocatedResource() + " used and " + getUnallocatedResource()
+          + " available after allocation");
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/958e8c0e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java
index 93646f4..44ec9c3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java
@@ -25,6 +25,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -239,6 +240,15 @@ public class FSSchedulerNode extends SchedulerNode {
   protected synchronized void allocateContainer(RMContainer rmContainer,
                                                 boolean launchedOnNode) {
     super.allocateContainer(rmContainer, launchedOnNode);
+    if (LOG.isDebugEnabled()) {
+      final Container container = rmContainer.getContainer();
+      LOG.debug("Assigned container " + container.getId() + " of capacity "
+          + container.getResource() + " on host " + getRMNode().getNodeAddress()
+          + ", which has " + getNumContainers() + " containers, "
+          + getAllocatedResource() + " used and " + getUnallocatedResource()
+          + " available after allocation");
+    }
+
     Resource allocated = rmContainer.getAllocatedResource();
     if (!Resources.isNone(allocated)) {
       // check for satisfied preemption request and update bookkeeping


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


[14/50] [abbrv] hadoop git commit: HDFS-7859. Erasure Coding: Persist erasure coding policies in NameNode. Contributed by Sammi Chen

Posted by ae...@apache.org.
HDFS-7859. Erasure Coding: Persist erasure coding policies in NameNode. Contributed by Sammi Chen


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ae8f55b9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ae8f55b9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ae8f55b9

Branch: refs/heads/HDFS-7240
Commit: ae8f55b93243560bd891962d6c64320ddc62a7d7
Parents: 61cee3a
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Sep 15 09:08:18 2017 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Fri Sep 15 09:08:18 2017 +0800

----------------------------------------------------------------------
 .../namenode/ErasureCodingPolicyManager.java    |  38 +++++
 .../server/namenode/FSImageFormatPBINode.java   |   6 +-
 .../server/namenode/FSImageFormatProtobuf.java  |  48 +++++-
 .../namenode/startupprogress/StepType.java      |   7 +-
 .../hadoop-hdfs/src/main/proto/fsimage.proto    |   4 +
 .../hadoop/hdfs/TestErasureCodingPolicies.java  |   5 -
 .../hdfs/server/namenode/TestFSImage.java       | 151 +++++++++++++++++++
 7 files changed, 249 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae8f55b9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
index 77deb85..3a46c30 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
@@ -289,6 +290,11 @@ public final class ErasureCodingPolicyManager {
     }
     ecPolicy.setState(ErasureCodingPolicyState.REMOVED);
     LOG.info("Remove erasure coding policy " + name);
+
+    /*
+     * TODO HDFS-12405 postpone the delete removed policy to Namenode restart
+     * time.
+     * */
   }
 
   @VisibleForTesting
@@ -338,4 +344,36 @@ public final class ErasureCodingPolicyManager {
         enabledPoliciesByName.values().toArray(new ErasureCodingPolicy[0]);
     LOG.info("Enable the erasure coding policy " + name);
   }
+
+  /**
+   * Load an erasure coding policy into erasure coding manager.
+   */
+  private void loadPolicy(ErasureCodingPolicy policy) {
+    if (!CodecUtil.hasCodec(policy.getCodecName()) ||
+        policy.getCellSize() > maxCellSize) {
+      // If policy is not supported in current system, set the policy state to
+      // DISABLED;
+      policy.setState(ErasureCodingPolicyState.DISABLED);
+    }
+
+    this.policiesByName.put(policy.getName(), policy);
+    this.policiesByID.put(policy.getId(), policy);
+    if (policy.isEnabled()) {
+      enablePolicy(policy.getName());
+    }
+  }
+
+  /**
+   * Reload erasure coding policies from fsImage.
+   *
+   * @param ecPolicies contains ErasureCodingPolicy list
+   *
+   */
+  public synchronized void loadPolicies(List<ErasureCodingPolicy> ecPolicies) {
+    Preconditions.checkNotNull(ecPolicies);
+    for (ErasureCodingPolicy p : ecPolicies) {
+      loadPolicy(p);
+    }
+    allPolicies = policiesByName.values().toArray(new ErasureCodingPolicy[0]);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae8f55b9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
index 9f8be89..5e60038 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
@@ -334,10 +334,10 @@ public final class FSImageFormatPBINode {
       boolean isStriped = f.hasErasureCodingPolicyID();
       assert ((!isStriped) || (isStriped && !f.hasReplication()));
       Short replication = (!isStriped ? (short) f.getReplication() : null);
+      Byte ecPolicyID = (isStriped ?
+          (byte) f.getErasureCodingPolicyID() : null);
       ErasureCodingPolicy ecPolicy = isStriped ?
-          fsn.getErasureCodingPolicyManager().getByID(
-              (byte) f.getErasureCodingPolicyID()) : null;
-      Byte ecPolicyID = (isStriped ? ecPolicy.getId() : null);
+          fsn.getErasureCodingPolicyManager().getByID(ecPolicyID) : null;
 
       BlockInfo[] blocks = new BlockInfo[bp.size()];
       for (int i = 0; i < bp.size(); ++i) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae8f55b9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
index 22331fe..ad8cdfc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
@@ -36,10 +36,13 @@ import java.security.MessageDigest;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.io.compress.CompressionOutputStream;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -47,6 +50,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -55,6 +59,7 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.NameSystemSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SecretManagerSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.StringTableSection;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.ErasureCodingSection;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FSImageFormatPBSnapshot;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
@@ -287,6 +292,12 @@ public final class FSImageFormatProtobuf {
           prog.endStep(Phase.LOADING_FSIMAGE, step);
         }
           break;
+        case ERASURE_CODING:
+          Step step = new Step(StepType.ERASURE_CODING_POLICIES);
+          prog.beginStep(Phase.LOADING_FSIMAGE, step);
+          loadErasureCodingSection(in);
+          prog.endStep(Phase.LOADING_FSIMAGE, step);
+          break;
         default:
           LOG.warn("Unrecognized section {}", n);
           break;
@@ -366,6 +377,17 @@ public final class FSImageFormatProtobuf {
           new CacheManager.PersistState(s, pools, directives));
     }
 
+    private void loadErasureCodingSection(InputStream in)
+        throws IOException {
+      ErasureCodingSection s = ErasureCodingSection.parseDelimitedFrom(in);
+      List<ErasureCodingPolicy> ecPolicies = Lists
+          .newArrayListWithCapacity(s.getPoliciesCount());
+      for (int i = 0; i < s.getPoliciesCount(); ++i) {
+        ecPolicies.add(PBHelperClient.convertErasureCodingPolicy(
+            s.getPolicies(i)));
+      }
+      fsn.getErasureCodingPolicyManager().loadPolicies(ecPolicies);
+    }
   }
 
   public static final class Saver {
@@ -497,7 +519,13 @@ public final class FSImageFormatProtobuf {
       // depends on this behavior.
       context.checkCancelled();
 
-      Step step = new Step(StepType.INODES, filePath);
+      // Erasure coding policies should be saved before inodes
+      Step step = new Step(StepType.ERASURE_CODING_POLICIES, filePath);
+      prog.beginStep(Phase.SAVING_CHECKPOINT, step);
+      saveErasureCodingSection(b);
+      prog.endStep(Phase.SAVING_CHECKPOINT, step);
+
+      step = new Step(StepType.INODES, filePath);
       prog.beginStep(Phase.SAVING_CHECKPOINT, step);
       saveInodes(b);
       saveSnapshots(b);
@@ -555,6 +583,23 @@ public final class FSImageFormatProtobuf {
       commitSection(summary, SectionName.CACHE_MANAGER);
     }
 
+    private void saveErasureCodingSection(
+        FileSummary.Builder summary) throws IOException {
+      final FSNamesystem fsn = context.getSourceNamesystem();
+      ErasureCodingPolicy[] ecPolicies =
+          fsn.getErasureCodingPolicyManager().getPolicies();
+      ArrayList<ErasureCodingPolicyProto> ecPolicyProtoes =
+          new ArrayList<ErasureCodingPolicyProto>();
+      for (ErasureCodingPolicy p : ecPolicies) {
+        ecPolicyProtoes.add(PBHelperClient.convertErasureCodingPolicy(p));
+      }
+
+      ErasureCodingSection section = ErasureCodingSection.newBuilder().
+          addAllPolicies(ecPolicyProtoes).build();
+      section.writeDelimitedTo(sectionOutputStream);
+      commitSection(summary, SectionName.ERASURE_CODING);
+    }
+
     private void saveNameSystemSection(FileSummary.Builder summary)
         throws IOException {
       final FSNamesystem fsn = context.getSourceNamesystem();
@@ -606,6 +651,7 @@ public final class FSImageFormatProtobuf {
     NS_INFO("NS_INFO"),
     STRING_TABLE("STRING_TABLE"),
     EXTENDED_ACL("EXTENDED_ACL"),
+    ERASURE_CODING("ERASURE_CODING"),
     INODE("INODE"),
     INODE_REFERENCE("INODE_REFERENCE"),
     SNAPSHOT("SNAPSHOT"),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae8f55b9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StepType.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StepType.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StepType.java
index 1b43d6a..83cf6cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StepType.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StepType.java
@@ -52,7 +52,12 @@ public enum StepType {
   /**
    * The namenode is performing an operation related to cache entries.
    */
-  CACHE_ENTRIES("CacheEntries", "cache entries");
+  CACHE_ENTRIES("CacheEntries", "cache entries"),
+
+  /**
+   * The namenode is performing an operation related to erasure coding policies.
+   */
+  ERASURE_CODING_POLICIES("ErasureCodingPolicies", "erasure coding policies");
 
   private final String name, description;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae8f55b9/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
index 4e21310..101a060 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
@@ -346,3 +346,7 @@ message CacheManagerSection {
   // repeated CachePoolInfoProto pools
   // repeated CacheDirectiveInfoProto directives
 }
+
+message ErasureCodingSection {
+  repeated ErasureCodingPolicyProto policies = 1;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae8f55b9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
index 3c549b1..19277c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
@@ -209,11 +209,6 @@ public class TestErasureCodingPolicies {
     cluster.restartNameNodes();
     cluster.waitActive();
 
-    // Only default policy should be enabled after restart
-    Assert.assertEquals("Only default policy should be enabled after restart",
-        1,
-        ErasureCodingPolicyManager.getInstance().getEnabledPolicies().length);
-
     // Already set directory-level policies should still be in effect
     Path disabledPolicy = new Path(dir1, "afterDisabled");
     Assert.assertEquals("Dir does not have policy set",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ae8f55b9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index 16f6258..c9d3255 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -34,7 +35,9 @@ import java.io.IOException;
 import java.util.EnumSet;
 
 import org.apache.hadoop.hdfs.StripedFileTestUtil;
+import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
@@ -43,6 +46,8 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.ipc.RemoteException;
 import org.junit.Assert;
 
 import org.apache.hadoop.fs.permission.PermissionStatus;
@@ -810,4 +815,150 @@ public class TestFSImage {
       }
     }
   }
+
+  /**
+   * Test persist and load erasure coding policies.
+   */
+  @Test
+  public void testSaveAndLoadErasureCodingPolicies() throws IOException{
+    Configuration conf = new Configuration();
+    final int blockSize = 16 * 1024 * 1024;
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    try (MiniDFSCluster cluster =
+             new MiniDFSCluster.Builder(conf).numDataNodes(10).build()) {
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      DFSTestUtil.enableAllECPolicies(fs);
+
+      // Save namespace and restart NameNode
+      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.saveNamespace();
+      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+
+      cluster.restartNameNodes();
+      cluster.waitActive();
+
+      assertEquals("Erasure coding policy number should match",
+          SystemErasureCodingPolicies.getPolicies().size(),
+          ErasureCodingPolicyManager.getInstance().getPolicies().length);
+
+      // Add new erasure coding policy
+      ECSchema newSchema = new ECSchema("rs", 5, 4);
+      ErasureCodingPolicy newPolicy =
+          new ErasureCodingPolicy(newSchema, 2 * 1024, (byte) 254);
+      ErasureCodingPolicy[] policies = new ErasureCodingPolicy[]{newPolicy};
+      AddECPolicyResponse[] ret = fs.addErasureCodingPolicies(policies);
+      assertEquals(1, ret.length);
+      assertEquals(true, ret[0].isSucceed());
+      newPolicy = ret[0].getPolicy();
+
+      // Save namespace and restart NameNode
+      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.saveNamespace();
+      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+
+      cluster.restartNameNodes();
+      cluster.waitActive();
+
+      assertEquals("Erasure coding policy number should match",
+          SystemErasureCodingPolicies.getPolicies().size() + 1,
+          ErasureCodingPolicyManager.getInstance().getPolicies().length);
+      ErasureCodingPolicy ecPolicy =
+          ErasureCodingPolicyManager.getInstance().getByID(newPolicy.getId());
+      assertEquals("Newly added erasure coding policy is not found",
+          newPolicy, ecPolicy);
+      assertEquals(
+          "Newly added erasure coding policy should be of disabled state",
+          ErasureCodingPolicyState.DISABLED, ecPolicy.getState());
+
+      // Test enable/disable/remove user customized erasure coding policy
+      testChangeErasureCodingPolicyState(cluster, blockSize, newPolicy);
+      // Test enable/disable built-in erasure coding policy
+      testChangeErasureCodingPolicyState(cluster, blockSize,
+          SystemErasureCodingPolicies.getByID((byte) 1));
+    }
+  }
+
+
+  private void testChangeErasureCodingPolicyState(MiniDFSCluster cluster,
+      int blockSize, ErasureCodingPolicy targetPolicy) throws IOException {
+    DistributedFileSystem fs = cluster.getFileSystem();
+
+    // 1. Enable an erasure coding policy
+    fs.enableErasureCodingPolicy(targetPolicy.getName());
+    targetPolicy.setState(ErasureCodingPolicyState.ENABLED);
+    // Create file, using the new policy
+    final Path dirPath = new Path("/striped");
+    final Path filePath = new Path(dirPath, "file");
+    final int fileLength = blockSize * targetPolicy.getNumDataUnits();
+    fs.mkdirs(dirPath);
+    fs.setErasureCodingPolicy(dirPath, targetPolicy.getName());
+    final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength);
+    DFSTestUtil.writeFile(fs, filePath, bytes);
+
+
+    // Save namespace and restart NameNode
+    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.saveNamespace();
+    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+
+    cluster.restartNameNodes();
+    cluster.waitActive();
+    ErasureCodingPolicy ecPolicy =
+        ErasureCodingPolicyManager.getInstance().getByID(targetPolicy.getId());
+    assertEquals("The erasure coding policy is not found",
+        targetPolicy, ecPolicy);
+    assertEquals("The erasure coding policy should be of enabled state",
+        ErasureCodingPolicyState.ENABLED, ecPolicy.getState());
+    // Read file regardless of the erasure coding policy state
+    DFSTestUtil.readFileAsBytes(fs, filePath);
+
+    // 2. Disable an erasure coding policy
+    fs.disableErasureCodingPolicy(ecPolicy.getName());
+    targetPolicy.setState(ErasureCodingPolicyState.DISABLED);
+    // Save namespace and restart NameNode
+    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.saveNamespace();
+    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+
+    cluster.restartNameNodes();
+    cluster.waitActive();
+    ecPolicy =
+        ErasureCodingPolicyManager.getInstance().getByID(targetPolicy.getId());
+    assertEquals("The erasure coding policy is not found",
+        targetPolicy, ecPolicy);
+    assertEquals("The erasure coding policy should be of disabled state",
+        ErasureCodingPolicyState.DISABLED, ecPolicy.getState());
+    // Read file regardless of the erasure coding policy state
+    DFSTestUtil.readFileAsBytes(fs, filePath);
+
+    // 3. Remove an erasure coding policy
+    try {
+      fs.removeErasureCodingPolicy(ecPolicy.getName());
+    } catch (RemoteException e) {
+      // built-in policy cannot been removed
+      assertTrue("Built-in policy cannot be removed",
+          ecPolicy.isSystemPolicy());
+      assertExceptionContains("System erasure coding policy", e);
+      return;
+    }
+
+    targetPolicy.setState(ErasureCodingPolicyState.REMOVED);
+    // Save namespace and restart NameNode
+    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.saveNamespace();
+    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+
+    cluster.restartNameNodes();
+    cluster.waitActive();
+    ecPolicy = ErasureCodingPolicyManager.getInstance().getByID(
+        targetPolicy.getId());
+    assertEquals("The erasure coding policy saved into and loaded from " +
+        "fsImage is bad", targetPolicy, ecPolicy);
+    assertEquals("The erasure coding policy should be of removed state",
+        ErasureCodingPolicyState.REMOVED, ecPolicy.getState());
+    // Read file regardless of the erasure coding policy state
+    DFSTestUtil.readFileAsBytes(fs, filePath);
+    fs.delete(dirPath, true);
+  }
 }


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


[43/50] [abbrv] hadoop git commit: YARN-7192. Add a pluggable StateMachine Listener that is notified of NM Container State changes. Contributed by Arun Suresh

Posted by ae...@apache.org.
YARN-7192. Add a pluggable StateMachine Listener that is notified of NM Container State changes. Contributed by Arun Suresh


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a4f9c7c9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a4f9c7c9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a4f9c7c9

Branch: refs/heads/HDFS-7240
Commit: a4f9c7c9247801dd37beec6fc195622af1b884ad
Parents: 0f9af24
Author: Jason Lowe <jl...@apache.org>
Authored: Mon Sep 18 10:16:09 2017 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Mon Sep 18 10:16:09 2017 -0500

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  6 +-
 .../state/MultiStateTransitionListener.java     | 61 ++++++++++++++++++
 .../hadoop/yarn/state/StateMachineFactory.java  | 40 ++++++++++++
 .../yarn/state/StateTransitionListener.java     | 50 ++++++++++++++
 .../src/main/resources/yarn-default.xml         |  6 ++
 .../ContainerStateTransitionListener.java       | 48 ++++++++++++++
 .../hadoop/yarn/server/nodemanager/Context.java |  2 +
 .../yarn/server/nodemanager/NodeManager.java    | 48 +++++++++++++-
 .../container/ContainerImpl.java                |  3 +-
 .../server/nodemanager/TestNodeManager.java     | 68 ++++++++++++++++++++
 .../amrmproxy/BaseAMRMProxyTest.java            |  8 +++
 .../container/TestContainer.java                | 53 +++++++++++++++
 12 files changed, 389 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 48910b3..114453f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -968,9 +968,13 @@ public class YarnConfiguration extends Configuration {
     NM_PREFIX + "bind-host";
 
   /** who will execute(launch) the containers.*/
-  public static final String NM_CONTAINER_EXECUTOR = 
+  public static final String NM_CONTAINER_EXECUTOR =
     NM_PREFIX + "container-executor.class";
 
+  /** List of container state transition listeners.*/
+  public static final String NM_CONTAINER_STATE_TRANSITION_LISTENERS =
+      NM_PREFIX + "container-state-transition-listener.classes";
+
   /**  
    * Adjustment to make to the container os scheduling priority.
    * The valid values for this could vary depending on the platform.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/MultiStateTransitionListener.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/MultiStateTransitionListener.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/MultiStateTransitionListener.java
new file mode 100644
index 0000000..1a28fc5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/MultiStateTransitionListener.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.state;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A {@link StateTransitionListener} that dispatches the pre and post
+ * state transitions to multiple registered listeners.
+ * NOTE: The registered listeners are called in a for loop. Clients should
+ *       know that a listener configured earlier might prevent a later listener
+ *       from being called, if for instance it throws an un-caught Exception.
+ */
+public abstract class MultiStateTransitionListener
+    <OPERAND, EVENT, STATE extends Enum<STATE>> implements
+    StateTransitionListener<OPERAND, EVENT, STATE> {
+
+  private final List<StateTransitionListener<OPERAND, EVENT, STATE>> listeners =
+      new ArrayList<>();
+
+  /**
+   * Add a listener to the list of listeners.
+   * @param listener A listener.
+   */
+  public void addListener(StateTransitionListener<OPERAND, EVENT, STATE>
+      listener) {
+    listeners.add(listener);
+  }
+
+  @Override
+  public void preTransition(OPERAND op, STATE beforeState,
+      EVENT eventToBeProcessed) {
+    for (StateTransitionListener<OPERAND, EVENT, STATE> listener : listeners) {
+      listener.preTransition(op, beforeState, eventToBeProcessed);
+    }
+  }
+
+  @Override
+  public void postTransition(OPERAND op, STATE beforeState, STATE afterState,
+      EVENT processedEvent) {
+    for (StateTransitionListener<OPERAND, EVENT, STATE> listener : listeners) {
+      listener.postTransition(op, beforeState, afterState, processedEvent);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java
index 5b76ce8..4bb005c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java
@@ -391,6 +391,21 @@ final public class StateMachineFactory
     }
   }
 
+  /**
+   * A StateMachine that accepts a transition listener.
+   * @param operand the object upon which the returned
+   *                {@link StateMachine} will operate.
+   * @param initialState the state in which the returned
+   *                {@link StateMachine} will start.
+   * @param listener An implementation of a {@link StateTransitionListener}.
+   * @return A (@link StateMachine}.
+   */
+  public StateMachine<STATE, EVENTTYPE, EVENT>
+        make(OPERAND operand, STATE initialState,
+             StateTransitionListener<OPERAND, EVENT, STATE> listener) {
+    return new InternalStateMachine(operand, initialState, listener);
+  }
+
   /* 
    * @return a {@link StateMachine} that starts in 
    *         {@code initialState} and whose {@link Transition} s are
@@ -424,14 +439,36 @@ final public class StateMachineFactory
     return new InternalStateMachine(operand, defaultInitialState);
   }
 
+  private static class NoopStateTransitionListener
+      implements StateTransitionListener {
+    @Override
+    public void preTransition(Object op, Enum beforeState,
+        Object eventToBeProcessed) { }
+
+    @Override
+    public void postTransition(Object op, Enum beforeState, Enum afterState,
+        Object processedEvent) { }
+  }
+
+  private static final NoopStateTransitionListener NOOP_LISTENER =
+      new NoopStateTransitionListener();
+
   private class InternalStateMachine
         implements StateMachine<STATE, EVENTTYPE, EVENT> {
     private final OPERAND operand;
     private STATE currentState;
+    private final StateTransitionListener<OPERAND, EVENT, STATE> listener;
 
     InternalStateMachine(OPERAND operand, STATE initialState) {
+      this(operand, initialState, null);
+    }
+
+    InternalStateMachine(OPERAND operand, STATE initialState,
+        StateTransitionListener<OPERAND, EVENT, STATE> transitionListener) {
       this.operand = operand;
       this.currentState = initialState;
+      this.listener =
+          (transitionListener == null) ? NOOP_LISTENER : transitionListener;
       if (!optimized) {
         maybeMakeStateMachineTable();
       }
@@ -445,8 +482,11 @@ final public class StateMachineFactory
     @Override
     public synchronized STATE doTransition(EVENTTYPE eventType, EVENT event)
          throws InvalidStateTransitionException  {
+      listener.preTransition(operand, currentState, event);
+      STATE oldState = currentState;
       currentState = StateMachineFactory.this.doTransition
           (operand, currentState, eventType, event);
+      listener.postTransition(operand, oldState, currentState, event);
       return currentState;
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateTransitionListener.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateTransitionListener.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateTransitionListener.java
new file mode 100644
index 0000000..657c193
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateTransitionListener.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.state;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A State Transition Listener.
+ * It exposes a pre and post transition hook called before and
+ * after the transition.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface StateTransitionListener
+    <OPERAND, EVENT, STATE extends Enum<STATE>> {
+
+  /**
+   * Pre Transition Hook. This will be called before transition.
+   * @param op Operand.
+   * @param beforeState State before transition.
+   * @param eventToBeProcessed Incoming Event.
+   */
+  void preTransition(OPERAND op, STATE beforeState, EVENT eventToBeProcessed);
+
+  /**
+   * Post Transition Hook. This will be called after the transition.
+   * @param op Operand.
+   * @param beforeState State before transition.
+   * @param afterState State after transition.
+   * @param processedEvent Processed Event.
+   */
+  void postTransition(OPERAND op, STATE beforeState, STATE afterState,
+      EVENT processedEvent);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 6444da9..0440458 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -1004,6 +1004,12 @@
   </property>
 
   <property>
+    <description>Comma separated List of container state transition listeners.</description>
+    <name>yarn.nodemanager.container-state-transition-listener.classes</name>
+    <value></value>
+  </property>
+
+  <property>
     <description>Number of threads container manager uses.</description>
     <name>yarn.nodemanager.container-manager.thread-count</name>
     <value>20</value>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerStateTransitionListener.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerStateTransitionListener.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerStateTransitionListener.java
new file mode 100644
index 0000000..24cdb1f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerStateTransitionListener.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.nodemanager;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
+import org.apache.hadoop.yarn.state.StateTransitionListener;
+
+/**
+ * Interface to be used by external cluster operators to implement a
+ * State Transition listener that is notified before and after a container
+ * state transition.
+ * NOTE: The pre and post transition callbacks will be made in the synchronized
+ *       block as the call to the instrumented transition - Serially, in the
+ *       order: preTransition, transition and postTransition. The implementor
+ *       must ensure that the callbacks return in a timely manner to avoid
+ *       blocking the state-machine.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface ContainerStateTransitionListener extends
+    StateTransitionListener<ContainerImpl, ContainerEvent, ContainerState> {
+
+  /**
+   * Init method which will be invoked by the Node Manager to inject the
+   * NM {@link Context}.
+   * @param context NM Context.
+   */
+  void init(Context context);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
index 00bd0ef..a2d00a4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
@@ -120,4 +120,6 @@ public interface Context {
   NMTimelinePublisher getNMTimelinePublisher();
 
   ContainerExecutor getContainerExecutor();
+
+  ContainerStateTransitionListener getContainerStateTransitionListener();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index 3e919c5..a97b3f2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -20,12 +20,18 @@ package org.apache.hadoop.yarn.server.nodemanager;
 
 import java.io.IOException;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
+import org.apache.hadoop.yarn.state.MultiStateTransitionListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -130,6 +136,17 @@ public class NodeManager extends CompositeService
   private boolean rmWorkPreservingRestartEnabled;
   private boolean shouldExitOnShutdownEvent = false;
 
+  /**
+   * Default Container State transition listener.
+   */
+  public static class DefaultContainerStateListener extends
+      MultiStateTransitionListener
+          <ContainerImpl, ContainerEvent, ContainerState>
+      implements ContainerStateTransitionListener {
+    @Override
+    public void init(Context context) {}
+  }
+
   public NodeManager() {
     super(NodeManager.class.getName());
   }
@@ -219,8 +236,22 @@ public class NodeManager extends CompositeService
       NMTokenSecretManagerInNM nmTokenSecretManager,
       NMStateStoreService stateStore, boolean isDistSchedulerEnabled,
       Configuration conf) {
-    return new NMContext(containerTokenSecretManager, nmTokenSecretManager,
-        dirsHandler, aclsManager, stateStore, isDistSchedulerEnabled, conf);
+    List<ContainerStateTransitionListener> listeners =
+        conf.getInstances(
+            YarnConfiguration.NM_CONTAINER_STATE_TRANSITION_LISTENERS,
+        ContainerStateTransitionListener.class);
+    NMContext nmContext = new NMContext(containerTokenSecretManager,
+        nmTokenSecretManager, dirsHandler, aclsManager, stateStore,
+        isDistSchedulerEnabled, conf);
+    DefaultContainerStateListener defaultListener =
+        new DefaultContainerStateListener();
+    nmContext.setContainerStateTransitionListener(defaultListener);
+    defaultListener.init(nmContext);
+    for (ContainerStateTransitionListener listener : listeners) {
+      listener.init(nmContext);
+      defaultListener.addListener(listener);
+    }
+    return nmContext;
   }
 
   protected void doSecureLogin() throws IOException {
@@ -563,6 +594,8 @@ public class NodeManager extends CompositeService
 
     private NMTimelinePublisher nmTimelinePublisher;
 
+    private ContainerStateTransitionListener containerStateTransitionListener;
+
     public NMContext(NMContainerTokenSecretManager containerTokenSecretManager,
         NMTokenSecretManagerInNM nmTokenSecretManager,
         LocalDirsHandlerService dirsHandler, ApplicationACLsManager aclsManager,
@@ -752,6 +785,17 @@ public class NodeManager extends CompositeService
     public void setContainerExecutor(ContainerExecutor executor) {
       this.executor = executor;
     }
+
+    @Override
+    public ContainerStateTransitionListener
+        getContainerStateTransitionListener() {
+      return this.containerStateTransitionListener;
+    }
+
+    public void setContainerStateTransitionListener(
+        ContainerStateTransitionListener transitionListener) {
+      this.containerStateTransitionListener = transitionListener;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index 9b9c47f..df107a7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -240,7 +240,8 @@ public class ContainerImpl implements Container {
     this.containerRetryContext = configureRetryContext(
         conf, launchContext, this.containerId);
     this.remainingRetryAttempts = this.containerRetryContext.getMaxRetries();
-    stateMachine = stateMachineFactory.make(this);
+    stateMachine = stateMachineFactory.make(this, ContainerState.NEW,
+        context.getContainerStateTransitionListener());
     this.context = context;
     this.resourceSet = new ResourceSet();
     this.resourceMappings = new ResourceMappings();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManager.java
index 2d390ac..9279711 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManager.java
@@ -25,6 +25,9 @@ import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
 import org.apache.hadoop.yarn.server.nodemanager.nodelabels.NodeLabelsProvider;
 import org.junit.Assert;
 import org.junit.Test;
@@ -57,6 +60,71 @@ public class TestNodeManager {
     }
   }
 
+  private static int initCalls = 0;
+  private static int preCalls = 0;
+  private static int postCalls = 0;
+
+  private static class DummyCSTListener1
+      implements ContainerStateTransitionListener {
+    @Override
+    public void init(Context context) {
+      initCalls++;
+    }
+
+    @Override
+    public void preTransition(ContainerImpl op, ContainerState beforeState,
+        ContainerEvent eventToBeProcessed) {
+      preCalls++;
+    }
+
+    @Override
+    public void postTransition(ContainerImpl op, ContainerState beforeState,
+        ContainerState afterState, ContainerEvent processedEvent) {
+      postCalls++;
+    }
+  }
+
+  private static class DummyCSTListener2
+      implements ContainerStateTransitionListener {
+    @Override
+    public void init(Context context) {
+      initCalls++;
+    }
+
+    @Override
+    public void preTransition(ContainerImpl op, ContainerState beforeState,
+        ContainerEvent eventToBeProcessed) {
+      preCalls++;
+    }
+
+    @Override
+    public void postTransition(ContainerImpl op, ContainerState beforeState,
+        ContainerState afterState, ContainerEvent processedEvent) {
+      postCalls++;
+    }
+  }
+
+  @Test
+  public void testListenerInitialization() throws Exception{
+    NodeManager nodeManager = new NodeManager();
+    Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.NM_CONTAINER_STATE_TRANSITION_LISTENERS,
+        DummyCSTListener1.class.getName() + ","
+            + DummyCSTListener2.class.getName());
+    initCalls = 0;
+    preCalls = 0;
+    postCalls = 0;
+    NodeManager.NMContext nmContext =
+        nodeManager.createNMContext(null, null, null, false, conf);
+    Assert.assertEquals(2, initCalls);
+    nmContext.getContainerStateTransitionListener().preTransition(
+        null, null, null);
+    nmContext.getContainerStateTransitionListener().postTransition(
+        null, null, null, null);
+    Assert.assertEquals(2, preCalls);
+    Assert.assertEquals(2, postCalls);
+  }
+
   @Test
   public void testCreationOfNodeLabelsProviderService()
       throws InterruptedException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
index 7c8551e..0838f1e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
@@ -33,6 +33,8 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.yarn.server.nodemanager.ContainerStateTransitionListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -765,5 +767,11 @@ public abstract class BaseAMRMProxyTest {
     public ContainerExecutor getContainerExecutor() {
       return null;
     }
+
+    @Override
+    public ContainerStateTransitionListener
+        getContainerStateTransitionListener() {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
index 8909088..64e6cf0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
@@ -71,7 +71,9 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
+import org.apache.hadoop.yarn.server.nodemanager.ContainerStateTransitionListener;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
@@ -287,6 +289,29 @@ public class TestContainer {
       assertEquals(ContainerState.DONE, wc.c.getContainerState());
       assertEquals(completed + 1, metrics.getCompletedContainers());
       assertEquals(running, metrics.getRunningContainers());
+
+      ContainerEventType e1 = wc.initStateToEvent.get(ContainerState.NEW);
+      ContainerState s2 = wc.eventToFinalState.get(e1);
+      ContainerEventType e2 = wc.initStateToEvent.get(s2);
+      ContainerState s3 = wc.eventToFinalState.get(e2);
+      ContainerEventType e3 = wc.initStateToEvent.get(s3);
+      ContainerState s4 = wc.eventToFinalState.get(e3);
+      ContainerEventType e4 = wc.initStateToEvent.get(s4);
+      ContainerState s5 = wc.eventToFinalState.get(e4);
+      ContainerEventType e5 = wc.initStateToEvent.get(s5);
+      ContainerState s6 = wc.eventToFinalState.get(e5);
+
+      Assert.assertEquals(ContainerState.LOCALIZING, s2);
+      Assert.assertEquals(ContainerState.SCHEDULED, s3);
+      Assert.assertEquals(ContainerState.RUNNING, s4);
+      Assert.assertEquals(ContainerState.EXITED_WITH_SUCCESS, s5);
+      Assert.assertEquals(ContainerState.DONE, s6);
+
+      Assert.assertEquals(ContainerEventType.INIT_CONTAINER, e1);
+      Assert.assertEquals(ContainerEventType.RESOURCE_LOCALIZED, e2);
+      Assert.assertEquals(ContainerEventType.CONTAINER_LAUNCHED, e3);
+      Assert.assertEquals(ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS, e4);
+      Assert.assertEquals(ContainerEventType.CONTAINER_RESOURCES_CLEANEDUP, e5);
     }
     finally {
       if (wc != null) {
@@ -401,6 +426,10 @@ public class TestContainer {
       Assert.assertTrue(
           containerMetrics.finishTime.value() > containerMetrics.startTime
               .value());
+      Assert.assertEquals(ContainerEventType.KILL_CONTAINER,
+          wc.initStateToEvent.get(ContainerState.NEW));
+      Assert.assertEquals(ContainerState.DONE,
+          wc.eventToFinalState.get(ContainerEventType.KILL_CONTAINER));
     } finally {
       if (wc != null) {
         wc.finished();
@@ -942,6 +971,10 @@ public class TestContainer {
     final Map<String, LocalResource> localResources;
     final Map<String, ByteBuffer> serviceData;
     final Context context = mock(Context.class);
+    private final Map<ContainerState, ContainerEventType> initStateToEvent =
+        new HashMap<>();
+    private final Map<ContainerEventType, ContainerState> eventToFinalState =
+        new HashMap<>();
 
     WrappedContainer(int appId, long timestamp, int id, String user)
         throws IOException {
@@ -1048,7 +1081,27 @@ public class TestContainer {
       }
       when(ctxt.getServiceData()).thenReturn(serviceData);
       when(ctxt.getContainerRetryContext()).thenReturn(containerRetryContext);
+      ContainerStateTransitionListener listener =
+          new ContainerStateTransitionListener() {
+        @Override
+        public void init(Context cntxt) {}
+
+        @Override
+        public void preTransition(ContainerImpl op, ContainerState beforeState,
+            ContainerEvent eventToBeProcessed) {
+          initStateToEvent.put(beforeState, eventToBeProcessed.getType());
+        }
 
+        @Override
+        public void postTransition(ContainerImpl op, ContainerState beforeState,
+            ContainerState afterState, ContainerEvent processedEvent) {
+          eventToFinalState.put(processedEvent.getType(), afterState);
+        }
+      };
+      NodeManager.DefaultContainerStateListener multi =
+          new NodeManager.DefaultContainerStateListener();
+      multi.addListener(listener);
+      when(context.getContainerStateTransitionListener()).thenReturn(multi);
       c = new ContainerImpl(conf, dispatcher, ctxt, null, metrics, identifier,
           context);
       dispatcher.register(ContainerEventType.class,


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


[32/50] [abbrv] hadoop git commit: HDFS-12349. Improve log message when it could not alloc enough blocks for EC. (Lei (Eddy) Xu)

Posted by ae...@apache.org.
HDFS-12349. Improve log message when it could not alloc enough blocks for EC. (Lei (Eddy) Xu)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fbe06b58
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fbe06b58
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fbe06b58

Branch: refs/heads/HDFS-7240
Commit: fbe06b58805aac4861fb27dfa273914b69e8bdc6
Parents: 3a8d57a
Author: Lei Xu <le...@apache.org>
Authored: Fri Sep 15 12:12:42 2017 -0700
Committer: Lei Xu <le...@apache.org>
Committed: Fri Sep 15 12:12:42 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/DFSStripedOutputStream.java     | 17 ++++++------
 .../server/blockmanagement/BlockManager.java    | 26 ++++++++++++------
 .../hdfs/server/namenode/FSDirWriteFileOp.java  | 24 ++++++++++------
 .../TestDFSStripedOutputStreamWithFailure.java  | 29 ++++++++++----------
 .../datatransfer/sasl/TestSaslDataTransfer.java |  4 +--
 .../blockmanagement/TestBlockManager.java       |  3 +-
 .../blockmanagement/TestBlockStatsMXBean.java   |  2 +-
 .../hdfs/server/namenode/TestDeadDatanode.java  |  4 +--
 8 files changed, 63 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbe06b58/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
index 7f05338..44db3a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@ -260,6 +260,7 @@ public class DFSStripedOutputStream extends DFSOutputStream
 
   private final Coordinator coordinator;
   private final CellBuffers cellBuffers;
+  private final ErasureCodingPolicy ecPolicy;
   private final RawErasureEncoder encoder;
   private final List<StripedDataStreamer> streamers;
   private final DFSPacket[] currentPackets; // current Packet of each streamer
@@ -286,7 +287,7 @@ public class DFSStripedOutputStream extends DFSOutputStream
       LOG.debug("Creating DFSStripedOutputStream for " + src);
     }
 
-    final ErasureCodingPolicy ecPolicy = stat.getErasureCodingPolicy();
+    ecPolicy = stat.getErasureCodingPolicy();
     final int numParityBlocks = ecPolicy.getNumParityUnits();
     cellSize = ecPolicy.getCellSize();
     numDataBlocks = ecPolicy.getNumDataUnits();
@@ -478,11 +479,6 @@ public class DFSStripedOutputStream extends DFSOutputStream
     final LocatedBlock lb = addBlock(excludedNodes, dfsClient, src,
         currentBlockGroup, fileId, favoredNodes, getAddBlockFlags());
     assert lb.isStriped();
-    if (lb.getLocations().length < numDataBlocks) {
-      throw new IOException("Failed to get " + numDataBlocks
-          + " nodes from namenode: blockGroupSize= " + numAllBlocks
-          + ", blocks.length= " + lb.getLocations().length);
-    }
     // assign the new block to the current block group
     currentBlockGroup = lb.getBlock();
     blockGroupIndex++;
@@ -494,11 +490,16 @@ public class DFSStripedOutputStream extends DFSOutputStream
       StripedDataStreamer si = getStripedDataStreamer(i);
       assert si.isHealthy();
       if (blocks[i] == null) {
+        // allocBlock() should guarantee that all data blocks are successfully
+        // allocated.
+        assert i >= numDataBlocks;
         // Set exception and close streamer as there is no block locations
         // found for the parity block.
-        LOG.warn("Failed to get block location for parity block, index=" + i);
+        LOG.warn("Cannot allocate parity block(index={}, policy={}). " +
+            "Not enough datanodes? Exclude nodes={}", i,  ecPolicy.getName(),
+            excludedNodes);
         si.getLastException().set(
-            new IOException("Failed to get following block, i=" + i));
+            new IOException("Failed to get parity block, index=" + i));
         si.getErrorState().setInternalError();
         si.close(true);
       } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbe06b58/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index f4e5cb4..f33ec63 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -2057,6 +2057,7 @@ public class BlockManager implements BlockStatsMXBean {
       final List<String> favoredNodes,
       final byte storagePolicyID,
       final BlockType blockType,
+      final ErasureCodingPolicy ecPolicy,
       final EnumSet<AddBlockFlag> flags) throws IOException {
     List<DatanodeDescriptor> favoredDatanodeDescriptors = 
         getDatanodeDescriptors(favoredNodes);
@@ -2067,14 +2068,23 @@ public class BlockManager implements BlockStatsMXBean {
     final DatanodeStorageInfo[] targets = blockplacement.chooseTarget(src,
         numOfReplicas, client, excludedNodes, blocksize, 
         favoredDatanodeDescriptors, storagePolicy, flags);
-    if (targets.length < minReplication) {
-      throw new IOException("File " + src + " could only be replicated to "
-          + targets.length + " nodes instead of minReplication (="
-          + minReplication + ").  There are "
-          + getDatanodeManager().getNetworkTopology().getNumOfLeaves()
-          + " datanode(s) running and "
-          + (excludedNodes == null? "no": excludedNodes.size())
-          + " node(s) are excluded in this operation.");
+
+    final String errorMessage = "File %s could only be written to %d of " +
+        "the %d %s. There are %d datanode(s) running and %s "
+        + "node(s) are excluded in this operation.";
+    if (blockType == BlockType.CONTIGUOUS && targets.length < minReplication) {
+      throw new IOException(String.format(errorMessage, src,
+          targets.length, minReplication, "minReplication nodes",
+          getDatanodeManager().getNetworkTopology().getNumOfLeaves(),
+          (excludedNodes == null? "no": excludedNodes.size())));
+    } else if (blockType == BlockType.STRIPED &&
+        targets.length < ecPolicy.getNumDataUnits()) {
+      throw new IOException(
+          String.format(errorMessage, src, targets.length,
+              ecPolicy.getNumDataUnits(),
+              String.format("required nodes for %s", ecPolicy.getName()),
+              getDatanodeManager().getNetworkTopology().getNumOfLeaves(),
+              (excludedNodes == null ? "no" : excludedNodes.size())));
     }
     return targets;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbe06b58/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index 012e916..b202212 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -201,7 +201,7 @@ class FSDirWriteFileOp {
     }
     storagePolicyID = pendingFile.getStoragePolicyID();
     return new ValidateAddBlockResult(blockSize, numTargets, storagePolicyID,
-                                      clientMachine, blockType);
+                                      clientMachine, blockType, ecPolicy);
   }
 
   static LocatedBlock makeLocatedBlock(FSNamesystem fsn, BlockInfo blk,
@@ -286,7 +286,7 @@ class FSDirWriteFileOp {
     return bm.chooseTarget4NewBlock(src, r.numTargets, clientNode,
                                     excludedNodesSet, r.blockSize,
                                     favoredNodesList, r.storagePolicyID,
-                                    r.blockType, flags);
+                                    r.blockType, r.ecPolicy, flags);
   }
 
   /**
@@ -831,20 +831,28 @@ class FSDirWriteFileOp {
   }
 
   static class ValidateAddBlockResult {
-    final long blockSize;
-    final int numTargets;
-    final byte storagePolicyID;
-    final String clientMachine;
-    final BlockType blockType;
+    private final long blockSize;
+    private final int numTargets;
+    private final byte storagePolicyID;
+    private final String clientMachine;
+    private final BlockType blockType;
+    private final ErasureCodingPolicy ecPolicy;
 
     ValidateAddBlockResult(
         long blockSize, int numTargets, byte storagePolicyID,
-        String clientMachine, BlockType blockType) {
+        String clientMachine, BlockType blockType,
+        ErasureCodingPolicy ecPolicy) {
       this.blockSize = blockSize;
       this.numTargets = numTargets;
       this.storagePolicyID = storagePolicyID;
       this.clientMachine = clientMachine;
       this.blockType = blockType;
+      this.ecPolicy = ecPolicy;
+
+      if (blockType == BlockType.STRIPED) {
+        Preconditions.checkArgument(ecPolicy != null,
+            "ecPolicy is not specified for striped block");
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbe06b58/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index 231f260..ea889e3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.io.erasurecode.ErasureCodeNative;
 import org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Level;
 import org.junit.Assert;
@@ -284,7 +285,7 @@ public class TestDFSStripedOutputStreamWithFailure {
 
   @Test(timeout = 90000)
   public void testAddBlockWhenNoSufficientDataBlockNumOfNodes()
-      throws IOException {
+      throws Exception {
     HdfsConfiguration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
     try {
@@ -303,20 +304,18 @@ public class TestDFSStripedOutputStreamWithFailure {
           DatanodeReportType.LIVE);
       assertEquals("Mismatches number of live Dns ", numDatanodes, info.length);
       final Path dirFile = new Path(dir, "ecfile");
-      FSDataOutputStream out;
-      try {
-        out = dfs.create(dirFile, true);
-        out.write("something".getBytes());
-        out.flush();
-        out.close();
-        Assert.fail("Failed to validate available dns against blkGroupSize");
-      } catch (IOException ioe) {
-        // expected
-        GenericTestUtils.assertExceptionContains("Failed to get " +
-            dataBlocks + " nodes from namenode: blockGroupSize= " +
-            (dataBlocks + parityBlocks) + ", blocks.length= " +
-            numDatanodes, ioe);
-      }
+      LambdaTestUtils.intercept(
+          IOException.class,
+          "File " + dirFile + " could only be written to " +
+              numDatanodes + " of the " + dataBlocks + " required nodes for " +
+              getEcPolicy().getName(),
+          () -> {
+            try (FSDataOutputStream out = dfs.create(dirFile, true)) {
+              out.write("something".getBytes());
+              out.flush();
+            }
+            return 0;
+          });
     } finally {
       tearDown();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbe06b58/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java
index 8555e5d..2fe0a1c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/TestSaslDataTransfer.java
@@ -118,7 +118,7 @@ public class TestSaslDataTransfer extends SaslDataTransferTestCase {
     HdfsConfiguration clientConf = new HdfsConfiguration(clusterConf);
     clientConf.set(DFS_DATA_TRANSFER_PROTECTION_KEY, "authentication");
     exception.expect(IOException.class);
-    exception.expectMessage("could only be replicated to 0 nodes");
+    exception.expectMessage("could only be written to 0");
     doTest(clientConf);
   }
 
@@ -140,7 +140,7 @@ public class TestSaslDataTransfer extends SaslDataTransferTestCase {
           "configured or not supported in client");
     } catch (IOException e) {
       GenericTestUtils.assertMatches(e.getMessage(), 
-          "could only be replicated to 0 nodes");
+          "could only be written to 0");
     } finally {
       logs.stopCapturing();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbe06b58/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index 4c1ea7b..10289ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -1030,8 +1030,7 @@ public class TestBlockManager {
         		0x1BAD5EED);
       }
       catch (RemoteException re) {
-    	  GenericTestUtils.assertExceptionContains("nodes instead of "
-    	  		+ "minReplication", re);
+        GenericTestUtils.assertExceptionContains("of the 1 minReplication", re);
       }
     }
     finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbe06b58/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
index b7583c4..bcf38d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
@@ -182,7 +182,7 @@ public class TestBlockStatsMXBean {
       fail("Should throw exception, becuase no DISK storage available");
     } catch (Exception e) {
       assertTrue(e.getMessage().contains(
-          "could only be replicated to 0 nodes instead"));
+          "could only be written to 0 of the 1 minReplication"));
     }
     // wait for heartbeat
     Thread.sleep(6000);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbe06b58/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
index 74be90c..b6c1318 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
@@ -175,8 +175,8 @@ public class TestDeadDatanode {
     // choose the targets, but local node should not get selected as this is not
     // part of the cluster anymore
     DatanodeStorageInfo[] results = bm.chooseTarget4NewBlock("/hello", 3,
-        clientNode, new HashSet<Node>(), 256 * 1024 * 1024L, null, (byte) 7,
-        BlockType.CONTIGUOUS, null);
+        clientNode, new HashSet<>(), 256 * 1024 * 1024L, null, (byte) 7,
+        BlockType.CONTIGUOUS, null, null);
     for (DatanodeStorageInfo datanodeStorageInfo : results) {
       assertFalse("Dead node should not be choosen", datanodeStorageInfo
           .getDatanodeDescriptor().equals(clientNode));


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


[25/50] [abbrv] hadoop git commit: HADOOP-14553. Add (parallelized) integration tests to hadoop-azure Contributed by Steve Loughran

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java
index 177477c..726b504 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java
@@ -18,12 +18,6 @@
 
 package org.apache.hadoop.fs.azure;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import java.io.BufferedReader;
 import java.io.BufferedWriter;
 import java.io.FileNotFoundException;
@@ -47,16 +41,18 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.Test;
-import org.apache.hadoop.fs.azure.AzureException;
 import org.apache.hadoop.fs.azure.NativeAzureFileSystem.FolderRenamePending;
 
 import com.microsoft.azure.storage.AccessCondition;
 import com.microsoft.azure.storage.StorageException;
 import com.microsoft.azure.storage.blob.CloudBlob;
 
+import static org.apache.hadoop.test.GenericTestUtils.*;
+
 /*
  * Tests the Native Azure file system (WASB) against an actual blob store if
  * provided in the environment.
@@ -71,15 +67,46 @@ public abstract class NativeAzureFileSystemBaseTest
   private final long modifiedTimeErrorMargin = 5 * 1000; // Give it +/-5 seconds
 
   public static final Log LOG = LogFactory.getLog(NativeAzureFileSystemBaseTest.class);
+  protected NativeAzureFileSystem fs;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    fs = getFileSystem();
+  }
+
+  /**
+   * Assert that a path does not exist.
+   *
+   * @param message message to include in the assertion failure message
+   * @param path path in the filesystem
+   * @throws IOException IO problems
+   */
+  public void assertPathDoesNotExist(String message,
+      Path path) throws IOException {
+    ContractTestUtils.assertPathDoesNotExist(fs, message, path);
+  }
+
+  /**
+   * Assert that a path exists.
+   *
+   * @param message message to include in the assertion failure message
+   * @param path path in the filesystem
+   * @throws IOException IO problems
+   */
+  public void assertPathExists(String message,
+      Path path) throws IOException {
+    ContractTestUtils.assertPathExists(fs, message, path);
+  }
 
   @Test
   public void testCheckingNonExistentOneLetterFile() throws Exception {
-    assertFalse(fs.exists(new Path("/a")));
+    assertPathDoesNotExist("one letter file", new Path("/a"));
   }
 
   @Test
   public void testStoreRetrieveFile() throws Exception {
-    Path testFile = new Path("unit-test-file");
+    Path testFile = methodPath();
     writeString(testFile, "Testing");
     assertTrue(fs.exists(testFile));
     FileStatus status = fs.getFileStatus(testFile);
@@ -93,7 +120,7 @@ public abstract class NativeAzureFileSystemBaseTest
 
   @Test
   public void testStoreDeleteFolder() throws Exception {
-    Path testFolder = new Path("storeDeleteFolder");
+    Path testFolder = methodPath();
     assertFalse(fs.exists(testFolder));
     assertTrue(fs.mkdirs(testFolder));
     assertTrue(fs.exists(testFolder));
@@ -105,22 +132,22 @@ public abstract class NativeAzureFileSystemBaseTest
     assertEquals(new FsPermission((short) 0755), status.getPermission());
     Path innerFile = new Path(testFolder, "innerFile");
     assertTrue(fs.createNewFile(innerFile));
-    assertTrue(fs.exists(innerFile));
+    assertPathExists("inner file", innerFile);
     assertTrue(fs.delete(testFolder, true));
-    assertFalse(fs.exists(innerFile));
-    assertFalse(fs.exists(testFolder));
+    assertPathDoesNotExist("inner file", innerFile);
+    assertPathDoesNotExist("testFolder", testFolder);
   }
 
   @Test
   public void testFileOwnership() throws Exception {
-    Path testFile = new Path("ownershipTestFile");
+    Path testFile = methodPath();
     writeString(testFile, "Testing");
     testOwnership(testFile);
   }
 
   @Test
   public void testFolderOwnership() throws Exception {
-    Path testFolder = new Path("ownershipTestFolder");
+    Path testFolder = methodPath();
     fs.mkdirs(testFolder);
     testOwnership(testFolder);
   }
@@ -147,7 +174,7 @@ public abstract class NativeAzureFileSystemBaseTest
 
   @Test
   public void testFilePermissions() throws Exception {
-    Path testFile = new Path("permissionTestFile");
+    Path testFile = methodPath();
     FsPermission permission = FsPermission.createImmutable((short) 644);
     createEmptyFile(testFile, permission);
     FileStatus ret = fs.getFileStatus(testFile);
@@ -157,7 +184,7 @@ public abstract class NativeAzureFileSystemBaseTest
 
   @Test
   public void testFolderPermissions() throws Exception {
-    Path testFolder = new Path("permissionTestFolder");
+    Path testFolder = methodPath();
     FsPermission permission = FsPermission.createImmutable((short) 644);
     fs.mkdirs(testFolder, permission);
     FileStatus ret = fs.getFileStatus(testFolder);
@@ -176,9 +203,9 @@ public abstract class NativeAzureFileSystemBaseTest
     createEmptyFile(testFile, permission);
     FsPermission rootPerm = fs.getFileStatus(firstDir.getParent()).getPermission();
     FsPermission inheritPerm = FsPermission.createImmutable((short)(rootPerm.toShort() | 0300));
-    assertTrue(fs.exists(testFile));
-    assertTrue(fs.exists(firstDir));
-    assertTrue(fs.exists(middleDir));
+    assertPathExists("test file", testFile);
+    assertPathExists("firstDir", firstDir);
+    assertPathExists("middleDir", middleDir);
     // verify that the indirectly created directory inherited its permissions from the root directory
     FileStatus directoryStatus = fs.getFileStatus(middleDir);
     assertTrue(directoryStatus.isDirectory());
@@ -188,7 +215,7 @@ public abstract class NativeAzureFileSystemBaseTest
     assertFalse(fileStatus.isDirectory());
     assertEqualsIgnoreStickyBit(umaskedPermission, fileStatus.getPermission());
     assertTrue(fs.delete(firstDir, true));
-    assertFalse(fs.exists(testFile));
+    assertPathDoesNotExist("deleted file", testFile);
 
     // An alternative test scenario would've been to delete the file first,
     // and then check for the existence of the upper folders still. But that
@@ -264,7 +291,7 @@ public abstract class NativeAzureFileSystemBaseTest
     assertTrue(fs.delete(new Path("deep"), true));
   }
 
-  private static enum RenameFolderVariation {
+  private enum RenameFolderVariation {
     CreateFolderAndInnerFile, CreateJustInnerFile, CreateJustFolder
   }
 
@@ -303,10 +330,10 @@ public abstract class NativeAzureFileSystemBaseTest
     localFs.delete(localFilePath, true);
     try {
       writeString(localFs, localFilePath, "Testing");
-      Path dstPath = new Path("copiedFromLocal");
+      Path dstPath = methodPath();
       assertTrue(FileUtil.copy(localFs, localFilePath, fs, dstPath, false,
           fs.getConf()));
-      assertTrue(fs.exists(dstPath));
+      assertPathExists("coied from local", dstPath);
       assertEquals("Testing", readString(fs, dstPath));
       fs.delete(dstPath, true);
     } finally {
@@ -423,32 +450,32 @@ public abstract class NativeAzureFileSystemBaseTest
 
   @Test
   public void testReadingDirectoryAsFile() throws Exception {
-    Path dir = new Path("/x");
+    Path dir = methodPath();
     assertTrue(fs.mkdirs(dir));
     try {
       fs.open(dir).close();
       assertTrue("Should've thrown", false);
     } catch (FileNotFoundException ex) {
-      assertEquals("/x is a directory not a file.", ex.getMessage());
+      assertExceptionContains("a directory not a file.", ex);
     }
   }
 
   @Test
   public void testCreatingFileOverDirectory() throws Exception {
-    Path dir = new Path("/x");
+    Path dir = methodPath();
     assertTrue(fs.mkdirs(dir));
     try {
       fs.create(dir).close();
       assertTrue("Should've thrown", false);
     } catch (IOException ex) {
-      assertEquals("Cannot create file /x; already exists as a directory.",
-          ex.getMessage());
+      assertExceptionContains("Cannot create file", ex);
+      assertExceptionContains("already exists as a directory", ex);
     }
   }
 
   @Test
   public void testInputStreamReadWithZeroSizeBuffer() throws Exception {
-    Path newFile = new Path("zeroSizeRead");
+    Path newFile = methodPath();
     OutputStream output = fs.create(newFile);
     output.write(10);
     output.close();
@@ -460,7 +487,7 @@ public abstract class NativeAzureFileSystemBaseTest
 
   @Test
   public void testInputStreamReadWithBufferReturnsMinusOneOnEof() throws Exception {
-    Path newFile = new Path("eofRead");
+    Path newFile = methodPath();
     OutputStream output = fs.create(newFile);
     output.write(10);
     output.close();
@@ -482,7 +509,7 @@ public abstract class NativeAzureFileSystemBaseTest
 
   @Test
   public void testInputStreamReadWithBufferReturnsMinusOneOnEofForLargeBuffer() throws Exception {
-    Path newFile = new Path("eofRead2");
+    Path newFile = methodPath();
     OutputStream output = fs.create(newFile);
     byte[] outputBuff = new byte[97331];
     for(int i = 0; i < outputBuff.length; ++i) {
@@ -508,7 +535,7 @@ public abstract class NativeAzureFileSystemBaseTest
 
   @Test
   public void testInputStreamReadIntReturnsMinusOneOnEof() throws Exception {
-    Path newFile = new Path("eofRead3");
+    Path newFile = methodPath();
     OutputStream output = fs.create(newFile);
     output.write(10);
     output.close();
@@ -525,7 +552,7 @@ public abstract class NativeAzureFileSystemBaseTest
 
   @Test
   public void testSetPermissionOnFile() throws Exception {
-    Path newFile = new Path("testPermission");
+    Path newFile = methodPath();
     OutputStream output = fs.create(newFile);
     output.write(13);
     output.close();
@@ -540,14 +567,14 @@ public abstract class NativeAzureFileSystemBaseTest
 
     // Don't check the file length for page blobs. Only block blobs
     // provide the actual length of bytes written.
-    if (!(this instanceof TestNativeAzureFSPageBlobLive)) {
+    if (!(this instanceof ITestNativeAzureFSPageBlobLive)) {
       assertEquals(1, newStatus.getLen());
     }
   }
 
   @Test
   public void testSetPermissionOnFolder() throws Exception {
-    Path newFolder = new Path("testPermission");
+    Path newFolder = methodPath();
     assertTrue(fs.mkdirs(newFolder));
     FsPermission newPermission = new FsPermission((short) 0600);
     fs.setPermission(newFolder, newPermission);
@@ -559,7 +586,7 @@ public abstract class NativeAzureFileSystemBaseTest
 
   @Test
   public void testSetOwnerOnFile() throws Exception {
-    Path newFile = new Path("testOwner");
+    Path newFile = methodPath();
     OutputStream output = fs.create(newFile);
     output.write(13);
     output.close();
@@ -571,7 +598,7 @@ public abstract class NativeAzureFileSystemBaseTest
 
     // File length is only reported to be the size of bytes written to the file for block blobs.
     // So only check it for block blobs, not page blobs.
-    if (!(this instanceof TestNativeAzureFSPageBlobLive)) {
+    if (!(this instanceof ITestNativeAzureFSPageBlobLive)) {
       assertEquals(1, newStatus.getLen());
     }
     fs.setOwner(newFile, null, "newGroup");
@@ -583,7 +610,7 @@ public abstract class NativeAzureFileSystemBaseTest
 
   @Test
   public void testSetOwnerOnFolder() throws Exception {
-    Path newFolder = new Path("testOwner");
+    Path newFolder = methodPath();
     assertTrue(fs.mkdirs(newFolder));
     fs.setOwner(newFolder, "newUser", null);
     FileStatus newStatus = fs.getFileStatus(newFolder);
@@ -594,21 +621,21 @@ public abstract class NativeAzureFileSystemBaseTest
 
   @Test
   public void testModifiedTimeForFile() throws Exception {
-    Path testFile = new Path("testFile");
+    Path testFile = methodPath();
     fs.create(testFile).close();
     testModifiedTime(testFile);
   }
 
   @Test
   public void testModifiedTimeForFolder() throws Exception {
-    Path testFolder = new Path("testFolder");
+    Path testFolder = methodPath();
     assertTrue(fs.mkdirs(testFolder));
     testModifiedTime(testFolder);
   }
 
   @Test
   public void testFolderLastModifiedTime() throws Exception {
-    Path parentFolder = new Path("testFolder");
+    Path parentFolder = methodPath();
     Path innerFile = new Path(parentFolder, "innerfile");
     assertTrue(fs.mkdirs(parentFolder));
 
@@ -983,7 +1010,7 @@ public abstract class NativeAzureFileSystemBaseTest
 
     // Make sure rename pending file is gone.
     FileStatus[] listed = fs.listStatus(new Path("/"));
-    assertEquals(1, listed.length);
+    assertEquals("Pending directory still found", 1, listed.length);
     assertTrue(listed[0].isDirectory());
   }
 
@@ -1681,7 +1708,7 @@ public abstract class NativeAzureFileSystemBaseTest
           assertTrue("Unanticipated exception", false);
         }
       } else {
-        assertTrue("Unknown thread name", false);
+        fail("Unknown thread name");
       }
 
       LOG.info(name + " is exiting.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/RunningLiveWasbTests.txt
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/RunningLiveWasbTests.txt b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/RunningLiveWasbTests.txt
deleted file mode 100644
index 54ba4d8..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/RunningLiveWasbTests.txt
+++ /dev/null
@@ -1,22 +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.
-=========================================================================
-
-In order to run Windows Azure Storage Blob (WASB) unit tests against a live 
-Azure Storage account, you need to provide test account details in a configuration
-file called azure-test.xml. See hadoop-tools/hadoop-azure/README.txt for details
-on configuration, and how to run the tests.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIo.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIo.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIo.java
deleted file mode 100644
index a10a366..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIo.java
+++ /dev/null
@@ -1,195 +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.azure;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.junit.Assume.assumeNotNull;
-
-import java.io.*;
-import java.util.Arrays;
-
-import org.apache.hadoop.fs.azure.AzureException;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestAzureConcurrentOutOfBandIo {
-
-  // Class constants.
-  static final int DOWNLOAD_BLOCK_SIZE = 8 * 1024 * 1024;
-  static final int UPLOAD_BLOCK_SIZE = 4 * 1024 * 1024;
-  static final int BLOB_SIZE = 32 * 1024 * 1024;
-
-  // Number of blocks to be written before flush.
-  static final int NUMBER_OF_BLOCKS = 2;
-
-  protected AzureBlobStorageTestAccount testAccount;
-
-  // Overridden TestCase methods.
-  @Before
-  public void setUp() throws Exception {
-    testAccount = AzureBlobStorageTestAccount.createOutOfBandStore(
-        UPLOAD_BLOCK_SIZE, DOWNLOAD_BLOCK_SIZE);
-    assumeNotNull(testAccount);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (testAccount != null) {
-      testAccount.cleanup();
-      testAccount = null;
-    }
-  }
-
-  class DataBlockWriter implements Runnable {
-
-    Thread runner;
-    AzureBlobStorageTestAccount writerStorageAccount;
-    String key;
-    boolean done = false;
-
-    /**
-     * Constructor captures the test account.
-     * 
-     * @param testAccount
-     */
-    public DataBlockWriter(AzureBlobStorageTestAccount testAccount, String key) {
-      writerStorageAccount = testAccount;
-      this.key = key;
-    }
-
-    /**
-     * Start writing blocks to Azure storage.
-     */
-    public void startWriting() {
-      runner = new Thread(this); // Create the block writer thread.
-      runner.start(); // Start the block writer thread.
-    }
-
-    /**
-     * Stop writing blocks to Azure storage.
-     */
-    public void stopWriting() {
-      done = true;
-    }
-
-    /**
-     * Implementation of the runnable interface. The run method is a tight loop
-     * which repeatedly updates the blob with a 4 MB block.
-     */
-    public void run() {
-      byte[] dataBlockWrite = new byte[UPLOAD_BLOCK_SIZE];
-
-      OutputStream outputStream = null;
-
-      try {
-        for (int i = 0; !done; i++) {
-          // Write two 4 MB blocks to the blob.
-          //
-          outputStream = writerStorageAccount.getStore().storefile(
-              key,
-              new PermissionStatus("", "", FsPermission.getDefault()),
-              key);
-
-          Arrays.fill(dataBlockWrite, (byte) (i % 256));
-          for (int j = 0; j < NUMBER_OF_BLOCKS; j++) {
-            outputStream.write(dataBlockWrite);
-          }
-
-          outputStream.flush();
-          outputStream.close();
-        }
-      } catch (AzureException e) {
-        System.out
-            .println("DatablockWriter thread encountered a storage exception."
-                + e.getMessage());
-      } catch (IOException e) {
-        System.out
-            .println("DatablockWriter thread encountered an I/O exception."
-            + e.getMessage());
-      }
-    }
-  }
-
-  @Test
-  public void testReadOOBWrites() throws Exception {
-
-    byte[] dataBlockWrite = new byte[UPLOAD_BLOCK_SIZE];
-    byte[] dataBlockRead = new byte[UPLOAD_BLOCK_SIZE];
-
-    // Write to blob to make sure it exists.
-    //
-   // Write five 4 MB blocks to the blob. To ensure there is data in the blob before
-   // reading.  This eliminates the race between the reader and writer threads.
-   OutputStream outputStream = testAccount.getStore().storefile(
-       "WASB_String.txt",
-       new PermissionStatus("", "", FsPermission.getDefault()),
-           "WASB_String.txt");
-   Arrays.fill(dataBlockWrite, (byte) 255);
-   for (int i = 0; i < NUMBER_OF_BLOCKS; i++) {
-     outputStream.write(dataBlockWrite);
-   }
-
-   outputStream.flush();
-   outputStream.close();
-
-   // Start writing blocks to Azure store using the DataBlockWriter thread.
-    DataBlockWriter writeBlockTask = new DataBlockWriter(testAccount,
-        "WASB_String.txt");
-   writeBlockTask.startWriting();
-   int count = 0;
-   InputStream inputStream = null;
-
-   for (int i = 0; i < 5; i++) {
-     try {
-        inputStream = testAccount.getStore().retrieve("WASB_String.txt");
-        count = 0;
-        int c = 0;
-
-        while (c >= 0) {
-          c = inputStream.read(dataBlockRead, 0, UPLOAD_BLOCK_SIZE);
-          if (c < 0) {
-            break;
-          }
-
-          // Counting the number of bytes.
-          count += c;
-        }
-     } catch (IOException e) {
-       System.out.println(e.getCause().toString());
-       e.printStackTrace();
-       fail();
-     }
-
-     // Close the stream.
-     if (null != inputStream){
-       inputStream.close();
-     }
-   }
-
-    // Stop writing blocks.
-    writeBlockTask.stopWriting();
-
-    // Validate that a block was read.
-    assertEquals(NUMBER_OF_BLOCKS * UPLOAD_BLOCK_SIZE, count);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIoWithSecureMode.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIoWithSecureMode.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIoWithSecureMode.java
deleted file mode 100644
index 687b785..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIoWithSecureMode.java
+++ /dev/null
@@ -1,50 +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.azure;
-
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.Arrays;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.junit.Assume.assumeNotNull;
-
-/**
- * Extends TestAzureConcurrentOutOfBandIo in order to run testReadOOBWrites with secure mode
- * (fs.azure.secure.mode) both enabled and disabled.
- */
-public class TestAzureConcurrentOutOfBandIoWithSecureMode extends  TestAzureConcurrentOutOfBandIo {
-
-  // Overridden TestCase methods.
-  @Before
-  @Override
-  public void setUp() throws Exception {
-    testAccount = AzureBlobStorageTestAccount.createOutOfBandStore(
-        UPLOAD_BLOCK_SIZE, DOWNLOAD_BLOCK_SIZE, true);
-    assumeNotNull(testAccount);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureFileSystemErrorConditions.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureFileSystemErrorConditions.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureFileSystemErrorConditions.java
deleted file mode 100644
index c985224..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureFileSystemErrorConditions.java
+++ /dev/null
@@ -1,244 +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.azure;
-
-import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.NO_ACCESS_TO_CONTAINER_MSG;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assume.assumeNotNull;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.HttpURLConnection;
-import java.net.URI;
-import java.util.Arrays;
-import java.util.HashMap;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.TestHookOperationContext;
-import org.apache.hadoop.test.GenericTestUtils;
-
-import org.junit.Test;
-
-import com.microsoft.azure.storage.OperationContext;
-import com.microsoft.azure.storage.SendingRequestEvent;
-import com.microsoft.azure.storage.StorageEvent;
-
-public class TestAzureFileSystemErrorConditions {
-  private static final int ALL_THREE_FILE_SIZE = 1024;
-
-  @Test
-  public void testNoInitialize() throws Exception {
-    AzureNativeFileSystemStore store = new AzureNativeFileSystemStore();
-    boolean passed = false;
-    try {
-      store.retrieveMetadata("foo");
-      passed = true;
-    } catch (AssertionError e) {
-    }
-    assertFalse(
-        "Doing an operation on the store should throw if not initalized.",
-        passed);
-  }
-
-  /**
-   * Try accessing an unauthorized or non-existent (treated the same) container
-   * from WASB.
-   */
-  @Test
-  public void testAccessUnauthorizedPublicContainer() throws Exception {
-    final String container = "nonExistentContainer";
-    final String account = "hopefullyNonExistentAccount";
-    Path noAccessPath = new Path(
-        "wasb://" + container + "@" + account + "/someFile");
-    NativeAzureFileSystem.suppressRetryPolicy();
-    try {
-      FileSystem.get(noAccessPath.toUri(), new Configuration())
-        .open(noAccessPath);
-      assertTrue("Should've thrown.", false);
-    } catch (AzureException ex) {
-      GenericTestUtils.assertExceptionContains(
-          String.format(NO_ACCESS_TO_CONTAINER_MSG, account, container), ex);
-    } finally {
-      NativeAzureFileSystem.resumeRetryPolicy();
-    }
-  }
-
-  @Test
-  public void testAccessContainerWithWrongVersion() throws Exception {
-    AzureNativeFileSystemStore store = new AzureNativeFileSystemStore();
-    MockStorageInterface mockStorage = new MockStorageInterface();
-    store.setAzureStorageInteractionLayer(mockStorage);
-    FileSystem fs = new NativeAzureFileSystem(store);
-    try {
-      Configuration conf = new Configuration();
-      AzureBlobStorageTestAccount.setMockAccountKey(conf);
-      HashMap<String, String> metadata = new HashMap<String, String>();
-      metadata.put(AzureNativeFileSystemStore.VERSION_METADATA_KEY,
-          "2090-04-05"); // It's from the future!
-      mockStorage.addPreExistingContainer(
-          AzureBlobStorageTestAccount.getMockContainerUri(), metadata);
-
-      boolean passed = false;
-      try {
-        fs.initialize(new URI(AzureBlobStorageTestAccount.MOCK_WASB_URI), conf);
-        fs.listStatus(new Path("/"));
-        passed = true;
-      } catch (AzureException ex) {
-        assertTrue("Unexpected exception message: " + ex,
-            ex.getMessage().contains("unsupported version: 2090-04-05."));
-      }
-      assertFalse("Should've thrown an exception because of the wrong version.",
-          passed);
-    } finally {
-      fs.close();
-    }
-  }
-
-  private interface ConnectionRecognizer {
-    boolean isTargetConnection(HttpURLConnection connection);
-  }
-
-  private class TransientErrorInjector extends StorageEvent<SendingRequestEvent> {
-    final ConnectionRecognizer connectionRecognizer;
-    private boolean injectedErrorOnce = false;
-
-    public TransientErrorInjector(ConnectionRecognizer connectionRecognizer) {
-      this.connectionRecognizer = connectionRecognizer;
-    }
-
-    @Override
-    public void eventOccurred(SendingRequestEvent eventArg) {
-      HttpURLConnection connection = (HttpURLConnection)eventArg.getConnectionObject();
-      if (!connectionRecognizer.isTargetConnection(connection)) {
-        return;
-      }
-      if (!injectedErrorOnce) {
-        connection.setReadTimeout(1);
-        connection.disconnect();
-        injectedErrorOnce = true;
-      }
-    }
-  }
-
-  private void injectTransientError(NativeAzureFileSystem fs,
-      final ConnectionRecognizer connectionRecognizer) {
-    fs.getStore().addTestHookToOperationContext(new TestHookOperationContext() {
-      @Override
-      public OperationContext modifyOperationContext(OperationContext original) {
-        original.getSendingRequestEventHandler().addListener(
-            new TransientErrorInjector(connectionRecognizer));
-        return original;
-      }
-    });
-  }
-
-  @Test
-  public void testTransientErrorOnDelete() throws Exception {
-    // Need to do this test against a live storage account
-    AzureBlobStorageTestAccount testAccount =
-        AzureBlobStorageTestAccount.create();
-    assumeNotNull(testAccount);
-    try {
-      NativeAzureFileSystem fs = testAccount.getFileSystem();
-      injectTransientError(fs, new ConnectionRecognizer() {
-        @Override
-        public boolean isTargetConnection(HttpURLConnection connection) {
-          return connection.getRequestMethod().equals("DELETE");
-        }
-      });
-      Path testFile = new Path("/a/b");
-      assertTrue(fs.createNewFile(testFile));
-      assertTrue(fs.rename(testFile, new Path("/x")));
-    } finally {
-      testAccount.cleanup();
-    }
-  }
-
-  private void writeAllThreeFile(NativeAzureFileSystem fs, Path testFile)
-      throws IOException {
-    byte[] buffer = new byte[ALL_THREE_FILE_SIZE];
-    Arrays.fill(buffer, (byte)3);
-    OutputStream stream = fs.create(testFile);
-    stream.write(buffer);
-    stream.close();
-  }
-
-  private void readAllThreeFile(NativeAzureFileSystem fs, Path testFile)
-      throws IOException {
-    byte[] buffer = new byte[ALL_THREE_FILE_SIZE];
-    InputStream inStream = fs.open(testFile);
-    assertEquals(buffer.length,
-        inStream.read(buffer, 0, buffer.length));
-    inStream.close();
-    for (int i = 0; i < buffer.length; i++) {
-      assertEquals(3, buffer[i]);
-    }
-  }
-
-  @Test
-  public void testTransientErrorOnCommitBlockList() throws Exception {
-    // Need to do this test against a live storage account
-    AzureBlobStorageTestAccount testAccount =
-        AzureBlobStorageTestAccount.create();
-    assumeNotNull(testAccount);
-    try {
-      NativeAzureFileSystem fs = testAccount.getFileSystem();
-      injectTransientError(fs, new ConnectionRecognizer() {
-        @Override
-        public boolean isTargetConnection(HttpURLConnection connection) {
-          return connection.getRequestMethod().equals("PUT")
-              && connection.getURL().getQuery() != null
-              && connection.getURL().getQuery().contains("blocklist");
-        }
-      });
-      Path testFile = new Path("/a/b");
-      writeAllThreeFile(fs, testFile);
-      readAllThreeFile(fs, testFile);
-    } finally {
-      testAccount.cleanup();
-    }
-  }
-
-  @Test
-  public void testTransientErrorOnRead() throws Exception {
-    // Need to do this test against a live storage account
-    AzureBlobStorageTestAccount testAccount =
-        AzureBlobStorageTestAccount.create();
-    assumeNotNull(testAccount);
-    try {
-      NativeAzureFileSystem fs = testAccount.getFileSystem();
-      Path testFile = new Path("/a/b");
-      writeAllThreeFile(fs, testFile);
-      injectTransientError(fs, new ConnectionRecognizer() {
-        @Override
-        public boolean isTargetConnection(HttpURLConnection connection) {
-          return connection.getRequestMethod().equals("GET");
-        }
-      });
-      readAllThreeFile(fs, testFile);
-    } finally {
-      testAccount.cleanup();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobDataValidation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobDataValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobDataValidation.java
deleted file mode 100644
index ea17b62..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobDataValidation.java
+++ /dev/null
@@ -1,237 +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.azure;
-
-import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.KEY_CHECK_BLOCK_MD5;
-import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.KEY_STORE_BLOB_MD5;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.fail;
-import static org.junit.Assume.assumeNotNull;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.HttpURLConnection;
-import java.util.Arrays;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.TestHookOperationContext;
-import org.junit.After;
-import org.junit.Test;
-
-import com.microsoft.azure.storage.Constants;
-import com.microsoft.azure.storage.OperationContext;
-import com.microsoft.azure.storage.ResponseReceivedEvent;
-import com.microsoft.azure.storage.StorageErrorCodeStrings;
-import com.microsoft.azure.storage.StorageEvent;
-import com.microsoft.azure.storage.StorageException;
-import com.microsoft.azure.storage.blob.BlockEntry;
-import com.microsoft.azure.storage.blob.BlockSearchMode;
-import com.microsoft.azure.storage.blob.CloudBlockBlob;
-import com.microsoft.azure.storage.core.Base64;
-
-/**
- * Test that we do proper data integrity validation with MD5 checks as
- * configured.
- */
-public class TestBlobDataValidation {
-  private AzureBlobStorageTestAccount testAccount;
-
-  @After
-  public void tearDown() throws Exception {
-    if (testAccount != null) {
-      testAccount.cleanup();
-      testAccount = null;
-    }
-  }
-
-  /**
-   * Test that by default we don't store the blob-level MD5.
-   */
-  @Test
-  public void testBlobMd5StoreOffByDefault() throws Exception {
-    testAccount = AzureBlobStorageTestAccount.create();
-    testStoreBlobMd5(false);
-  }
-
-  /**
-   * Test that we get blob-level MD5 storage and validation if we specify that
-   * in the configuration.
-   */
-  @Test
-  public void testStoreBlobMd5() throws Exception {
-    Configuration conf = new Configuration();
-    conf.setBoolean(KEY_STORE_BLOB_MD5, true);
-    testAccount = AzureBlobStorageTestAccount.create(conf);
-    testStoreBlobMd5(true);
-  }
-
-  private void testStoreBlobMd5(boolean expectMd5Stored) throws Exception {
-    assumeNotNull(testAccount);
-    // Write a test file.
-    String testFileKey = "testFile";
-    Path testFilePath = new Path("/" + testFileKey);
-    OutputStream outStream = testAccount.getFileSystem().create(testFilePath);
-    outStream.write(new byte[] { 5, 15 });
-    outStream.close();
-
-    // Check that we stored/didn't store the MD5 field as configured.
-    CloudBlockBlob blob = testAccount.getBlobReference(testFileKey);
-    blob.downloadAttributes();
-    String obtainedMd5 = blob.getProperties().getContentMD5();
-    if (expectMd5Stored) {
-      assertNotNull(obtainedMd5);
-    } else {
-      assertNull("Expected no MD5, found: " + obtainedMd5, obtainedMd5);
-    }
-
-    // Mess with the content so it doesn't match the MD5.
-    String newBlockId = Base64.encode(new byte[] { 55, 44, 33, 22 });
-    blob.uploadBlock(newBlockId,
-        new ByteArrayInputStream(new byte[] { 6, 45 }), 2);
-    blob.commitBlockList(Arrays.asList(new BlockEntry[] { new BlockEntry(
-        newBlockId, BlockSearchMode.UNCOMMITTED) }));
-
-    // Now read back the content. If we stored the MD5 for the blob content
-    // we should get a data corruption error.
-    InputStream inStream = testAccount.getFileSystem().open(testFilePath);
-    try {
-      byte[] inBuf = new byte[100];
-      while (inStream.read(inBuf) > 0){
-        //nothing;
-      }
-      inStream.close();
-      if (expectMd5Stored) {
-        fail("Should've thrown because of data corruption.");
-      }
-    } catch (IOException ex) {
-      if (!expectMd5Stored) {
-        throw ex;
-      }
-      StorageException cause = (StorageException)ex.getCause();
-      assertNotNull(cause);
-      assertEquals("Unexpected cause: " + cause,
-          StorageErrorCodeStrings.INVALID_MD5, cause.getErrorCode());
-    }
-  }
-
-  /**
-   * Test that by default we check block-level MD5.
-   */
-  @Test
-  public void testCheckBlockMd5() throws Exception {
-    testAccount = AzureBlobStorageTestAccount.create();
-    testCheckBlockMd5(true);
-  }
-
-  /**
-   * Test that we don't check block-level MD5 if we specify that in the
-   * configuration.
-   */
-  @Test
-  public void testDontCheckBlockMd5() throws Exception {
-    Configuration conf = new Configuration();
-    conf.setBoolean(KEY_CHECK_BLOCK_MD5, false);
-    testAccount = AzureBlobStorageTestAccount.create(conf);
-    testCheckBlockMd5(false);
-  }
-
-  /**
-   * Connection inspector to check that MD5 fields for content is set/not set as
-   * expected.
-   */
-  private static class ContentMD5Checker extends
-      StorageEvent<ResponseReceivedEvent> {
-    private final boolean expectMd5;
-
-    public ContentMD5Checker(boolean expectMd5) {
-      this.expectMd5 = expectMd5;
-    }
-
-    @Override
-    public void eventOccurred(ResponseReceivedEvent eventArg) {
-      HttpURLConnection connection = (HttpURLConnection) eventArg
-          .getConnectionObject();
-      if (isGetRange(connection)) {
-        checkObtainedMd5(connection
-            .getHeaderField(Constants.HeaderConstants.CONTENT_MD5));
-      } else if (isPutBlock(connection)) {
-        checkObtainedMd5(connection
-            .getRequestProperty(Constants.HeaderConstants.CONTENT_MD5));
-      }
-    }
-
-    private void checkObtainedMd5(String obtainedMd5) {
-      if (expectMd5) {
-        assertNotNull(obtainedMd5);
-      } else {
-        assertNull("Expected no MD5, found: " + obtainedMd5, obtainedMd5);
-      }
-    }
-
-    private static boolean isPutBlock(HttpURLConnection connection) {
-      return connection.getRequestMethod().equals("PUT")
-          && connection.getURL().getQuery() != null
-          && connection.getURL().getQuery().contains("blockid");
-    }
-
-    private static boolean isGetRange(HttpURLConnection connection) {
-      return connection.getRequestMethod().equals("GET")
-          && connection
-              .getHeaderField(Constants.HeaderConstants.STORAGE_RANGE_HEADER) != null;
-    }
-  }
-
-  private void testCheckBlockMd5(final boolean expectMd5Checked)
-      throws Exception {
-    assumeNotNull(testAccount);
-    Path testFilePath = new Path("/testFile");
-
-    // Add a hook to check that for GET/PUT requests we set/don't set
-    // the block-level MD5 field as configured. I tried to do clever
-    // testing by also messing with the raw data to see if we actually
-    // validate the data as expected, but the HttpURLConnection wasn't
-    // pluggable enough for me to do that.
-    testAccount.getFileSystem().getStore()
-    .addTestHookToOperationContext(new TestHookOperationContext() {
-    @Override
-          public OperationContext modifyOperationContext(
-              OperationContext original) {
-      original.getResponseReceivedEventHandler().addListener(
-          new ContentMD5Checker(expectMd5Checked));
-      return original;
-          }
-        });
-
-    OutputStream outStream = testAccount.getFileSystem().create(testFilePath);
-    outStream.write(new byte[] { 5, 15 });
-    outStream.close();
-
-    InputStream inStream = testAccount.getFileSystem().open(testFilePath);
-    byte[] inBuf = new byte[100];
-    while (inStream.read(inBuf) > 0){
-      //nothing;
-    }
-    inStream.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobMetadata.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobMetadata.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobMetadata.java
index 6c49926..30c1028 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobMetadata.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobMetadata.java
@@ -18,11 +18,6 @@
 
 package org.apache.hadoop.fs.azure;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.net.URI;
@@ -42,7 +37,7 @@ import org.junit.Test;
 /**
  * Tests that we put the correct metadata on blobs created through WASB.
  */
-public class TestBlobMetadata {
+public class TestBlobMetadata extends AbstractWasbTestWithTimeout {
   private AzureBlobStorageTestAccount testAccount;
   private FileSystem fs;
   private InMemoryBlockBlobStore backingStore;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobOperationDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobOperationDescriptor.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobOperationDescriptor.java
index 07d4ebc..aca5f81 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobOperationDescriptor.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobOperationDescriptor.java
@@ -33,9 +33,6 @@ import org.junit.Test;
 
 import java.net.HttpURLConnection;
 
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertEquals;
-
 /**
  * Tests for <code>BlobOperationDescriptor</code>.
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobTypeSpeedDifference.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobTypeSpeedDifference.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobTypeSpeedDifference.java
deleted file mode 100644
index afb16ef..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobTypeSpeedDifference.java
+++ /dev/null
@@ -1,160 +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.azure;
-
-import java.io.*;
-import java.util.*;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
-
-import junit.framework.*;
-
-import org.junit.Test;
-
-
-/**
- * A simple benchmark to find out the difference in speed between block
- * and page blobs.
- */
-public class TestBlobTypeSpeedDifference extends TestCase {
-  /**
-   * Writes data to the given stream of the given size, flushing every
-   * x bytes.
-   */
-  private static void writeTestFile(OutputStream writeStream,
-      long size, long flushInterval) throws IOException {
-    int bufferSize = (int) Math.min(1000, flushInterval);
-    byte[] buffer = new byte[bufferSize];
-    Arrays.fill(buffer, (byte) 7);
-    int bytesWritten = 0;
-    int bytesUnflushed = 0;
-    while (bytesWritten < size) {
-      int numberToWrite = (int) Math.min(bufferSize, size - bytesWritten);
-      writeStream.write(buffer, 0, numberToWrite);
-      bytesWritten += numberToWrite;
-      bytesUnflushed += numberToWrite;
-      if (bytesUnflushed >= flushInterval) {
-        writeStream.flush();
-        bytesUnflushed = 0;
-      }
-    }
-  }
-
-  private static class TestResult {
-    final long timeTakenInMs;
-    final long totalNumberOfRequests;
-
-    TestResult(long timeTakenInMs, long totalNumberOfRequests) {
-      this.timeTakenInMs = timeTakenInMs;
-      this.totalNumberOfRequests = totalNumberOfRequests;
-    }
-  }
-
-  /**
-   * Writes data to the given file of the given size, flushing every
-   * x bytes. Measure performance of that and return it.
-   */
-  private static TestResult writeTestFile(NativeAzureFileSystem fs, Path path,
-      long size, long flushInterval) throws IOException {
-    AzureFileSystemInstrumentation instrumentation =
-        fs.getInstrumentation();
-    long initialRequests = instrumentation.getCurrentWebResponses();
-    Date start = new Date();
-    OutputStream output = fs.create(path);
-    writeTestFile(output, size, flushInterval);
-    output.close();
-    long finalRequests = instrumentation.getCurrentWebResponses();
-    return new TestResult(new Date().getTime() - start.getTime(),
-        finalRequests - initialRequests);
-  }
-
-  /**
-   * Writes data to a block blob of the given size, flushing every
-   * x bytes. Measure performance of that and return it.
-   */
-  private static TestResult writeBlockBlobTestFile(NativeAzureFileSystem fs,
-      long size, long flushInterval) throws IOException {
-    return writeTestFile(fs, new Path("/blockBlob"), size, flushInterval);
-  }
-
-  /**
-   * Writes data to a page blob of the given size, flushing every
-   * x bytes. Measure performance of that and return it.
-   */
-  private static TestResult writePageBlobTestFile(NativeAzureFileSystem fs,
-      long size, long flushInterval) throws IOException {
-    return writeTestFile(fs,
-        AzureBlobStorageTestAccount.pageBlobPath("pageBlob"),
-        size, flushInterval);
-  }
-
-  /**
-   * Runs the benchmark over a small 10 KB file, flushing every 500 bytes.
-   */
-  @Test
-  public void testTenKbFileFrequentFlush() throws Exception {
-    AzureBlobStorageTestAccount testAccount =
-        AzureBlobStorageTestAccount.create();
-    if (testAccount == null) {
-      return;
-    }
-    try {
-      testForSizeAndFlushInterval(testAccount.getFileSystem(), 10 * 1000, 500);
-    } finally {
-      testAccount.cleanup();
-    }
-  }
-
-  /**
-   * Runs the benchmark for the given file size and flush frequency.
-   */
-  private static void testForSizeAndFlushInterval(NativeAzureFileSystem fs,
-      final long size, final long flushInterval) throws IOException {
-    for (int i = 0; i < 5; i++) {
-      TestResult pageBlobResults = writePageBlobTestFile(fs, size, flushInterval);
-      System.out.printf(
-          "Page blob upload took %d ms. Total number of requests: %d.\n",
-          pageBlobResults.timeTakenInMs, pageBlobResults.totalNumberOfRequests);
-      TestResult blockBlobResults = writeBlockBlobTestFile(fs, size, flushInterval);
-      System.out.printf(
-          "Block blob upload took %d ms. Total number of requests: %d.\n",
-          blockBlobResults.timeTakenInMs, blockBlobResults.totalNumberOfRequests);
-    }
-  }
-
-  /**
-   * Runs the benchmark for the given file size and flush frequency from the
-   * command line.
-   */
-  public static void main(String argv[]) throws Exception {
-    Configuration conf = new Configuration();
-    long size = 10 * 1000 * 1000;
-    long flushInterval = 2000;
-    if (argv.length > 0) {
-      size = Long.parseLong(argv[0]);
-    }
-    if (argv.length > 1) {
-      flushInterval = Long.parseLong(argv[1]);
-    }
-    testForSizeAndFlushInterval((NativeAzureFileSystem)FileSystem.get(conf),
-        size, flushInterval);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlockBlobInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlockBlobInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlockBlobInputStream.java
deleted file mode 100644
index 0ae4012..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlockBlobInputStream.java
+++ /dev/null
@@ -1,875 +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.azure;
-
-import java.io.EOFException;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.EnumSet;
-import java.util.Random;
-import java.util.concurrent.Callable;
-
-import org.junit.FixMethodOrder;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-import org.junit.runners.MethodSorters;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FSExceptionMessages;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.apache.hadoop.fs.contract.ContractTestUtils.NanoTimer;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assume.assumeNotNull;
-
-import static org.apache.hadoop.test.LambdaTestUtils.*;
-
-/**
- * Test semantics and performance of the original block blob input stream
- * (KEY_INPUT_STREAM_VERSION=1) and the new
- * <code>BlockBlobInputStream</code> (KEY_INPUT_STREAM_VERSION=2).
- */
-@FixMethodOrder(MethodSorters.NAME_ASCENDING)
-
-public class TestBlockBlobInputStream extends AbstractWasbTestBase {
-  private static final Logger LOG = LoggerFactory.getLogger(
-      TestBlockBlobInputStream.class);
-  private static final int KILOBYTE = 1024;
-  private static final int MEGABYTE = KILOBYTE * KILOBYTE;
-  private static final int TEST_FILE_SIZE = 6 * MEGABYTE;
-  private static final Path TEST_FILE_PATH = new Path(
-      "TestBlockBlobInputStream.txt");
-
-  private AzureBlobStorageTestAccount accountUsingInputStreamV1;
-  private AzureBlobStorageTestAccount accountUsingInputStreamV2;
-  private long testFileLength;
-
-  /**
-   * Long test timeout.
-   */
-  @Rule
-  public Timeout testTimeout = new Timeout(10 * 60 * 1000);
-  private FileStatus testFileStatus;
-  private Path hugefile;
-
-  @Override
-  public void setUp() throws Exception {
-    super.setUp();
-    Configuration conf = new Configuration();
-    conf.setInt(AzureNativeFileSystemStore.KEY_INPUT_STREAM_VERSION, 1);
-
-    accountUsingInputStreamV1 = AzureBlobStorageTestAccount.create(
-        "testblockblobinputstream",
-        EnumSet.of(AzureBlobStorageTestAccount.CreateOptions.CreateContainer),
-        conf,
-        true);
-
-    accountUsingInputStreamV2 = AzureBlobStorageTestAccount.create(
-        "testblockblobinputstream",
-        EnumSet.noneOf(AzureBlobStorageTestAccount.CreateOptions.class),
-        null,
-        true);
-
-    assumeNotNull(accountUsingInputStreamV1);
-    assumeNotNull(accountUsingInputStreamV2);
-    hugefile = fs.makeQualified(TEST_FILE_PATH);
-    try {
-      testFileStatus = fs.getFileStatus(TEST_FILE_PATH);
-      testFileLength = testFileStatus.getLen();
-    } catch (FileNotFoundException e) {
-      // file doesn't exist
-      testFileLength = 0;
-    }
-  }
-
-  @Override
-  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
-    Configuration conf = new Configuration();
-    conf.setInt(AzureNativeFileSystemStore.KEY_INPUT_STREAM_VERSION, 1);
-
-    accountUsingInputStreamV1 = AzureBlobStorageTestAccount.create(
-        "testblockblobinputstream",
-        EnumSet.of(AzureBlobStorageTestAccount.CreateOptions.CreateContainer),
-        conf,
-        true);
-
-    accountUsingInputStreamV2 = AzureBlobStorageTestAccount.create(
-        "testblockblobinputstream",
-        EnumSet.noneOf(AzureBlobStorageTestAccount.CreateOptions.class),
-        null,
-        true);
-
-    assumeNotNull(accountUsingInputStreamV1);
-    assumeNotNull(accountUsingInputStreamV2);
-    return accountUsingInputStreamV1;
-  }
-
-  /**
-   * Create a test file by repeating the characters in the alphabet.
-   * @throws IOException
-   */
-  private void createTestFileAndSetLength() throws IOException {
-    FileSystem fs = accountUsingInputStreamV1.getFileSystem();
-
-    // To reduce test run time, the test file can be reused.
-    if (fs.exists(TEST_FILE_PATH)) {
-      testFileStatus = fs.getFileStatus(TEST_FILE_PATH);
-      testFileLength = testFileStatus.getLen();
-      LOG.info("Reusing test file: {}", testFileStatus);
-      return;
-    }
-
-    int sizeOfAlphabet = ('z' - 'a' + 1);
-    byte[] buffer = new byte[26 * KILOBYTE];
-    char character = 'a';
-    for (int i = 0; i < buffer.length; i++) {
-      buffer[i] = (byte) character;
-      character = (character == 'z') ? 'a' : (char) ((int) character + 1);
-    }
-
-    LOG.info("Creating test file {} of size: {}", TEST_FILE_PATH,
-        TEST_FILE_SIZE);
-    ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
-
-    try(FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) {
-      int bytesWritten = 0;
-      while (bytesWritten < TEST_FILE_SIZE) {
-        outputStream.write(buffer);
-        bytesWritten += buffer.length;
-      }
-      LOG.info("Closing stream {}", outputStream);
-      ContractTestUtils.NanoTimer closeTimer
-          = new ContractTestUtils.NanoTimer();
-      outputStream.close();
-      closeTimer.end("time to close() output stream");
-    }
-    timer.end("time to write %d KB", TEST_FILE_SIZE / 1024);
-    testFileLength = fs.getFileStatus(TEST_FILE_PATH).getLen();
-  }
-
-  void assumeHugeFileExists() throws IOException {
-    ContractTestUtils.assertPathExists(fs, "huge file not created", hugefile);
-    FileStatus status = fs.getFileStatus(hugefile);
-    ContractTestUtils.assertIsFile(hugefile, status);
-    assertTrue("File " + hugefile + " is empty", status.getLen() > 0);
-  }
-
-  /**
-   * Calculate megabits per second from the specified values for bytes and
-   * milliseconds.
-   * @param bytes The number of bytes.
-   * @param milliseconds The number of milliseconds.
-   * @return The number of megabits per second.
-   */
-  private static double toMbps(long bytes, long milliseconds) {
-    return bytes / 1000.0 * 8 / milliseconds;
-  }
-
-  @Test
-  public void test_0100_CreateHugeFile() throws IOException {
-    createTestFileAndSetLength();
-  }
-
-  @Test
-  public void test_0200_BasicReadTest() throws Exception {
-    assumeHugeFileExists();
-
-    try (
-        FSDataInputStream inputStreamV1
-            = accountUsingInputStreamV1.getFileSystem().open(TEST_FILE_PATH);
-
-        FSDataInputStream inputStreamV2
-            = accountUsingInputStreamV2.getFileSystem().open(TEST_FILE_PATH);
-    ) {
-      byte[] bufferV1 = new byte[3 * MEGABYTE];
-      byte[] bufferV2 = new byte[bufferV1.length];
-
-      // v1 forward seek and read a kilobyte into first kilobyte of bufferV1
-      inputStreamV1.seek(5 * MEGABYTE);
-      int numBytesReadV1 = inputStreamV1.read(bufferV1, 0, KILOBYTE);
-      assertEquals(KILOBYTE, numBytesReadV1);
-
-      // v2 forward seek and read a kilobyte into first kilobyte of bufferV2
-      inputStreamV2.seek(5 * MEGABYTE);
-      int numBytesReadV2 = inputStreamV2.read(bufferV2, 0, KILOBYTE);
-      assertEquals(KILOBYTE, numBytesReadV2);
-
-      assertArrayEquals(bufferV1, bufferV2);
-
-      int len = MEGABYTE;
-      int offset = bufferV1.length - len;
-
-      // v1 reverse seek and read a megabyte into last megabyte of bufferV1
-      inputStreamV1.seek(3 * MEGABYTE);
-      numBytesReadV1 = inputStreamV1.read(bufferV1, offset, len);
-      assertEquals(len, numBytesReadV1);
-
-      // v2 reverse seek and read a megabyte into last megabyte of bufferV2
-      inputStreamV2.seek(3 * MEGABYTE);
-      numBytesReadV2 = inputStreamV2.read(bufferV2, offset, len);
-      assertEquals(len, numBytesReadV2);
-
-      assertArrayEquals(bufferV1, bufferV2);
-    }
-  }
-
-  @Test
-  public void test_0201_RandomReadTest() throws Exception {
-    assumeHugeFileExists();
-
-    try (
-        FSDataInputStream inputStreamV1
-            = accountUsingInputStreamV1.getFileSystem().open(TEST_FILE_PATH);
-
-        FSDataInputStream inputStreamV2
-            = accountUsingInputStreamV2.getFileSystem().open(TEST_FILE_PATH);
-    ) {
-      final int bufferSize = 4 * KILOBYTE;
-      byte[] bufferV1 = new byte[bufferSize];
-      byte[] bufferV2 = new byte[bufferV1.length];
-
-      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
-
-      inputStreamV1.seek(0);
-      inputStreamV2.seek(0);
-
-      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
-
-      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
-
-      int seekPosition = 2 * KILOBYTE;
-      inputStreamV1.seek(seekPosition);
-      inputStreamV2.seek(seekPosition);
-
-      inputStreamV1.seek(0);
-      inputStreamV2.seek(0);
-
-      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
-
-      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
-
-      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
-
-      seekPosition = 5 * KILOBYTE;
-      inputStreamV1.seek(seekPosition);
-      inputStreamV2.seek(seekPosition);
-
-      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
-
-      seekPosition = 10 * KILOBYTE;
-      inputStreamV1.seek(seekPosition);
-      inputStreamV2.seek(seekPosition);
-
-      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
-
-      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
-
-      seekPosition = 4100 * KILOBYTE;
-      inputStreamV1.seek(seekPosition);
-      inputStreamV2.seek(seekPosition);
-
-      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
-    }
-  }
-
-  private void verifyConsistentReads(FSDataInputStream inputStreamV1,
-      FSDataInputStream inputStreamV2,
-      byte[] bufferV1,
-      byte[] bufferV2) throws IOException {
-    int size = bufferV1.length;
-    final int numBytesReadV1 = inputStreamV1.read(bufferV1, 0, size);
-    assertEquals("Bytes read from V1 stream", size, numBytesReadV1);
-
-    final int numBytesReadV2 = inputStreamV2.read(bufferV2, 0, size);
-    assertEquals("Bytes read from V2 stream", size, numBytesReadV2);
-
-    assertArrayEquals("Mismatch in read data", bufferV1, bufferV2);
-  }
-
-  /**
-   * Validates the implementation of InputStream.markSupported.
-   * @throws IOException
-   */
-  @Test
-  public void test_0301_MarkSupportedV1() throws IOException {
-    validateMarkSupported(accountUsingInputStreamV1.getFileSystem());
-  }
-
-  /**
-   * Validates the implementation of InputStream.markSupported.
-   * @throws IOException
-   */
-  @Test
-  public void test_0302_MarkSupportedV2() throws IOException {
-    validateMarkSupported(accountUsingInputStreamV1.getFileSystem());
-  }
-
-  private void validateMarkSupported(FileSystem fs) throws IOException {
-    assumeHugeFileExists();
-    try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
-      assertTrue("mark is not supported", inputStream.markSupported());
-    }
-  }
-
-  /**
-   * Validates the implementation of InputStream.mark and reset
-   * for version 1 of the block blob input stream.
-   * @throws Exception
-   */
-  @Test
-  public void test_0303_MarkAndResetV1() throws Exception {
-    validateMarkAndReset(accountUsingInputStreamV1.getFileSystem());
-  }
-
-  /**
-   * Validates the implementation of InputStream.mark and reset
-   * for version 2 of the block blob input stream.
-   * @throws Exception
-   */
-  @Test
-  public void test_0304_MarkAndResetV2() throws Exception {
-    validateMarkAndReset(accountUsingInputStreamV2.getFileSystem());
-  }
-
-  private void validateMarkAndReset(FileSystem fs) throws Exception {
-    assumeHugeFileExists();
-    try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
-      inputStream.mark(KILOBYTE - 1);
-
-      byte[] buffer = new byte[KILOBYTE];
-      int bytesRead = inputStream.read(buffer);
-      assertEquals(buffer.length, bytesRead);
-
-      inputStream.reset();
-      assertEquals("rest -> pos 0", 0, inputStream.getPos());
-
-      inputStream.mark(8 * KILOBYTE - 1);
-
-      buffer = new byte[8 * KILOBYTE];
-      bytesRead = inputStream.read(buffer);
-      assertEquals(buffer.length, bytesRead);
-
-      intercept(IOException.class,
-          "Resetting to invalid mark",
-          new Callable<FSDataInputStream>() {
-            @Override
-            public FSDataInputStream call() throws Exception {
-              inputStream.reset();
-              return inputStream;
-            }
-          }
-      );
-    }
-  }
-
-  /**
-   * Validates the implementation of Seekable.seekToNewSource, which should
-   * return false for version 1 of the block blob input stream.
-   * @throws IOException
-   */
-  @Test
-  public void test_0305_SeekToNewSourceV1() throws IOException {
-    validateSeekToNewSource(accountUsingInputStreamV1.getFileSystem());
-  }
-
-  /**
-   * Validates the implementation of Seekable.seekToNewSource, which should
-   * return false for version 2 of the block blob input stream.
-   * @throws IOException
-   */
-  @Test
-  public void test_0306_SeekToNewSourceV2() throws IOException {
-    validateSeekToNewSource(accountUsingInputStreamV2.getFileSystem());
-  }
-
-  private void validateSeekToNewSource(FileSystem fs) throws IOException {
-    assumeHugeFileExists();
-    try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
-      assertFalse(inputStream.seekToNewSource(0));
-    }
-  }
-
-  /**
-   * Validates the implementation of InputStream.skip and ensures there is no
-   * network I/O for version 1 of the block blob input stream.
-   * @throws Exception
-   */
-  @Test
-  public void test_0307_SkipBoundsV1() throws Exception {
-    validateSkipBounds(accountUsingInputStreamV1.getFileSystem());
-  }
-
-  /**
-   * Validates the implementation of InputStream.skip and ensures there is no
-   * network I/O for version 2 of the block blob input stream.
-   * @throws Exception
-   */
-  @Test
-  public void test_0308_SkipBoundsV2() throws Exception {
-    validateSkipBounds(accountUsingInputStreamV2.getFileSystem());
-  }
-
-  private void validateSkipBounds(FileSystem fs) throws Exception {
-    assumeHugeFileExists();
-    try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
-      NanoTimer timer = new NanoTimer();
-
-      long skipped = inputStream.skip(-1);
-      assertEquals(0, skipped);
-
-      skipped = inputStream.skip(0);
-      assertEquals(0, skipped);
-
-      assertTrue(testFileLength > 0);
-
-      skipped = inputStream.skip(testFileLength);
-      assertEquals(testFileLength, skipped);
-
-      intercept(EOFException.class,
-          new Callable<Long>() {
-            @Override
-            public Long call() throws Exception {
-              return inputStream.skip(1);
-            }
-          }
-      );
-      long elapsedTimeMs = timer.elapsedTimeMs();
-      assertTrue(
-          String.format(
-              "There should not be any network I/O (elapsedTimeMs=%1$d).",
-              elapsedTimeMs),
-          elapsedTimeMs < 20);
-    }
-  }
-
-  /**
-   * Validates the implementation of Seekable.seek and ensures there is no
-   * network I/O for forward seek.
-   * @throws Exception
-   */
-  @Test
-  public void test_0309_SeekBoundsV1() throws Exception {
-    validateSeekBounds(accountUsingInputStreamV1.getFileSystem());
-  }
-
-  /**
-   * Validates the implementation of Seekable.seek and ensures there is no
-   * network I/O for forward seek.
-   * @throws Exception
-   */
-  @Test
-  public void test_0310_SeekBoundsV2() throws Exception {
-    validateSeekBounds(accountUsingInputStreamV2.getFileSystem());
-  }
-
-  private void validateSeekBounds(FileSystem fs) throws Exception {
-    assumeHugeFileExists();
-    try (
-        FSDataInputStream inputStream = fs.open(TEST_FILE_PATH);
-    ) {
-      NanoTimer timer = new NanoTimer();
-
-      inputStream.seek(0);
-      assertEquals(0, inputStream.getPos());
-
-      intercept(EOFException.class,
-          FSExceptionMessages.NEGATIVE_SEEK,
-          new Callable<FSDataInputStream>() {
-            @Override
-            public FSDataInputStream call() throws Exception {
-              inputStream.seek(-1);
-              return inputStream;
-            }
-          }
-      );
-
-      assertTrue("Test file length only " + testFileLength, testFileLength > 0);
-      inputStream.seek(testFileLength);
-      assertEquals(testFileLength, inputStream.getPos());
-
-      intercept(EOFException.class,
-          FSExceptionMessages.CANNOT_SEEK_PAST_EOF,
-          new Callable<FSDataInputStream>() {
-            @Override
-            public FSDataInputStream call() throws Exception {
-              inputStream.seek(testFileLength + 1);
-              return inputStream;
-            }
-          }
-      );
-
-      long elapsedTimeMs = timer.elapsedTimeMs();
-      assertTrue(
-          String.format(
-              "There should not be any network I/O (elapsedTimeMs=%1$d).",
-              elapsedTimeMs),
-          elapsedTimeMs < 20);
-    }
-  }
-
-  /**
-   * Validates the implementation of Seekable.seek, Seekable.getPos,
-   * and InputStream.available.
-   * @throws Exception
-   */
-  @Test
-  public void test_0311_SeekAndAvailableAndPositionV1() throws Exception {
-    validateSeekAndAvailableAndPosition(
-        accountUsingInputStreamV1.getFileSystem());
-  }
-
-  /**
-   * Validates the implementation of Seekable.seek, Seekable.getPos,
-   * and InputStream.available.
-   * @throws Exception
-   */
-  @Test
-  public void test_0312_SeekAndAvailableAndPositionV2() throws Exception {
-    validateSeekAndAvailableAndPosition(
-        accountUsingInputStreamV2.getFileSystem());
-  }
-
-  private void validateSeekAndAvailableAndPosition(FileSystem fs)
-      throws Exception {
-    assumeHugeFileExists();
-    try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
-      byte[] expected1 = {(byte) 'a', (byte) 'b', (byte) 'c'};
-      byte[] expected2 = {(byte) 'd', (byte) 'e', (byte) 'f'};
-      byte[] expected3 = {(byte) 'b', (byte) 'c', (byte) 'd'};
-      byte[] expected4 = {(byte) 'g', (byte) 'h', (byte) 'i'};
-      byte[] buffer = new byte[3];
-
-      int bytesRead = inputStream.read(buffer);
-      assertEquals(buffer.length, bytesRead);
-      assertArrayEquals(expected1, buffer);
-      assertEquals(buffer.length, inputStream.getPos());
-      assertEquals(testFileLength - inputStream.getPos(),
-          inputStream.available());
-
-      bytesRead = inputStream.read(buffer);
-      assertEquals(buffer.length, bytesRead);
-      assertArrayEquals(expected2, buffer);
-      assertEquals(2 * buffer.length, inputStream.getPos());
-      assertEquals(testFileLength - inputStream.getPos(),
-          inputStream.available());
-
-      // reverse seek
-      int seekPos = 0;
-      inputStream.seek(seekPos);
-
-      bytesRead = inputStream.read(buffer);
-      assertEquals(buffer.length, bytesRead);
-      assertArrayEquals(expected1, buffer);
-      assertEquals(buffer.length + seekPos, inputStream.getPos());
-      assertEquals(testFileLength - inputStream.getPos(),
-          inputStream.available());
-
-      // reverse seek
-      seekPos = 1;
-      inputStream.seek(seekPos);
-
-      bytesRead = inputStream.read(buffer);
-      assertEquals(buffer.length, bytesRead);
-      assertArrayEquals(expected3, buffer);
-      assertEquals(buffer.length + seekPos, inputStream.getPos());
-      assertEquals(testFileLength - inputStream.getPos(),
-          inputStream.available());
-
-      // forward seek
-      seekPos = 6;
-      inputStream.seek(seekPos);
-
-      bytesRead = inputStream.read(buffer);
-      assertEquals(buffer.length, bytesRead);
-      assertArrayEquals(expected4, buffer);
-      assertEquals(buffer.length + seekPos, inputStream.getPos());
-      assertEquals(testFileLength - inputStream.getPos(),
-          inputStream.available());
-    }
-  }
-
-  /**
-   * Validates the implementation of InputStream.skip, Seekable.getPos,
-   * and InputStream.available.
-   * @throws IOException
-   */
-  @Test
-  public void test_0313_SkipAndAvailableAndPositionV1() throws IOException {
-    validateSkipAndAvailableAndPosition(
-        accountUsingInputStreamV1.getFileSystem());
-  }
-
-  /**
-   * Validates the implementation of InputStream.skip, Seekable.getPos,
-   * and InputStream.available.
-   * @throws IOException
-   */
-  @Test
-  public void test_0314_SkipAndAvailableAndPositionV2() throws IOException {
-    validateSkipAndAvailableAndPosition(
-        accountUsingInputStreamV1.getFileSystem());
-  }
-
-  private void validateSkipAndAvailableAndPosition(FileSystem fs)
-      throws IOException {
-    assumeHugeFileExists();
-    try (
-        FSDataInputStream inputStream = fs.open(TEST_FILE_PATH);
-    ) {
-      byte[] expected1 = {(byte) 'a', (byte) 'b', (byte) 'c'};
-      byte[] expected2 = {(byte) 'd', (byte) 'e', (byte) 'f'};
-      byte[] expected3 = {(byte) 'b', (byte) 'c', (byte) 'd'};
-      byte[] expected4 = {(byte) 'g', (byte) 'h', (byte) 'i'};
-
-      assertEquals(testFileLength, inputStream.available());
-      assertEquals(0, inputStream.getPos());
-
-      int n = 3;
-      long skipped = inputStream.skip(n);
-
-      assertEquals(skipped, inputStream.getPos());
-      assertEquals(testFileLength - inputStream.getPos(),
-          inputStream.available());
-      assertEquals(skipped, n);
-
-      byte[] buffer = new byte[3];
-      int bytesRead = inputStream.read(buffer);
-      assertEquals(buffer.length, bytesRead);
-      assertArrayEquals(expected2, buffer);
-      assertEquals(buffer.length + skipped, inputStream.getPos());
-      assertEquals(testFileLength - inputStream.getPos(),
-          inputStream.available());
-
-      // does skip still work after seek?
-      int seekPos = 1;
-      inputStream.seek(seekPos);
-
-      bytesRead = inputStream.read(buffer);
-      assertEquals(buffer.length, bytesRead);
-      assertArrayEquals(expected3, buffer);
-      assertEquals(buffer.length + seekPos, inputStream.getPos());
-      assertEquals(testFileLength - inputStream.getPos(),
-          inputStream.available());
-
-      long currentPosition = inputStream.getPos();
-      n = 2;
-      skipped = inputStream.skip(n);
-
-      assertEquals(currentPosition + skipped, inputStream.getPos());
-      assertEquals(testFileLength - inputStream.getPos(),
-          inputStream.available());
-      assertEquals(skipped, n);
-
-      bytesRead = inputStream.read(buffer);
-      assertEquals(buffer.length, bytesRead);
-      assertArrayEquals(expected4, buffer);
-      assertEquals(buffer.length + skipped + currentPosition,
-          inputStream.getPos());
-      assertEquals(testFileLength - inputStream.getPos(),
-          inputStream.available());
-    }
-  }
-
-  /**
-   * Ensures parity in the performance of sequential read for
-   * version 1 and version 2 of the block blob input stream.
-   * @throws IOException
-   */
-  @Test
-  public void test_0315_SequentialReadPerformance() throws IOException {
-    assumeHugeFileExists();
-    final int maxAttempts = 10;
-    final double maxAcceptableRatio = 1.01;
-    double v1ElapsedMs = 0, v2ElapsedMs = 0;
-    double ratio = Double.MAX_VALUE;
-    for (int i = 0; i < maxAttempts && ratio >= maxAcceptableRatio; i++) {
-      v1ElapsedMs = sequentialRead(1,
-          accountUsingInputStreamV1.getFileSystem(), false);
-      v2ElapsedMs = sequentialRead(2,
-          accountUsingInputStreamV2.getFileSystem(), false);
-      ratio = v2ElapsedMs / v1ElapsedMs;
-      LOG.info(String.format(
-          "v1ElapsedMs=%1$d, v2ElapsedMs=%2$d, ratio=%3$.2f",
-          (long) v1ElapsedMs,
-          (long) v2ElapsedMs,
-          ratio));
-    }
-    assertTrue(String.format(
-        "Performance of version 2 is not acceptable: v1ElapsedMs=%1$d,"
-            + " v2ElapsedMs=%2$d, ratio=%3$.2f",
-        (long) v1ElapsedMs,
-        (long) v2ElapsedMs,
-        ratio),
-        ratio < maxAcceptableRatio);
-  }
-
-  /**
-   * Ensures parity in the performance of sequential read after reverse seek for
-   * version 2 of the block blob input stream.
-   * @throws IOException
-   */
-  @Test
-  public void test_0316_SequentialReadAfterReverseSeekPerformanceV2()
-      throws IOException {
-    assumeHugeFileExists();
-    final int maxAttempts = 10;
-    final double maxAcceptableRatio = 1.01;
-    double beforeSeekElapsedMs = 0, afterSeekElapsedMs = 0;
-    double ratio = Double.MAX_VALUE;
-    for (int i = 0; i < maxAttempts && ratio >= maxAcceptableRatio; i++) {
-      beforeSeekElapsedMs = sequentialRead(2,
-          accountUsingInputStreamV2.getFileSystem(), false);
-      afterSeekElapsedMs = sequentialRead(2,
-          accountUsingInputStreamV2.getFileSystem(), true);
-      ratio = afterSeekElapsedMs / beforeSeekElapsedMs;
-      LOG.info(String.format(
-          "beforeSeekElapsedMs=%1$d, afterSeekElapsedMs=%2$d, ratio=%3$.2f",
-          (long) beforeSeekElapsedMs,
-          (long) afterSeekElapsedMs,
-          ratio));
-    }
-    assertTrue(String.format(
-        "Performance of version 2 after reverse seek is not acceptable:"
-            + " beforeSeekElapsedMs=%1$d, afterSeekElapsedMs=%2$d,"
-            + " ratio=%3$.2f",
-        (long) beforeSeekElapsedMs,
-        (long) afterSeekElapsedMs,
-        ratio),
-        ratio < maxAcceptableRatio);
-  }
-
-  private long sequentialRead(int version,
-      FileSystem fs,
-      boolean afterReverseSeek) throws IOException {
-    byte[] buffer = new byte[16 * KILOBYTE];
-    long totalBytesRead = 0;
-    long bytesRead = 0;
-
-    try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
-      if (afterReverseSeek) {
-        while (bytesRead > 0 && totalBytesRead < 4 * MEGABYTE) {
-          bytesRead = inputStream.read(buffer);
-          totalBytesRead += bytesRead;
-        }
-        totalBytesRead = 0;
-        inputStream.seek(0);
-      }
-
-      NanoTimer timer = new NanoTimer();
-      while ((bytesRead = inputStream.read(buffer)) > 0) {
-        totalBytesRead += bytesRead;
-      }
-      long elapsedTimeMs = timer.elapsedTimeMs();
-
-      LOG.info(String.format(
-          "v%1$d: bytesRead=%2$d, elapsedMs=%3$d, Mbps=%4$.2f,"
-              + " afterReverseSeek=%5$s",
-          version,
-          totalBytesRead,
-          elapsedTimeMs,
-          toMbps(totalBytesRead, elapsedTimeMs),
-          afterReverseSeek));
-
-      assertEquals(testFileLength, totalBytesRead);
-      inputStream.close();
-      return elapsedTimeMs;
-    }
-  }
-
-  @Test
-  public void test_0317_RandomReadPerformance() throws IOException {
-    assumeHugeFileExists();
-    final int maxAttempts = 10;
-    final double maxAcceptableRatio = 0.10;
-    double v1ElapsedMs = 0, v2ElapsedMs = 0;
-    double ratio = Double.MAX_VALUE;
-    for (int i = 0; i < maxAttempts && ratio >= maxAcceptableRatio; i++) {
-      v1ElapsedMs = randomRead(1,
-          accountUsingInputStreamV1.getFileSystem());
-      v2ElapsedMs = randomRead(2,
-          accountUsingInputStreamV2.getFileSystem());
-      ratio = v2ElapsedMs / v1ElapsedMs;
-      LOG.info(String.format(
-          "v1ElapsedMs=%1$d, v2ElapsedMs=%2$d, ratio=%3$.2f",
-          (long) v1ElapsedMs,
-          (long) v2ElapsedMs,
-          ratio));
-    }
-    assertTrue(String.format(
-        "Performance of version 2 is not acceptable: v1ElapsedMs=%1$d,"
-            + " v2ElapsedMs=%2$d, ratio=%3$.2f",
-        (long) v1ElapsedMs,
-        (long) v2ElapsedMs,
-        ratio),
-        ratio < maxAcceptableRatio);
-  }
-
-  private long randomRead(int version, FileSystem fs) throws IOException {
-    assumeHugeFileExists();
-    final int minBytesToRead = 2 * MEGABYTE;
-    Random random = new Random();
-    byte[] buffer = new byte[8 * KILOBYTE];
-    long totalBytesRead = 0;
-    long bytesRead = 0;
-    try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
-      NanoTimer timer = new NanoTimer();
-
-      do {
-        bytesRead = inputStream.read(buffer);
-        totalBytesRead += bytesRead;
-        inputStream.seek(random.nextInt(
-            (int) (testFileLength - buffer.length)));
-      } while (bytesRead > 0 && totalBytesRead < minBytesToRead);
-
-      long elapsedTimeMs = timer.elapsedTimeMs();
-
-      inputStream.close();
-
-      LOG.info(String.format(
-          "v%1$d: totalBytesRead=%2$d, elapsedTimeMs=%3$d, Mbps=%4$.2f",
-          version,
-          totalBytesRead,
-          elapsedTimeMs,
-          toMbps(totalBytesRead, elapsedTimeMs)));
-
-      assertTrue(minBytesToRead <= totalBytesRead);
-
-      return elapsedTimeMs;
-    }
-  }
-
-  @Test
-  public void test_999_DeleteHugeFiles() throws IOException {
-    ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
-    fs.delete(TEST_FILE_PATH, false);
-    timer.end("time to delete %s", TEST_FILE_PATH);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestClientThrottlingAnalyzer.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestClientThrottlingAnalyzer.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestClientThrottlingAnalyzer.java
index 307e5af..c2496d7 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestClientThrottlingAnalyzer.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestClientThrottlingAnalyzer.java
@@ -21,13 +21,10 @@ package org.apache.hadoop.fs.azure;
 import org.apache.hadoop.fs.contract.ContractTestUtils.NanoTimer;
 import org.junit.Test;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-
 /**
  * Tests for <code>ClientThrottlingAnalyzer</code>.
  */
-public class TestClientThrottlingAnalyzer {
+public class TestClientThrottlingAnalyzer extends AbstractWasbTestWithTimeout {
   private static final int ANALYSIS_PERIOD = 1000;
   private static final int ANALYSIS_PERIOD_PLUS_10_PERCENT = ANALYSIS_PERIOD
       + ANALYSIS_PERIOD / 10;


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


[26/50] [abbrv] hadoop git commit: HADOOP-14553. Add (parallelized) integration tests to hadoop-azure Contributed by Steve Loughran

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java
new file mode 100644
index 0000000..f969968
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java
@@ -0,0 +1,236 @@
+/**
+ * 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.azure;
+
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+import org.junit.Test;
+
+import com.microsoft.azure.storage.StorageException;
+
+/**
+ * Tests the Native Azure file system (WASB) against an actual blob store.
+ */
+public class ITestNativeAzureFileSystemLive extends
+    NativeAzureFileSystemBaseTest {
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create();
+  }
+
+  @Test
+  public void testLazyRenamePendingCanOverwriteExistingFile()
+    throws Exception {
+    final String srcFile = "srcFile";
+    final String dstFile = "dstFile";
+    Path srcPath = path(srcFile);
+    FSDataOutputStream srcStream = fs.create(srcPath);
+    assertTrue(fs.exists(srcPath));
+    Path dstPath = path(dstFile);
+    FSDataOutputStream dstStream = fs.create(dstPath);
+    assertTrue(fs.exists(dstPath));
+    NativeAzureFileSystem nfs = fs;
+    final String fullSrcKey = nfs.pathToKey(nfs.makeAbsolute(srcPath));
+    final String fullDstKey = nfs.pathToKey(nfs.makeAbsolute(dstPath));
+    nfs.getStoreInterface().rename(fullSrcKey, fullDstKey, true, null);
+    assertTrue(fs.exists(dstPath));
+    assertFalse(fs.exists(srcPath));
+    IOUtils.cleanupWithLogger(null, srcStream);
+    IOUtils.cleanupWithLogger(null, dstStream);
+  }
+  /**
+   * Tests fs.delete() function to delete a blob when another blob is holding a
+   * lease on it. Delete if called without a lease should fail if another process
+   * is holding a lease and throw appropriate exception
+   * This is a scenario that would happen in HMaster startup when it tries to
+   * clean up the temp dirs while the HMaster process which was killed earlier
+   * held lease on the blob when doing some DDL operation
+   */
+  @Test
+  public void testDeleteThrowsExceptionWithLeaseExistsErrorMessage()
+      throws Exception {
+    LOG.info("Starting test");
+    // Create the file
+    Path path = methodPath();
+    fs.create(path);
+    assertPathExists("test file", path);
+    NativeAzureFileSystem nfs = fs;
+    final String fullKey = nfs.pathToKey(nfs.makeAbsolute(path));
+    final AzureNativeFileSystemStore store = nfs.getStore();
+
+    // Acquire the lease on the file in a background thread
+    final CountDownLatch leaseAttemptComplete = new CountDownLatch(1);
+    final CountDownLatch beginningDeleteAttempt = new CountDownLatch(1);
+    Thread t = new Thread() {
+      @Override
+      public void run() {
+        // Acquire the lease and then signal the main test thread.
+        SelfRenewingLease lease = null;
+        try {
+          lease = store.acquireLease(fullKey);
+          LOG.info("Lease acquired: " + lease.getLeaseID());
+        } catch (AzureException e) {
+          LOG.warn("Lease acqusition thread unable to acquire lease", e);
+        } finally {
+          leaseAttemptComplete.countDown();
+        }
+
+        // Wait for the main test thread to signal it will attempt the delete.
+        try {
+          beginningDeleteAttempt.await();
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+        }
+
+        // Keep holding the lease past the lease acquisition retry interval, so
+        // the test covers the case of delete retrying to acquire the lease.
+        try {
+          Thread.sleep(SelfRenewingLease.LEASE_ACQUIRE_RETRY_INTERVAL * 3);
+        } catch (InterruptedException ex) {
+          Thread.currentThread().interrupt();
+        }
+
+        try {
+          if (lease != null){
+            LOG.info("Freeing lease");
+            lease.free();
+          }
+        } catch (StorageException se) {
+          LOG.warn("Unable to free lease.", se);
+        }
+      }
+    };
+
+    // Start the background thread and wait for it to signal the lease is held.
+    t.start();
+    try {
+      leaseAttemptComplete.await();
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+    }
+
+    // Try to delete the same file
+    beginningDeleteAttempt.countDown();
+    store.delete(fullKey);
+
+    // At this point file SHOULD BE DELETED
+    assertPathDoesNotExist("Leased path", path);
+  }
+
+  /**
+   * Check that isPageBlobKey works as expected. This assumes that
+   * in the test configuration, the list of supported page blob directories
+   * only includes "pageBlobs". That's why this test is made specific
+   * to this subclass.
+   */
+  @Test
+  public void testIsPageBlobKey() {
+    AzureNativeFileSystemStore store = fs.getStore();
+
+    // Use literal strings so it's easier to understand the tests.
+    // In case the constant changes, we want to know about it so we can update this test.
+    assertEquals(AzureBlobStorageTestAccount.DEFAULT_PAGE_BLOB_DIRECTORY, "pageBlobs");
+
+    // URI prefix for test environment.
+    String uriPrefix = "file:///";
+
+    // negative tests
+    String[] negativeKeys = { "", "/", "bar", "bar/", "bar/pageBlobs", "bar/pageBlobs/foo",
+        "bar/pageBlobs/foo/", "/pageBlobs/", "/pageBlobs", "pageBlobs", "pageBlobsxyz/" };
+    for (String s : negativeKeys) {
+      assertFalse(store.isPageBlobKey(s));
+      assertFalse(store.isPageBlobKey(uriPrefix + s));
+    }
+
+    // positive tests
+    String[] positiveKeys = { "pageBlobs/", "pageBlobs/foo/", "pageBlobs/foo/bar/" };
+    for (String s : positiveKeys) {
+      assertTrue(store.isPageBlobKey(s));
+      assertTrue(store.isPageBlobKey(uriPrefix + s));
+    }
+  }
+
+  /**
+   * Test that isAtomicRenameKey() works as expected.
+   */
+  @Test
+  public void testIsAtomicRenameKey() {
+
+    AzureNativeFileSystemStore store = fs.getStore();
+
+    // We want to know if the default configuration changes so we can fix
+    // this test.
+    assertEquals(AzureBlobStorageTestAccount.DEFAULT_ATOMIC_RENAME_DIRECTORIES,
+        "/atomicRenameDir1,/atomicRenameDir2");
+
+    // URI prefix for test environment.
+    String uriPrefix = "file:///";
+
+    // negative tests
+    String[] negativeKeys = { "", "/", "bar", "bar/", "bar/hbase",
+        "bar/hbase/foo", "bar/hbase/foo/", "/hbase/", "/hbase", "hbase",
+        "hbasexyz/", "foo/atomicRenameDir1/"};
+    for (String s : negativeKeys) {
+      assertFalse(store.isAtomicRenameKey(s));
+      assertFalse(store.isAtomicRenameKey(uriPrefix + s));
+    }
+
+    // Positive tests. The directories for atomic rename are /hbase
+    // plus the ones in the configuration (DEFAULT_ATOMIC_RENAME_DIRECTORIES
+    // for this test).
+    String[] positiveKeys = { "hbase/", "hbase/foo/", "hbase/foo/bar/",
+        "atomicRenameDir1/foo/", "atomicRenameDir2/bar/"};
+    for (String s : positiveKeys) {
+      assertTrue(store.isAtomicRenameKey(s));
+      assertTrue(store.isAtomicRenameKey(uriPrefix + s));
+    }
+  }
+
+  /**
+   * Tests fs.mkdir() function to create a target blob while another thread
+   * is holding the lease on the blob. mkdir should not fail since the blob
+   * already exists.
+   * This is a scenario that would happen in HBase distributed log splitting.
+   * Multiple threads will try to create and update "recovered.edits" folder
+   * under the same path.
+   */
+  @Test
+  public void testMkdirOnExistingFolderWithLease() throws Exception {
+    SelfRenewingLease lease;
+    // Create the folder
+    Path path = methodPath();
+    fs.mkdirs(path);
+    NativeAzureFileSystem nfs = fs;
+    String fullKey = nfs.pathToKey(nfs.makeAbsolute(path));
+    AzureNativeFileSystemStore store = nfs.getStore();
+    // Acquire the lease on the folder
+    lease = store.acquireLease(fullKey);
+    assertNotNull("lease ID", lease.getLeaseID() != null);
+    // Try to create the same folder
+    store.storeEmptyFolder(fullKey,
+      nfs.createPermissionStatus(FsPermission.getDirDefault()));
+    lease.free();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestOutOfBandAzureBlobOperationsLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestOutOfBandAzureBlobOperationsLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestOutOfBandAzureBlobOperationsLive.java
new file mode 100644
index 0000000..b63aaf0
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestOutOfBandAzureBlobOperationsLive.java
@@ -0,0 +1,185 @@
+/**
+ * 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.azure;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.Test;
+
+import com.microsoft.azure.storage.blob.BlobOutputStream;
+import com.microsoft.azure.storage.blob.CloudBlockBlob;
+
+/**
+ * Live blob operations.
+ */
+public class ITestOutOfBandAzureBlobOperationsLive extends AbstractWasbTestBase {
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create();
+  }
+
+  // scenario for this particular test described at MONARCH-HADOOP-764
+  // creating a file out-of-band would confuse mkdirs("<oobfilesUncleFolder>")
+  // eg oob creation of "user/<name>/testFolder/a/input/file"
+  // Then wasb creation of "user/<name>/testFolder/a/output" fails
+  @Test
+  public void outOfBandFolder_uncleMkdirs() throws Exception {
+
+    // NOTE: manual use of CloubBlockBlob targets working directory explicitly.
+    // WASB driver methods prepend working directory implicitly.
+    String workingDir = "user/"
+        + UserGroupInformation.getCurrentUser().getShortUserName() + "/";
+
+    CloudBlockBlob blob = testAccount.getBlobReference(workingDir
+        + "testFolder1/a/input/file");
+    BlobOutputStream s = blob.openOutputStream();
+    s.close();
+    assertTrue(fs.exists(new Path("testFolder1/a/input/file")));
+
+    Path targetFolder = new Path("testFolder1/a/output");
+    assertTrue(fs.mkdirs(targetFolder));
+  }
+
+  // scenario for this particular test described at MONARCH-HADOOP-764
+  @Test
+  public void outOfBandFolder_parentDelete() throws Exception {
+
+    // NOTE: manual use of CloubBlockBlob targets working directory explicitly.
+    // WASB driver methods prepend working directory implicitly.
+    String workingDir = "user/"
+        + UserGroupInformation.getCurrentUser().getShortUserName() + "/";
+    CloudBlockBlob blob = testAccount.getBlobReference(workingDir
+        + "testFolder2/a/input/file");
+    BlobOutputStream s = blob.openOutputStream();
+    s.close();
+    assertTrue(fs.exists(new Path("testFolder2/a/input/file")));
+
+    Path targetFolder = new Path("testFolder2/a/input");
+    assertTrue(fs.delete(targetFolder, true));
+  }
+
+  @Test
+  public void outOfBandFolder_rootFileDelete() throws Exception {
+
+    CloudBlockBlob blob = testAccount.getBlobReference("fileY");
+    BlobOutputStream s = blob.openOutputStream();
+    s.close();
+    assertTrue(fs.exists(new Path("/fileY")));
+    assertTrue(fs.delete(new Path("/fileY"), true));
+  }
+
+  @Test
+  public void outOfBandFolder_firstLevelFolderDelete() throws Exception {
+
+    CloudBlockBlob blob = testAccount.getBlobReference("folderW/file");
+    BlobOutputStream s = blob.openOutputStream();
+    s.close();
+    assertTrue(fs.exists(new Path("/folderW")));
+    assertTrue(fs.exists(new Path("/folderW/file")));
+    assertTrue(fs.delete(new Path("/folderW"), true));
+  }
+
+  // scenario for this particular test described at MONARCH-HADOOP-764
+  @Test
+  public void outOfBandFolder_siblingCreate() throws Exception {
+
+    // NOTE: manual use of CloubBlockBlob targets working directory explicitly.
+    // WASB driver methods prepend working directory implicitly.
+    String workingDir = "user/"
+        + UserGroupInformation.getCurrentUser().getShortUserName() + "/";
+    CloudBlockBlob blob = testAccount.getBlobReference(workingDir
+        + "testFolder3/a/input/file");
+    BlobOutputStream s = blob.openOutputStream();
+    s.close();
+    assertTrue(fs.exists(new Path("testFolder3/a/input/file")));
+
+    Path targetFile = new Path("testFolder3/a/input/file2");
+    FSDataOutputStream s2 = fs.create(targetFile);
+    s2.close();
+  }
+
+  // scenario for this particular test described at MONARCH-HADOOP-764
+  // creating a new file in the root folder
+  @Test
+  public void outOfBandFolder_create_rootDir() throws Exception {
+    Path targetFile = new Path("/newInRoot");
+    FSDataOutputStream s2 = fs.create(targetFile);
+    s2.close();
+  }
+
+  // scenario for this particular test described at MONARCH-HADOOP-764
+  @Test
+  public void outOfBandFolder_rename() throws Exception {
+
+    // NOTE: manual use of CloubBlockBlob targets working directory explicitly.
+    // WASB driver methods prepend working directory implicitly.
+    String workingDir = "user/"
+        + UserGroupInformation.getCurrentUser().getShortUserName() + "/";
+    CloudBlockBlob blob = testAccount.getBlobReference(workingDir
+        + "testFolder4/a/input/file");
+    BlobOutputStream s = blob.openOutputStream();
+    s.close();
+
+    Path srcFilePath = new Path("testFolder4/a/input/file");
+    assertTrue(fs.exists(srcFilePath));
+
+    Path destFilePath = new Path("testFolder4/a/input/file2");
+    fs.rename(srcFilePath, destFilePath);
+  }
+
+  // Verify that you can rename a file which is the only file in an implicit folder in the
+  // WASB file system.
+  // scenario for this particular test described at MONARCH-HADOOP-892
+  @Test
+  public void outOfBandSingleFile_rename() throws Exception {
+
+    //NOTE: manual use of CloubBlockBlob targets working directory explicitly.
+    //       WASB driver methods prepend working directory implicitly.
+    String workingDir = "user/" + UserGroupInformation.getCurrentUser().getShortUserName() + "/";
+    CloudBlockBlob blob = testAccount.getBlobReference(workingDir + "testFolder5/a/input/file");
+    BlobOutputStream s = blob.openOutputStream();
+    s.close();
+
+    Path srcFilePath = new Path("testFolder5/a/input/file");
+    assertTrue(fs.exists(srcFilePath));
+
+    Path destFilePath = new Path("testFolder5/file2");
+    fs.rename(srcFilePath, destFilePath);
+  }
+
+  // WASB must force explicit parent directories in create, delete, mkdirs, rename.
+  // scenario for this particular test described at MONARCH-HADOOP-764
+  @Test
+  public void outOfBandFolder_rename_rootLevelFiles() throws Exception {
+
+    // NOTE: manual use of CloubBlockBlob targets working directory explicitly.
+    // WASB driver methods prepend working directory implicitly.
+    CloudBlockBlob blob = testAccount.getBlobReference("fileX");
+    BlobOutputStream s = blob.openOutputStream();
+    s.close();
+
+    Path srcFilePath = new Path("/fileX");
+    assertTrue(fs.exists(srcFilePath));
+
+    Path destFilePath = new Path("/fileXrename");
+    fs.rename(srcFilePath, destFilePath);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestReadAndSeekPageBlobAfterWrite.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestReadAndSeekPageBlobAfterWrite.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestReadAndSeekPageBlobAfterWrite.java
new file mode 100644
index 0000000..f2af116
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestReadAndSeekPageBlobAfterWrite.java
@@ -0,0 +1,341 @@
+/**
+ * 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.azure;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Random;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+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.integration.AbstractAzureScaleTest;
+import org.apache.hadoop.util.Time;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.fs.azure.integration.AzureTestUtils .*;
+
+/**
+ * Write data into a page blob and verify you can read back all of it
+ * or just a part of it.
+ */
+public class ITestReadAndSeekPageBlobAfterWrite extends AbstractAzureScaleTest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestReadAndSeekPageBlobAfterWrite.class);
+
+  private FileSystem fs;
+  private byte[] randomData;
+
+  // Page blob physical page size
+  private static final int PAGE_SIZE = PageBlobFormatHelpers.PAGE_SIZE;
+
+  // Size of data on page (excluding header)
+  private static final int PAGE_DATA_SIZE = PAGE_SIZE - PageBlobFormatHelpers.PAGE_HEADER_SIZE;
+  private static final int MAX_BYTES = 33554432; // maximum bytes in a file that we'll test
+  private static final int MAX_PAGES = MAX_BYTES / PAGE_SIZE; // maximum number of pages we'll test
+  private Random rand = new Random();
+
+  // A key with a prefix under /pageBlobs, which for the test file system will
+  // force use of a page blob.
+  private static final String KEY = "/pageBlobs/file.dat";
+
+  // path of page blob file to read and write
+  private Path blobPath;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    fs = getTestAccount().getFileSystem();
+    // Make sure we are using an integral number of pages.
+    assertEquals(0, MAX_BYTES % PAGE_SIZE);
+
+    // load an in-memory array of random data
+    randomData = new byte[PAGE_SIZE * MAX_PAGES];
+    rand.nextBytes(randomData);
+
+    blobPath = blobPath("ITestReadAndSeekPageBlobAfterWrite");
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    deleteQuietly(fs, blobPath, true);
+    super.tearDown();
+  }
+
+  /**
+   * Make sure the file name (key) is a page blob file name. If anybody changes that,
+   * we need to come back and update this test class.
+   */
+  @Test
+  public void testIsPageBlobFileName() {
+    AzureNativeFileSystemStore store = ((NativeAzureFileSystem) fs).getStore();
+    String[] a = blobPath.toUri().getPath().split("/");
+    String key2 = a[1] + "/";
+    assertTrue("Not a page blob: " + blobPath, store.isPageBlobKey(key2));
+  }
+
+  /**
+   * For a set of different file sizes, write some random data to a page blob,
+   * read it back, and compare that what was read is the same as what was written.
+   */
+  @Test
+  public void testReadAfterWriteRandomData() throws IOException {
+
+    // local shorthand
+    final int pds = PAGE_DATA_SIZE;
+
+    // Test for sizes at and near page boundaries
+    int[] dataSizes = {
+
+        // on first page
+        0, 1, 2, 3,
+
+        // Near first physical page boundary (because the implementation
+        // stores PDS + the page header size bytes on each page).
+        pds - 1, pds, pds + 1, pds + 2, pds + 3,
+
+        // near second physical page boundary
+        (2 * pds) - 1, (2 * pds), (2 * pds) + 1, (2 * pds) + 2, (2 * pds) + 3,
+
+        // near tenth physical page boundary
+        (10 * pds) - 1, (10 * pds), (10 * pds) + 1, (10 * pds) + 2, (10 * pds) + 3,
+
+        // test one big size, >> 4MB (an internal buffer size in the code)
+        MAX_BYTES
+    };
+
+    for (int i : dataSizes) {
+      testReadAfterWriteRandomData(i);
+    }
+  }
+
+  private void testReadAfterWriteRandomData(int size) throws IOException {
+    writeRandomData(size);
+    readRandomDataAndVerify(size);
+  }
+
+  /**
+   * Read "size" bytes of data and verify that what was read and what was written
+   * are the same.
+   */
+  private void readRandomDataAndVerify(int size) throws AzureException, IOException {
+    byte[] b = new byte[size];
+    FSDataInputStream stream = fs.open(blobPath);
+    int bytesRead = stream.read(b);
+    stream.close();
+    assertEquals(bytesRead, size);
+
+    // compare the data read to the data written
+    assertTrue(comparePrefix(randomData, b, size));
+  }
+
+  // return true if the beginning "size" values of the arrays are the same
+  private boolean comparePrefix(byte[] a, byte[] b, int size) {
+    if (a.length < size || b.length < size) {
+      return false;
+    }
+    for (int i = 0; i < size; i++) {
+      if (a[i] != b[i]) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  // Write a specified amount of random data to the file path for this test class.
+  private void writeRandomData(int size) throws IOException {
+    OutputStream output = fs.create(blobPath);
+    output.write(randomData, 0, size);
+    output.close();
+  }
+
+  /**
+   * Write data to a page blob, open it, seek, and then read a range of data.
+   * Then compare that the data read from that range is the same as the data originally written.
+   */
+  @Test
+  public void testPageBlobSeekAndReadAfterWrite() throws IOException {
+    writeRandomData(PAGE_SIZE * MAX_PAGES);
+    int recordSize = 100;
+    byte[] b = new byte[recordSize];
+
+
+    try(FSDataInputStream stream = fs.open(blobPath)) {
+      // Seek to a boundary around the middle of the 6th page
+      int seekPosition = 5 * PAGE_SIZE + 250;
+      stream.seek(seekPosition);
+
+      // Read a record's worth of bytes and verify results
+      int bytesRead = stream.read(b);
+      verifyReadRandomData(b, bytesRead, seekPosition, recordSize);
+
+      // Seek to another spot and read a record greater than a page
+      seekPosition = 10 * PAGE_SIZE + 250;
+      stream.seek(seekPosition);
+      recordSize = 1000;
+      b = new byte[recordSize];
+      bytesRead = stream.read(b);
+      verifyReadRandomData(b, bytesRead, seekPosition, recordSize);
+
+      // Read the last 100 bytes of the file
+      recordSize = 100;
+      seekPosition = PAGE_SIZE * MAX_PAGES - recordSize;
+      stream.seek(seekPosition);
+      b = new byte[recordSize];
+      bytesRead = stream.read(b);
+      verifyReadRandomData(b, bytesRead, seekPosition, recordSize);
+
+      // Read past the end of the file and we should get only partial data.
+      recordSize = 100;
+      seekPosition = PAGE_SIZE * MAX_PAGES - recordSize + 50;
+      stream.seek(seekPosition);
+      b = new byte[recordSize];
+      bytesRead = stream.read(b);
+      assertEquals(50, bytesRead);
+
+      // compare last 50 bytes written with those read
+      byte[] tail = Arrays.copyOfRange(randomData, seekPosition, randomData.length);
+      assertTrue(comparePrefix(tail, b, 50));
+    }
+  }
+
+  // Verify that reading a record of data after seeking gives the expected data.
+  private void verifyReadRandomData(byte[] b, int bytesRead, int seekPosition, int recordSize) {
+    byte[] originalRecordData =
+        Arrays.copyOfRange(randomData, seekPosition, seekPosition + recordSize + 1);
+    assertEquals(recordSize, bytesRead);
+    assertTrue(comparePrefix(originalRecordData, b, recordSize));
+  }
+
+  // Test many small flushed writes interspersed with periodic hflush calls.
+  // For manual testing, increase NUM_WRITES to a large number.
+  // The goal for a long-running manual test is to make sure that it finishes
+  // and the close() call does not time out. It also facilitates debugging into
+  // hflush/hsync.
+  @Test
+  public void testManySmallWritesWithHFlush() throws IOException {
+    writeAndReadOneFile(50, 100, 20);
+  }
+
+  /**
+   * Write a total of numWrites * recordLength data to a file, read it back,
+   * and check to make sure what was read is the same as what was written.
+   * The syncInterval is the number of writes after which to call hflush to
+   * force the data to storage.
+   */
+  private void writeAndReadOneFile(int numWrites,
+      int recordLength, int syncInterval) throws IOException {
+
+    // A lower bound on the minimum time we think it will take to do
+    // a write to Azure storage.
+    final long MINIMUM_EXPECTED_TIME = 20;
+    LOG.info("Writing " + numWrites * recordLength + " bytes to " + blobPath.getName());
+    FSDataOutputStream output = fs.create(blobPath);
+    int writesSinceHFlush = 0;
+    try {
+
+      // Do a flush and hflush to exercise case for empty write queue in PageBlobOutputStream,
+      // to test concurrent execution gates.
+      output.flush();
+      output.hflush();
+      for (int i = 0; i < numWrites; i++) {
+        output.write(randomData, i * recordLength, recordLength);
+        writesSinceHFlush++;
+        output.flush();
+        if ((i % syncInterval) == 0) {
+          output.hflush();
+          writesSinceHFlush = 0;
+        }
+      }
+    } finally {
+      long start = Time.monotonicNow();
+      output.close();
+      long end = Time.monotonicNow();
+      LOG.debug("close duration = " + (end - start) + " msec.");
+      if (writesSinceHFlush > 0) {
+        assertTrue(String.format(
+            "close duration with >= 1 pending write is %d, less than minimum expected of %d",
+            end - start, MINIMUM_EXPECTED_TIME),
+            end - start >= MINIMUM_EXPECTED_TIME);
+        }
+    }
+
+    // Read the data back and check it.
+    FSDataInputStream stream = fs.open(blobPath);
+    int SIZE = numWrites * recordLength;
+    byte[] b = new byte[SIZE];
+    try {
+      stream.seek(0);
+      stream.read(b, 0, SIZE);
+      verifyReadRandomData(b, SIZE, 0, SIZE);
+    } finally {
+      stream.close();
+    }
+
+    // delete the file
+    fs.delete(blobPath, false);
+  }
+
+  // Test writing to a large file repeatedly as a stress test.
+  // Set the repetitions to a larger number for manual testing
+  // for a longer stress run.
+  @Test
+  public void testLargeFileStress() throws IOException {
+    int numWrites = 32;
+    int recordSize = 1024 * 1024;
+    int syncInterval = 10;
+    int repetitions = 1;
+    for (int i = 0; i < repetitions; i++) {
+      writeAndReadOneFile(numWrites, recordSize, syncInterval);
+    }
+  }
+  
+  // Write to a file repeatedly to verify that it extends.
+  // The page blob file should start out at 128MB and finish at 256MB.
+  public void testFileSizeExtension() throws IOException {
+    final int writeSize = 1024 * 1024;
+    final int numWrites = 129;
+    final byte dataByte = 5;
+    byte[] data = new byte[writeSize];
+    Arrays.fill(data, dataByte);
+    try (FSDataOutputStream output = fs.create(blobPath)) {
+      for (int i = 0; i < numWrites; i++) {
+        output.write(data);
+        output.hflush();
+        LOG.debug("total writes = " + (i + 1));
+      }
+    }
+
+    // Show that we wrote more than the default page blob file size.
+    assertTrue(numWrites * writeSize > PageBlobOutputStream.PAGE_BLOB_MIN_SIZE);
+
+    // Verify we can list the new size. That will prove we expanded the file.
+    FileStatus[] status = fs.listStatus(blobPath);
+    assertEquals("File size hasn't changed " + status,
+        numWrites * writeSize, status[0].getLen());
+    LOG.debug("Total bytes written to " + blobPath + " = " + status[0].getLen());
+    fs.delete(blobPath, false);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestWasbRemoteCallHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestWasbRemoteCallHelper.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestWasbRemoteCallHelper.java
new file mode 100644
index 0000000..062bc36
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestWasbRemoteCallHelper.java
@@ -0,0 +1,568 @@
+/**
+ * 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.azure;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.io.retry.RetryUtils;
+import org.apache.http.Header;
+import org.apache.http.HttpResponse;
+import org.apache.http.HttpEntity;
+import org.apache.http.HttpStatus;
+import org.apache.http.StatusLine;
+import org.apache.http.ProtocolVersion;
+import org.apache.http.ParseException;
+import org.apache.http.HeaderElement;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpGet;
+import org.hamcrest.Description;
+import org.hamcrest.TypeSafeMatcher;
+import org.junit.Assume;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.mockito.ArgumentMatcher;
+import org.mockito.Mockito;
+
+import java.io.ByteArrayInputStream;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.nio.charset.StandardCharsets;
+
+import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.KEY_USE_SECURE_MODE;
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Mockito.atLeast;
+import static org.mockito.Mockito.times;
+
+/**
+ * Test class to hold all WasbRemoteCallHelper tests.
+ */
+public class ITestWasbRemoteCallHelper
+    extends AbstractWasbTestBase {
+  public static final String EMPTY_STRING = "";
+  private static final int INVALID_HTTP_STATUS_CODE_999 = 999;
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(NativeAzureFileSystem.KEY_AZURE_AUTHORIZATION, "true");
+    conf.set(RemoteWasbAuthorizerImpl.KEY_REMOTE_AUTH_SERVICE_URLS, "http://localhost1/,http://localhost2/,http://localhost:8080");
+    return AzureBlobStorageTestAccount.create(conf);
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    boolean useSecureMode = fs.getConf().getBoolean(KEY_USE_SECURE_MODE, false);
+    boolean useAuthorization = fs.getConf()
+        .getBoolean(NativeAzureFileSystem.KEY_AZURE_AUTHORIZATION, false);
+    Assume.assumeTrue("Test valid when both SecureMode and Authorization are enabled .. skipping",
+        useSecureMode && useAuthorization);
+  }
+
+  @Rule
+  public ExpectedException expectedEx = ExpectedException.none();
+
+  /**
+   * Test invalid status-code.
+   * @throws Throwable
+   */
+  @Test // (expected = WasbAuthorizationException.class)
+  public void testInvalidStatusCode() throws Throwable {
+
+    setupExpectations();
+
+    // set up mocks
+    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
+    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
+    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any()))
+        .thenReturn(mockHttpResponse);
+    Mockito.when(mockHttpResponse.getStatusLine())
+        .thenReturn(newStatusLine(INVALID_HTTP_STATUS_CODE_999));
+    // finished setting up mocks
+
+    performop(mockHttpClient);
+  }
+
+  /**
+   * Test invalid Content-Type.
+   * @throws Throwable
+   */
+  @Test // (expected = WasbAuthorizationException.class)
+  public void testInvalidContentType() throws Throwable {
+
+    setupExpectations();
+
+    // set up mocks
+    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
+    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
+    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
+    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
+    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
+        .thenReturn(newHeader("Content-Type", "text/plain"));
+    // finished setting up mocks
+
+    performop(mockHttpClient);
+  }
+
+  /**
+   * Test missing Content-Length.
+   * @throws Throwable
+   */
+  @Test // (expected = WasbAuthorizationException.class)
+  public void testMissingContentLength() throws Throwable {
+
+    setupExpectations();
+
+    // set up mocks
+    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
+    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
+    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
+    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
+    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
+        .thenReturn(newHeader("Content-Type", "application/json"));
+    // finished setting up mocks
+
+    performop(mockHttpClient);
+  }
+
+  /**
+   * Test Content-Length exceeds max.
+   * @throws Throwable
+   */
+  @Test // (expected = WasbAuthorizationException.class)
+  public void testContentLengthExceedsMax() throws Throwable {
+
+    setupExpectations();
+
+    // set up mocks
+    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
+    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
+    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
+    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
+    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
+        .thenReturn(newHeader("Content-Type", "application/json"));
+    Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
+        .thenReturn(newHeader("Content-Length", "2048"));
+    // finished setting up mocks
+
+    performop(mockHttpClient);
+  }
+
+  /**
+   * Test invalid Content-Length value
+   * @throws Throwable
+   */
+  @Test // (expected = WasbAuthorizationException.class)
+  public void testInvalidContentLengthValue() throws Throwable {
+
+    setupExpectations();
+
+    // set up mocks
+    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
+    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
+    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
+    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
+    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
+        .thenReturn(newHeader("Content-Type", "application/json"));
+    Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
+        .thenReturn(newHeader("Content-Length", "20abc48"));
+    // finished setting up mocks
+
+    performop(mockHttpClient);
+  }
+
+  /**
+   * Test valid JSON response.
+   * @throws Throwable
+   */
+  @Test
+  public void testValidJSONResponse() throws Throwable {
+
+    // set up mocks
+    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
+
+    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
+    HttpEntity mockHttpEntity = Mockito.mock(HttpEntity.class);
+
+    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
+    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
+    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
+        .thenReturn(newHeader("Content-Type", "application/json"));
+    Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
+        .thenReturn(newHeader("Content-Length", "1024"));
+    Mockito.when(mockHttpResponse.getEntity()).thenReturn(mockHttpEntity);
+    Mockito.when(mockHttpEntity.getContent())
+        .thenReturn(new ByteArrayInputStream(validJsonResponse().getBytes(StandardCharsets.UTF_8)))
+        .thenReturn(new ByteArrayInputStream(validJsonResponse().getBytes(StandardCharsets.UTF_8)))
+        .thenReturn(new ByteArrayInputStream(validJsonResponse().getBytes(StandardCharsets.UTF_8)));
+    // finished setting up mocks
+
+    performop(mockHttpClient);
+  }
+
+  /**
+   * Test malformed JSON response.
+   * @throws Throwable
+   */
+  @Test // (expected = WasbAuthorizationException.class)
+  public void testMalFormedJSONResponse() throws Throwable {
+
+    expectedEx.expect(WasbAuthorizationException.class);
+    expectedEx.expectMessage("com.fasterxml.jackson.core.JsonParseException: Unexpected end-of-input in FIELD_NAME");
+
+    // set up mocks
+    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
+
+    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
+    HttpEntity mockHttpEntity = Mockito.mock(HttpEntity.class);
+
+    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
+    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
+    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
+        .thenReturn(newHeader("Content-Type", "application/json"));
+    Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
+        .thenReturn(newHeader("Content-Length", "1024"));
+    Mockito.when(mockHttpResponse.getEntity()).thenReturn(mockHttpEntity);
+    Mockito.when(mockHttpEntity.getContent())
+        .thenReturn(new ByteArrayInputStream(malformedJsonResponse().getBytes(StandardCharsets.UTF_8)));
+    // finished setting up mocks
+
+    performop(mockHttpClient);
+  }
+
+  /**
+   * Test valid JSON response failure response code.
+   * @throws Throwable
+   */
+  @Test // (expected = WasbAuthorizationException.class)
+  public void testFailureCodeJSONResponse() throws Throwable {
+
+    expectedEx.expect(WasbAuthorizationException.class);
+    expectedEx.expectMessage("Remote authorization service encountered an error Unauthorized");
+
+    // set up mocks
+    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
+
+    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
+    HttpEntity mockHttpEntity = Mockito.mock(HttpEntity.class);
+
+    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
+    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
+    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
+        .thenReturn(newHeader("Content-Type", "application/json"));
+    Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
+        .thenReturn(newHeader("Content-Length", "1024"));
+    Mockito.when(mockHttpResponse.getEntity()).thenReturn(mockHttpEntity);
+    Mockito.when(mockHttpEntity.getContent())
+        .thenReturn(new ByteArrayInputStream(failureCodeJsonResponse().getBytes(StandardCharsets.UTF_8)));
+    // finished setting up mocks
+
+    performop(mockHttpClient);
+  }
+
+  @Test
+  public void testWhenOneInstanceIsDown() throws Throwable {
+
+    boolean isAuthorizationCachingEnabled = fs.getConf().getBoolean(CachingAuthorizer.KEY_AUTH_SERVICE_CACHING_ENABLE, false);
+
+    // set up mocks
+    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
+    HttpEntity mockHttpEntity = Mockito.mock(HttpEntity.class);
+
+    HttpResponse mockHttpResponseService1 = Mockito.mock(HttpResponse.class);
+    Mockito.when(mockHttpResponseService1.getStatusLine())
+        .thenReturn(newStatusLine(HttpStatus.SC_INTERNAL_SERVER_ERROR));
+    Mockito.when(mockHttpResponseService1.getFirstHeader("Content-Type"))
+        .thenReturn(newHeader("Content-Type", "application/json"));
+    Mockito.when(mockHttpResponseService1.getFirstHeader("Content-Length"))
+        .thenReturn(newHeader("Content-Length", "1024"));
+    Mockito.when(mockHttpResponseService1.getEntity())
+        .thenReturn(mockHttpEntity);
+
+    HttpResponse mockHttpResponseService2 = Mockito.mock(HttpResponse.class);
+    Mockito.when(mockHttpResponseService2.getStatusLine())
+        .thenReturn(newStatusLine(HttpStatus.SC_OK));
+    Mockito.when(mockHttpResponseService2.getFirstHeader("Content-Type"))
+        .thenReturn(newHeader("Content-Type", "application/json"));
+    Mockito.when(mockHttpResponseService2.getFirstHeader("Content-Length"))
+        .thenReturn(newHeader("Content-Length", "1024"));
+    Mockito.when(mockHttpResponseService2.getEntity())
+        .thenReturn(mockHttpEntity);
+
+    HttpResponse mockHttpResponseServiceLocal = Mockito.mock(HttpResponse.class);
+    Mockito.when(mockHttpResponseServiceLocal.getStatusLine())
+        .thenReturn(newStatusLine(HttpStatus.SC_INTERNAL_SERVER_ERROR));
+    Mockito.when(mockHttpResponseServiceLocal.getFirstHeader("Content-Type"))
+        .thenReturn(newHeader("Content-Type", "application/json"));
+    Mockito.when(mockHttpResponseServiceLocal.getFirstHeader("Content-Length"))
+        .thenReturn(newHeader("Content-Length", "1024"));
+    Mockito.when(mockHttpResponseServiceLocal.getEntity())
+        .thenReturn(mockHttpEntity);
+
+
+
+    class HttpGetForService1 extends ArgumentMatcher<HttpGet>{
+      @Override public boolean matches(Object o) {
+        return checkHttpGetMatchHost((HttpGet) o, "localhost1");
+      }
+    }
+    class HttpGetForService2 extends ArgumentMatcher<HttpGet>{
+      @Override public boolean matches(Object o) {
+        return checkHttpGetMatchHost((HttpGet) o, "localhost2");
+      }
+    }
+    class HttpGetForServiceLocal extends ArgumentMatcher<HttpGet>{
+      @Override public boolean matches(Object o) {
+        try {
+          return checkHttpGetMatchHost((HttpGet) o, InetAddress.getLocalHost().getCanonicalHostName());
+        } catch (UnknownHostException e) {
+          return checkHttpGetMatchHost((HttpGet) o, "localhost");
+        }
+      }
+    }
+    Mockito.when(mockHttpClient.execute(argThat(new HttpGetForService1())))
+        .thenReturn(mockHttpResponseService1);
+    Mockito.when(mockHttpClient.execute(argThat(new HttpGetForService2())))
+        .thenReturn(mockHttpResponseService2);
+    Mockito.when(mockHttpClient.execute(argThat(new HttpGetForServiceLocal())))
+        .thenReturn(mockHttpResponseServiceLocal);
+
+    //Need 2 times because performop()  does 2 fs operations.
+    Mockito.when(mockHttpEntity.getContent())
+        .thenReturn(new ByteArrayInputStream(validJsonResponse()
+            .getBytes(StandardCharsets.UTF_8)))
+        .thenReturn(new ByteArrayInputStream(validJsonResponse()
+            .getBytes(StandardCharsets.UTF_8)))
+        .thenReturn(new ByteArrayInputStream(validJsonResponse()
+            .getBytes(StandardCharsets.UTF_8)));
+    // finished setting up mocks
+
+    performop(mockHttpClient);
+
+    int expectedNumberOfInvocations = isAuthorizationCachingEnabled ? 1 : 2;
+    Mockito.verify(mockHttpClient, times(expectedNumberOfInvocations)).execute(Mockito.argThat(new HttpGetForServiceLocal()));
+    Mockito.verify(mockHttpClient, times(expectedNumberOfInvocations)).execute(Mockito.argThat(new HttpGetForService2()));
+  }
+
+  @Test
+  public void testWhenServiceInstancesAreDown() throws Throwable {
+    //expectedEx.expect(WasbAuthorizationException.class);
+    // set up mocks
+    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
+    HttpEntity mockHttpEntity = Mockito.mock(HttpEntity.class);
+
+    HttpResponse mockHttpResponseService1 = Mockito.mock(HttpResponse.class);
+    Mockito.when(mockHttpResponseService1.getStatusLine())
+        .thenReturn(newStatusLine(HttpStatus.SC_INTERNAL_SERVER_ERROR));
+    Mockito.when(mockHttpResponseService1.getFirstHeader("Content-Type"))
+        .thenReturn(newHeader("Content-Type", "application/json"));
+    Mockito.when(mockHttpResponseService1.getFirstHeader("Content-Length"))
+        .thenReturn(newHeader("Content-Length", "1024"));
+    Mockito.when(mockHttpResponseService1.getEntity())
+        .thenReturn(mockHttpEntity);
+
+    HttpResponse mockHttpResponseService2 = Mockito.mock(HttpResponse.class);
+    Mockito.when(mockHttpResponseService2.getStatusLine())
+        .thenReturn(newStatusLine(
+        HttpStatus.SC_INTERNAL_SERVER_ERROR));
+    Mockito.when(mockHttpResponseService2.getFirstHeader("Content-Type"))
+        .thenReturn(newHeader("Content-Type", "application/json"));
+    Mockito.when(mockHttpResponseService2.getFirstHeader("Content-Length"))
+        .thenReturn(newHeader("Content-Length", "1024"));
+    Mockito.when(mockHttpResponseService2.getEntity())
+        .thenReturn(mockHttpEntity);
+
+    HttpResponse mockHttpResponseService3 = Mockito.mock(HttpResponse.class);
+    Mockito.when(mockHttpResponseService3.getStatusLine())
+        .thenReturn(newStatusLine(
+            HttpStatus.SC_INTERNAL_SERVER_ERROR));
+    Mockito.when(mockHttpResponseService3.getFirstHeader("Content-Type"))
+        .thenReturn(newHeader("Content-Type", "application/json"));
+    Mockito.when(mockHttpResponseService3.getFirstHeader("Content-Length"))
+        .thenReturn(newHeader("Content-Length", "1024"));
+    Mockito.when(mockHttpResponseService3.getEntity())
+        .thenReturn(mockHttpEntity);
+
+    class HttpGetForService1 extends ArgumentMatcher<HttpGet>{
+      @Override public boolean matches(Object o) {
+        return checkHttpGetMatchHost((HttpGet) o, "localhost1");
+      }
+    }
+    class HttpGetForService2 extends ArgumentMatcher<HttpGet>{
+      @Override public boolean matches(Object o) {
+        return checkHttpGetMatchHost((HttpGet) o, "localhost2");
+      }
+    }
+    class HttpGetForService3 extends ArgumentMatcher<HttpGet> {
+      @Override public boolean matches(Object o){
+        try {
+          return checkHttpGetMatchHost((HttpGet) o, InetAddress.getLocalHost().getCanonicalHostName());
+        } catch (UnknownHostException e) {
+          return checkHttpGetMatchHost((HttpGet) o, "localhost");
+        }
+      }
+    }
+    Mockito.when(mockHttpClient.execute(argThat(new HttpGetForService1())))
+        .thenReturn(mockHttpResponseService1);
+    Mockito.when(mockHttpClient.execute(argThat(new HttpGetForService2())))
+        .thenReturn(mockHttpResponseService2);
+    Mockito.when(mockHttpClient.execute(argThat(new HttpGetForService3())))
+        .thenReturn(mockHttpResponseService3);
+
+    //Need 3 times because performop()  does 3 fs operations.
+    Mockito.when(mockHttpEntity.getContent())
+        .thenReturn(new ByteArrayInputStream(
+            validJsonResponse().getBytes(StandardCharsets.UTF_8)))
+        .thenReturn(new ByteArrayInputStream(
+            validJsonResponse().getBytes(StandardCharsets.UTF_8)))
+        .thenReturn(new ByteArrayInputStream(
+            validJsonResponse().getBytes(StandardCharsets.UTF_8)));
+    // finished setting up mocks
+    try {
+      performop(mockHttpClient);
+    }catch (WasbAuthorizationException e){
+      e.printStackTrace();
+      Mockito.verify(mockHttpClient, atLeast(2))
+          .execute(argThat(new HttpGetForService1()));
+      Mockito.verify(mockHttpClient, atLeast(2))
+          .execute(argThat(new HttpGetForService2()));
+      Mockito.verify(mockHttpClient, atLeast(3))
+          .execute(argThat(new HttpGetForService3()));
+      Mockito.verify(mockHttpClient, times(7)).execute(Mockito.<HttpGet>any());
+    }
+  }
+
+  private void setupExpectations() {
+    expectedEx.expect(WasbAuthorizationException.class);
+
+    class MatchesPattern extends TypeSafeMatcher<String> {
+      private String pattern;
+
+      MatchesPattern(String pattern) {
+        this.pattern = pattern;
+      }
+
+      @Override protected boolean matchesSafely(String item) {
+        return item.matches(pattern);
+      }
+
+      @Override public void describeTo(Description description) {
+        description.appendText("matches pattern ").appendValue(pattern);
+      }
+
+      @Override protected void describeMismatchSafely(String item,
+          Description mismatchDescription) {
+        mismatchDescription.appendText("does not match");
+      }
+    }
+
+    expectedEx.expectMessage(new MatchesPattern(
+        "org\\.apache\\.hadoop\\.fs\\.azure\\.WasbRemoteCallException: "
+            + "Encountered error while making remote call to "
+            + "http:\\/\\/localhost1\\/,http:\\/\\/localhost2\\/,http:\\/\\/localhost:8080 retried 6 time\\(s\\)\\."));
+  }
+
+  private void performop(HttpClient mockHttpClient) throws Throwable {
+
+    Path testPath = new Path("/", "test.dat");
+
+    RemoteWasbAuthorizerImpl authorizer = new RemoteWasbAuthorizerImpl();
+    authorizer.init(fs.getConf());
+    WasbRemoteCallHelper mockWasbRemoteCallHelper = new WasbRemoteCallHelper(
+        RetryUtils.getMultipleLinearRandomRetry(new Configuration(),
+            EMPTY_STRING, true,
+            EMPTY_STRING, "1000,3,10000,2"));
+    mockWasbRemoteCallHelper.updateHttpClient(mockHttpClient);
+    authorizer.updateWasbRemoteCallHelper(mockWasbRemoteCallHelper);
+    fs.updateWasbAuthorizer(authorizer);
+
+    fs.create(testPath);
+    ContractTestUtils.assertPathExists(fs, "testPath was not created", testPath);
+    fs.delete(testPath, false);
+  }
+
+  private String validJsonResponse() {
+    return "{"
+        + "\"responseCode\": 0,"
+        + "\"authorizationResult\": true,"
+        + "\"responseMessage\": \"Authorized\""
+        + "}";
+  }
+
+  private String malformedJsonResponse() {
+    return "{"
+        + "\"responseCode\": 0,"
+        + "\"authorizationResult\": true,"
+        + "\"responseMessage\":";
+  }
+
+  private String failureCodeJsonResponse() {
+    return "{"
+        + "\"responseCode\": 1,"
+        + "\"authorizationResult\": false,"
+        + "\"responseMessage\": \"Unauthorized\""
+        + "}";
+  }
+
+  private StatusLine newStatusLine(int statusCode) {
+    return new StatusLine() {
+      @Override
+      public ProtocolVersion getProtocolVersion() {
+        return new ProtocolVersion("HTTP", 1, 1);
+      }
+
+      @Override
+      public int getStatusCode() {
+        return statusCode;
+      }
+
+      @Override
+      public String getReasonPhrase() {
+        return "Reason Phrase";
+      }
+    };
+  }
+
+  private Header newHeader(String name, String value) {
+    return new Header() {
+      @Override
+      public String getName() {
+        return name;
+      }
+
+      @Override
+      public String getValue() {
+        return value;
+      }
+
+      @Override
+      public HeaderElement[] getElements() throws ParseException {
+        return new HeaderElement[0];
+      }
+    };
+  }
+
+  /** Check that a HttpGet request is with given remote host. */
+  private static boolean checkHttpGetMatchHost(HttpGet g, String h) {
+    return g != null && g.getURI().getHost().equals(h);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestWasbUriAndConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestWasbUriAndConfiguration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestWasbUriAndConfiguration.java
new file mode 100644
index 0000000..bee0220
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestWasbUriAndConfiguration.java
@@ -0,0 +1,610 @@
+/**
+ * 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.azure;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
+import static org.junit.Assume.assumeNotNull;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.util.Date;
+import java.util.EnumSet;
+import java.io.File;
+
+import org.apache.hadoop.fs.azure.integration.AzureTestUtils;
+import org.apache.hadoop.security.ProviderUtils;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.AbstractFileSystem;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.CreateOptions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import com.microsoft.azure.storage.blob.CloudBlobContainer;
+import com.microsoft.azure.storage.blob.CloudBlockBlob;
+
+public class ITestWasbUriAndConfiguration extends AbstractWasbTestWithTimeout {
+
+  private static final int FILE_SIZE = 4096;
+  private static final String PATH_DELIMITER = "/";
+
+  protected String accountName;
+  protected String accountKey;
+  protected static Configuration conf = null;
+  private boolean runningInSASMode = false;
+  @Rule
+  public final TemporaryFolder tempDir = new TemporaryFolder();
+
+  private AzureBlobStorageTestAccount testAccount;
+
+  @After
+  public void tearDown() throws Exception {
+    testAccount = AzureTestUtils.cleanupTestAccount(testAccount);
+  }
+
+  @Before
+  public void setMode() {
+    runningInSASMode = AzureBlobStorageTestAccount.createTestConfiguration().
+        getBoolean(AzureNativeFileSystemStore.KEY_USE_SECURE_MODE, false);
+  }
+
+  private boolean validateIOStreams(Path filePath) throws IOException {
+    // Capture the file system from the test account.
+    FileSystem fs = testAccount.getFileSystem();
+    return validateIOStreams(fs, filePath);
+  }
+
+  private boolean validateIOStreams(FileSystem fs, Path filePath)
+      throws IOException {
+
+    // Create and write a file
+    OutputStream outputStream = fs.create(filePath);
+    outputStream.write(new byte[FILE_SIZE]);
+    outputStream.close();
+
+    // Return true if the the count is equivalent to the file size.
+    return (FILE_SIZE == readInputStream(fs, filePath));
+  }
+
+  private int readInputStream(Path filePath) throws IOException {
+    // Capture the file system from the test account.
+    FileSystem fs = testAccount.getFileSystem();
+    return readInputStream(fs, filePath);
+  }
+
+  private int readInputStream(FileSystem fs, Path filePath) throws IOException {
+    // Read the file
+    InputStream inputStream = fs.open(filePath);
+    int count = 0;
+    while (inputStream.read() >= 0) {
+      count++;
+    }
+    inputStream.close();
+
+    // Return true if the the count is equivalent to the file size.
+    return count;
+  }
+
+  // Positive tests to exercise making a connection with to Azure account using
+  // account key.
+  @Test
+  public void testConnectUsingKey() throws Exception {
+
+    testAccount = AzureBlobStorageTestAccount.create();
+    assumeNotNull(testAccount);
+
+    // Validate input and output on the connection.
+    assertTrue(validateIOStreams(new Path("/wasb_scheme")));
+  }
+
+  @Test
+  public void testConnectUsingSAS() throws Exception {
+
+    Assume.assumeFalse(runningInSASMode);
+    // Create the test account with SAS credentials.
+    testAccount = AzureBlobStorageTestAccount.create("",
+        EnumSet.of(CreateOptions.UseSas, CreateOptions.CreateContainer));
+    assumeNotNull(testAccount);
+    // Validate input and output on the connection.
+    // NOTE: As of 4/15/2013, Azure Storage has a deficiency that prevents the
+    // full scenario from working (CopyFromBlob doesn't work with SAS), so
+    // just do a minor check until that is corrected.
+    assertFalse(testAccount.getFileSystem().exists(new Path("/IDontExist")));
+    //assertTrue(validateIOStreams(new Path("/sastest.txt")));
+  }
+
+  @Test
+  public void testConnectUsingSASReadonly() throws Exception {
+
+    Assume.assumeFalse(runningInSASMode);
+    // Create the test account with SAS credentials.
+    testAccount = AzureBlobStorageTestAccount.create("", EnumSet.of(
+        CreateOptions.UseSas, CreateOptions.CreateContainer,
+        CreateOptions.Readonly));
+    assumeNotNull(testAccount);
+
+    // Create a blob in there
+    final String blobKey = "blobForReadonly";
+    CloudBlobContainer container = testAccount.getRealContainer();
+    CloudBlockBlob blob = container.getBlockBlobReference(blobKey);
+    ByteArrayInputStream inputStream = new ByteArrayInputStream(new byte[] { 1,
+        2, 3 });
+    blob.upload(inputStream, 3);
+    inputStream.close();
+
+    // Make sure we can read it from the file system
+    Path filePath = new Path("/" + blobKey);
+    FileSystem fs = testAccount.getFileSystem();
+    assertTrue(fs.exists(filePath));
+    byte[] obtained = new byte[3];
+    DataInputStream obtainedInputStream = fs.open(filePath);
+    obtainedInputStream.readFully(obtained);
+    obtainedInputStream.close();
+    assertEquals(3, obtained[2]);
+  }
+
+  @Test
+  public void testConnectUsingAnonymous() throws Exception {
+
+    // Create test account with anonymous credentials
+    testAccount = AzureBlobStorageTestAccount.createAnonymous("testWasb.txt",
+        FILE_SIZE);
+    assumeNotNull(testAccount);
+
+    // Read the file from the public folder using anonymous credentials.
+    assertEquals(FILE_SIZE, readInputStream(new Path("/testWasb.txt")));
+  }
+
+  @Test
+  public void testConnectToEmulator() throws Exception {
+    testAccount = AzureBlobStorageTestAccount.createForEmulator();
+    assumeNotNull(testAccount);
+    assertTrue(validateIOStreams(new Path("/testFile")));
+  }
+
+  /**
+   * Tests that we can connect to fully qualified accounts outside of
+   * blob.core.windows.net
+   */
+  @Test
+  public void testConnectToFullyQualifiedAccountMock() throws Exception {
+    Configuration conf = new Configuration();
+    AzureBlobStorageTestAccount.setMockAccountKey(conf,
+        "mockAccount.mock.authority.net");
+    AzureNativeFileSystemStore store = new AzureNativeFileSystemStore();
+    MockStorageInterface mockStorage = new MockStorageInterface();
+    store.setAzureStorageInteractionLayer(mockStorage);
+    NativeAzureFileSystem fs = new NativeAzureFileSystem(store);
+    fs.initialize(
+        new URI("wasb://mockContainer@mockAccount.mock.authority.net"), conf);
+    fs.createNewFile(new Path("/x"));
+    assertTrue(mockStorage.getBackingStore().exists(
+        "http://mockAccount.mock.authority.net/mockContainer/x"));
+    fs.close();
+  }
+
+  public void testConnectToRoot() throws Exception {
+
+    // Set up blob names.
+    final String blobPrefix = String.format("wasbtests-%s-%tQ-blob",
+        System.getProperty("user.name"), new Date());
+    final String inblobName = blobPrefix + "_In" + ".txt";
+    final String outblobName = blobPrefix + "_Out" + ".txt";
+
+    // Create test account with default root access.
+    testAccount = AzureBlobStorageTestAccount.createRoot(inblobName, FILE_SIZE);
+    assumeNotNull(testAccount);
+
+    // Read the file from the default container.
+    assertEquals(FILE_SIZE, readInputStream(new Path(PATH_DELIMITER
+        + inblobName)));
+
+    try {
+      // Capture file system.
+      FileSystem fs = testAccount.getFileSystem();
+
+      // Create output path and open an output stream to the root folder.
+      Path outputPath = new Path(PATH_DELIMITER + outblobName);
+      OutputStream outputStream = fs.create(outputPath);
+      fail("Expected an AzureException when writing to root folder.");
+      outputStream.write(new byte[FILE_SIZE]);
+      outputStream.close();
+    } catch (AzureException e) {
+      assertTrue(true);
+    } catch (Exception e) {
+      String errMsg = String.format(
+          "Expected AzureException but got %s instead.", e);
+      assertTrue(errMsg, false);
+    }
+  }
+
+  // Positive tests to exercise throttling I/O path. Connections are made to an
+  // Azure account using account key.
+  //
+  public void testConnectWithThrottling() throws Exception {
+
+    testAccount = AzureBlobStorageTestAccount.createThrottled();
+
+    // Validate input and output on the connection.
+    assertTrue(validateIOStreams(new Path("/wasb_scheme")));
+  }
+
+  /**
+   * Creates a file and writes a single byte with the given value in it.
+   */
+  private static void writeSingleByte(FileSystem fs, Path testFile, int toWrite)
+      throws Exception {
+    OutputStream outputStream = fs.create(testFile);
+    outputStream.write(toWrite);
+    outputStream.close();
+  }
+
+  /**
+   * Reads the file given and makes sure that it's a single-byte file with the
+   * given value in it.
+   */
+  private static void assertSingleByteValue(FileSystem fs, Path testFile,
+      int expectedValue) throws Exception {
+    InputStream inputStream = fs.open(testFile);
+    int byteRead = inputStream.read();
+    assertTrue("File unexpectedly empty: " + testFile, byteRead >= 0);
+    assertTrue("File has more than a single byte: " + testFile,
+        inputStream.read() < 0);
+    inputStream.close();
+    assertEquals("Unxpected content in: " + testFile, expectedValue, byteRead);
+  }
+
+  @Test
+  public void testMultipleContainers() throws Exception {
+    AzureBlobStorageTestAccount firstAccount = AzureBlobStorageTestAccount
+        .create("first"), secondAccount = AzureBlobStorageTestAccount
+        .create("second");
+    assumeNotNull(firstAccount);
+    assumeNotNull(secondAccount);
+    try {
+      FileSystem firstFs = firstAccount.getFileSystem(),
+          secondFs = secondAccount.getFileSystem();
+      Path testFile = new Path("/testWasb");
+      assertTrue(validateIOStreams(firstFs, testFile));
+      assertTrue(validateIOStreams(secondFs, testFile));
+      // Make sure that we're really dealing with two file systems here.
+      writeSingleByte(firstFs, testFile, 5);
+      writeSingleByte(secondFs, testFile, 7);
+      assertSingleByteValue(firstFs, testFile, 5);
+      assertSingleByteValue(secondFs, testFile, 7);
+    } finally {
+      firstAccount.cleanup();
+      secondAccount.cleanup();
+    }
+  }
+
+  @Test
+  public void testDefaultKeyProvider() throws Exception {
+    Configuration conf = new Configuration();
+    String account = "testacct";
+    String key = "testkey";
+
+    conf.set(SimpleKeyProvider.KEY_ACCOUNT_KEY_PREFIX + account, key);
+
+    String result = AzureNativeFileSystemStore.getAccountKeyFromConfiguration(
+        account, conf);
+    assertEquals(key, result);
+  }
+
+  @Test
+  public void testCredsFromCredentialProvider() throws Exception {
+
+    Assume.assumeFalse(runningInSASMode);
+    String account = "testacct";
+    String key = "testkey";
+    // set up conf to have a cred provider
+    final Configuration conf = new Configuration();
+    final File file = tempDir.newFile("test.jks");
+    final URI jks = ProviderUtils.nestURIForLocalJavaKeyStoreProvider(
+        file.toURI());
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
+        jks.toString());
+
+    provisionAccountKey(conf, account, key);
+
+    // also add to configuration as clear text that should be overridden
+    conf.set(SimpleKeyProvider.KEY_ACCOUNT_KEY_PREFIX + account,
+        key + "cleartext");
+
+    String result = AzureNativeFileSystemStore.getAccountKeyFromConfiguration(
+        account, conf);
+    // result should contain the credential provider key not the config key
+    assertEquals("AccountKey incorrect.", key, result);
+  }
+
+  void provisionAccountKey(
+      final Configuration conf, String account, String key) throws Exception {
+    // add our creds to the provider
+    final CredentialProvider provider =
+        CredentialProviderFactory.getProviders(conf).get(0);
+    provider.createCredentialEntry(
+        SimpleKeyProvider.KEY_ACCOUNT_KEY_PREFIX + account, key.toCharArray());
+    provider.flush();
+  }
+
+  @Test
+  public void testValidKeyProvider() throws Exception {
+    Configuration conf = new Configuration();
+    String account = "testacct";
+    String key = "testkey";
+
+    conf.set(SimpleKeyProvider.KEY_ACCOUNT_KEY_PREFIX + account, key);
+    conf.setClass("fs.azure.account.keyprovider." + account,
+        SimpleKeyProvider.class, KeyProvider.class);
+    String result = AzureNativeFileSystemStore.getAccountKeyFromConfiguration(
+        account, conf);
+    assertEquals(key, result);
+  }
+
+  @Test
+  public void testInvalidKeyProviderNonexistantClass() throws Exception {
+    Configuration conf = new Configuration();
+    String account = "testacct";
+
+    conf.set("fs.azure.account.keyprovider." + account,
+        "org.apache.Nonexistant.Class");
+    try {
+      AzureNativeFileSystemStore.getAccountKeyFromConfiguration(account, conf);
+      Assert.fail("Nonexistant key provider class should have thrown a "
+          + "KeyProviderException");
+    } catch (KeyProviderException e) {
+    }
+  }
+
+  @Test
+  public void testInvalidKeyProviderWrongClass() throws Exception {
+    Configuration conf = new Configuration();
+    String account = "testacct";
+
+    conf.set("fs.azure.account.keyprovider." + account, "java.lang.String");
+    try {
+      AzureNativeFileSystemStore.getAccountKeyFromConfiguration(account, conf);
+      Assert.fail("Key provider class that doesn't implement KeyProvider "
+          + "should have thrown a KeyProviderException");
+    } catch (KeyProviderException e) {
+    }
+  }
+
+  /**
+   * Tests the cases when the URI is specified with no authority, i.e.
+   * wasb:///path/to/file.
+   */
+  @Test
+  public void testNoUriAuthority() throws Exception {
+    // For any combination of default FS being asv(s)/wasb(s)://c@a/ and
+    // the actual URI being asv(s)/wasb(s):///, it should work.
+
+    String[] wasbAliases = new String[] { "wasb", "wasbs" };
+    for (String defaultScheme : wasbAliases) {
+      for (String wantedScheme : wasbAliases) {
+        testAccount = AzureBlobStorageTestAccount.createMock();
+        Configuration conf = testAccount.getFileSystem().getConf();
+        String authority = testAccount.getFileSystem().getUri().getAuthority();
+        URI defaultUri = new URI(defaultScheme, authority, null, null, null);
+        conf.set(FS_DEFAULT_NAME_KEY, defaultUri.toString());
+        // Add references to file system implementations for wasb and wasbs.
+        conf.addResource("azure-test.xml");
+        URI wantedUri = new URI(wantedScheme + ":///random/path");
+        NativeAzureFileSystem obtained = (NativeAzureFileSystem) FileSystem
+            .get(wantedUri, conf);
+        assertNotNull(obtained);
+        assertEquals(new URI(wantedScheme, authority, null, null, null),
+            obtained.getUri());
+        // Make sure makeQualified works as expected
+        Path qualified = obtained.makeQualified(new Path(wantedUri));
+        assertEquals(new URI(wantedScheme, authority, wantedUri.getPath(),
+            null, null), qualified.toUri());
+        // Cleanup for the next iteration to not cache anything in FS
+        testAccount.cleanup();
+        FileSystem.closeAll();
+      }
+    }
+    // If the default FS is not a WASB FS, then specifying a URI without
+    // authority for the Azure file system should throw.
+    testAccount = AzureBlobStorageTestAccount.createMock();
+    Configuration conf = testAccount.getFileSystem().getConf();
+    conf.set(FS_DEFAULT_NAME_KEY, "file:///");
+    try {
+      FileSystem.get(new URI("wasb:///random/path"), conf);
+      fail("Should've thrown.");
+    } catch (IllegalArgumentException e) {
+    }
+  }
+
+  @Test
+  public void testWasbAsDefaultFileSystemHasNoPort() throws Exception {
+    try {
+      testAccount = AzureBlobStorageTestAccount.createMock();
+      Configuration conf = testAccount.getFileSystem().getConf();
+      String authority = testAccount.getFileSystem().getUri().getAuthority();
+      URI defaultUri = new URI("wasb", authority, null, null, null);
+      conf.set(FS_DEFAULT_NAME_KEY, defaultUri.toString());
+      conf.addResource("azure-test.xml");
+
+      FileSystem fs = FileSystem.get(conf);
+      assertTrue(fs instanceof NativeAzureFileSystem);
+      assertEquals(-1, fs.getUri().getPort());
+
+      AbstractFileSystem afs = FileContext.getFileContext(conf)
+          .getDefaultFileSystem();
+      assertTrue(afs instanceof Wasb);
+      assertEquals(-1, afs.getUri().getPort());
+    } finally {
+      testAccount.cleanup();
+      FileSystem.closeAll();
+    }
+  }
+
+   /**
+   * Tests the cases when the scheme specified is 'wasbs'.
+   */
+  @Test
+  public void testAbstractFileSystemImplementationForWasbsScheme() throws Exception {
+    try {
+      testAccount = AzureBlobStorageTestAccount.createMock();
+      Configuration conf = testAccount.getFileSystem().getConf();
+      String authority = testAccount.getFileSystem().getUri().getAuthority();
+      URI defaultUri = new URI("wasbs", authority, null, null, null);
+      conf.set(FS_DEFAULT_NAME_KEY, defaultUri.toString());
+      conf.set("fs.AbstractFileSystem.wasbs.impl", "org.apache.hadoop.fs.azure.Wasbs");
+      conf.addResource("azure-test.xml");
+
+      FileSystem fs = FileSystem.get(conf);
+      assertTrue(fs instanceof NativeAzureFileSystem);
+      assertEquals("wasbs", fs.getScheme());
+
+      AbstractFileSystem afs = FileContext.getFileContext(conf)
+          .getDefaultFileSystem();
+      assertTrue(afs instanceof Wasbs);
+      assertEquals(-1, afs.getUri().getPort());
+      assertEquals("wasbs", afs.getUri().getScheme());
+    } finally {
+      testAccount.cleanup();
+      FileSystem.closeAll();
+    }
+  }
+
+  @Test
+  public void testNoAbstractFileSystemImplementationSpecifiedForWasbsScheme() throws Exception {
+    try {
+      testAccount = AzureBlobStorageTestAccount.createMock();
+      Configuration conf = testAccount.getFileSystem().getConf();
+      String authority = testAccount.getFileSystem().getUri().getAuthority();
+      URI defaultUri = new URI("wasbs", authority, null, null, null);
+      conf.set(FS_DEFAULT_NAME_KEY, defaultUri.toString());
+
+      FileSystem fs = FileSystem.get(conf);
+      assertTrue(fs instanceof NativeAzureFileSystem);
+      assertEquals("wasbs", fs.getScheme());
+
+      // should throw if 'fs.AbstractFileSystem.wasbs.impl'' is not specified
+      try{
+        FileContext.getFileContext(conf).getDefaultFileSystem();
+        fail("Should've thrown.");
+      }catch(UnsupportedFileSystemException e){
+      }
+
+    } finally {
+      testAccount.cleanup();
+      FileSystem.closeAll();
+    }
+  }
+
+  @Test
+  public void testCredentialProviderPathExclusions() throws Exception {
+    String providerPath =
+        "user:///,jceks://wasb/user/hrt_qa/sqoopdbpasswd.jceks," +
+        "jceks://hdfs@nn1.example.com/my/path/test.jceks";
+    Configuration config = new Configuration();
+    config.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
+        providerPath);
+    String newPath = "user:///,jceks://hdfs@nn1.example.com/my/path/test.jceks";
+
+    excludeAndTestExpectations(config, newPath);
+  }
+
+  @Test
+  public void testExcludeAllProviderTypesFromConfig() throws Exception {
+    String providerPath =
+        "jceks://wasb/tmp/test.jceks," +
+        "jceks://wasb@/my/path/test.jceks";
+    Configuration config = new Configuration();
+    config.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
+        providerPath);
+    String newPath = null;
+
+    excludeAndTestExpectations(config, newPath);
+  }
+
+  void excludeAndTestExpectations(Configuration config, String newPath)
+    throws Exception {
+    Configuration conf = ProviderUtils.excludeIncompatibleCredentialProviders(
+        config, NativeAzureFileSystem.class);
+    String effectivePath = conf.get(
+        CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, null);
+    assertEquals(newPath, effectivePath);
+  }
+
+  @Test
+  public void testUserAgentConfig() throws Exception {
+    // Set the user agent
+    try {
+      testAccount = AzureBlobStorageTestAccount.createMock();
+      Configuration conf = testAccount.getFileSystem().getConf();
+      String authority = testAccount.getFileSystem().getUri().getAuthority();
+      URI defaultUri = new URI("wasbs", authority, null, null, null);
+      conf.set(FS_DEFAULT_NAME_KEY, defaultUri.toString());
+      conf.set("fs.AbstractFileSystem.wasbs.impl", "org.apache.hadoop.fs.azure.Wasbs");
+
+      conf.set(AzureNativeFileSystemStore.USER_AGENT_ID_KEY, "TestClient");
+
+      FileSystem fs = FileSystem.get(conf);
+      AbstractFileSystem afs = FileContext.getFileContext(conf).getDefaultFileSystem();
+
+      assertTrue(afs instanceof Wasbs);
+      assertEquals(-1, afs.getUri().getPort());
+      assertEquals("wasbs", afs.getUri().getScheme());
+
+    } finally {
+      testAccount.cleanup();
+      FileSystem.closeAll();
+    }
+
+    // Unset the user agent
+    try {
+      testAccount = AzureBlobStorageTestAccount.createMock();
+      Configuration conf = testAccount.getFileSystem().getConf();
+      String authority = testAccount.getFileSystem().getUri().getAuthority();
+      URI defaultUri = new URI("wasbs", authority, null, null, null);
+      conf.set(FS_DEFAULT_NAME_KEY, defaultUri.toString());
+      conf.set("fs.AbstractFileSystem.wasbs.impl", "org.apache.hadoop.fs.azure.Wasbs");
+
+      conf.unset(AzureNativeFileSystemStore.USER_AGENT_ID_KEY);
+
+      FileSystem fs = FileSystem.get(conf);
+      AbstractFileSystem afs = FileContext.getFileContext(conf).getDefaultFileSystem();
+      assertTrue(afs instanceof Wasbs);
+      assertEquals(-1, afs.getUri().getPort());
+      assertEquals("wasbs", afs.getUri().getScheme());
+
+    } finally {
+      testAccount.cleanup();
+      FileSystem.closeAll();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockWasbAuthorizerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockWasbAuthorizerImpl.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockWasbAuthorizerImpl.java
index 9fbab49..7354499 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockWasbAuthorizerImpl.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockWasbAuthorizerImpl.java
@@ -38,11 +38,12 @@ public class MockWasbAuthorizerImpl implements WasbAuthorizerInterface {
   private boolean performOwnerMatch;
   private CachingAuthorizer<CachedAuthorizerEntry, Boolean> cache;
 
- // The full qualified URL to the root directory
+  // The full qualified URL to the root directory
   private String qualifiedPrefixUrl;
 
   public MockWasbAuthorizerImpl(NativeAzureFileSystem fs) {
-    qualifiedPrefixUrl = new Path("/").makeQualified(fs.getUri(), fs.getWorkingDirectory())
+    qualifiedPrefixUrl = new Path("/").makeQualified(fs.getUri(),
+        fs.getWorkingDirectory())
         .toString().replaceAll("/$", "");
     cache = new CachingAuthorizer<>(TimeUnit.MINUTES.convert(5L, TimeUnit.MINUTES), "AUTHORIZATION");
   }
@@ -64,19 +65,23 @@ public class MockWasbAuthorizerImpl implements WasbAuthorizerInterface {
 
   public void addAuthRule(String wasbAbsolutePath,
       String accessType, boolean access) {
-        wasbAbsolutePath = qualifiedPrefixUrl + wasbAbsolutePath;
-        AuthorizationComponent component = wasbAbsolutePath.endsWith("*")
-        ? new AuthorizationComponent("^" + wasbAbsolutePath.replace("*", ".*"), accessType)
+    wasbAbsolutePath = qualifiedPrefixUrl + wasbAbsolutePath;
+    AuthorizationComponent component = wasbAbsolutePath.endsWith("*")
+        ? new AuthorizationComponent("^" + wasbAbsolutePath.replace("*", ".*"),
+        accessType)
         : new AuthorizationComponent(wasbAbsolutePath, accessType);
 
     this.authRules.put(component, access);
   }
 
   @Override
-  public boolean authorize(String wasbAbsolutePath, String accessType, String owner)
+  public boolean authorize(String wasbAbsolutePath,
+      String accessType,
+      String owner)
       throws WasbAuthorizationException {
 
-    if (wasbAbsolutePath.endsWith(NativeAzureFileSystem.FolderRenamePending.SUFFIX)) {
+    if (wasbAbsolutePath.endsWith(
+        NativeAzureFileSystem.FolderRenamePending.SUFFIX)) {
       return true;
     }
 
@@ -108,20 +113,23 @@ public class MockWasbAuthorizerImpl implements WasbAuthorizerInterface {
     // In case of root("/"), owner match does not happen because owner is returned as empty string.
     // we try to force owner match just for purpose of tests to make sure all operations work seemlessly with owner.
     if (this.performOwnerMatch
-      && StringUtils.equalsIgnoreCase(wasbAbsolutePath, qualifiedPrefixUrl + "/")) {
+        && StringUtils.equalsIgnoreCase(wasbAbsolutePath,
+        qualifiedPrefixUrl + "/")) {
       owner = currentUserShortName;
     }
 
     boolean shouldEvaluateOwnerAccess = owner != null && !owner.isEmpty()
-      && this.performOwnerMatch;
+        && this.performOwnerMatch;
 
-    boolean isOwnerMatch =  StringUtils.equalsIgnoreCase(currentUserShortName, owner);
+    boolean isOwnerMatch = StringUtils.equalsIgnoreCase(currentUserShortName,
+        owner);
 
     AuthorizationComponent component =
         new AuthorizationComponent(wasbAbsolutePath, accessType);
 
     if (authRules.containsKey(component)) {
-      return shouldEvaluateOwnerAccess ? isOwnerMatch && authRules.get(component) : authRules.get(component);
+      return shouldEvaluateOwnerAccess ? isOwnerMatch && authRules.get(
+          component) : authRules.get(component);
     } else {
       // Regex-pattern match if we don't have a straight match
       for (Map.Entry<AuthorizationComponent, Boolean> entry : authRules.entrySet()) {
@@ -129,8 +137,11 @@ public class MockWasbAuthorizerImpl implements WasbAuthorizerInterface {
         String keyPath = key.getWasbAbsolutePath();
         String keyAccess = key.getAccessType();
 
-        if (keyPath.endsWith("*") && Pattern.matches(keyPath, wasbAbsolutePath) && keyAccess.equals(accessType)) {
-          return shouldEvaluateOwnerAccess ? isOwnerMatch && entry.getValue() : entry.getValue();
+        if (keyPath.endsWith("*") && Pattern.matches(keyPath, wasbAbsolutePath)
+            && keyAccess.equals(accessType)) {
+          return shouldEvaluateOwnerAccess
+              ? isOwnerMatch && entry.getValue()
+              : entry.getValue();
         }
       }
       return false;
@@ -141,47 +152,47 @@ public class MockWasbAuthorizerImpl implements WasbAuthorizerInterface {
     authRules.clear();
     cache.clear();
   }
-}
 
-class AuthorizationComponent {
+  private static class AuthorizationComponent {
 
-  private String wasbAbsolutePath;
-  private String accessType;
+    private final String wasbAbsolutePath;
+    private final String accessType;
 
-  public AuthorizationComponent(String wasbAbsolutePath,
-      String accessType) {
-    this.wasbAbsolutePath = wasbAbsolutePath;
-    this.accessType = accessType;
-  }
+    AuthorizationComponent(String wasbAbsolutePath,
+        String accessType) {
+      this.wasbAbsolutePath = wasbAbsolutePath;
+      this.accessType = accessType;
+    }
 
-  @Override
-  public int hashCode() {
-    return this.wasbAbsolutePath.hashCode() ^ this.accessType.hashCode();
-  }
+    @Override
+    public int hashCode() {
+      return this.wasbAbsolutePath.hashCode() ^ this.accessType.hashCode();
+    }
 
-  @Override
-  public boolean equals(Object obj) {
+    @Override
+    public boolean equals(Object obj) {
 
-    if (obj == this) {
-      return true;
-    }
+      if (obj == this) {
+        return true;
+      }
 
-    if (obj == null
-        || !(obj instanceof AuthorizationComponent)) {
-      return false;
-    }
+      if (obj == null
+          || !(obj instanceof AuthorizationComponent)) {
+        return false;
+      }
 
-    return ((AuthorizationComponent)obj).
-              getWasbAbsolutePath().equals(this.wasbAbsolutePath)
-            && ((AuthorizationComponent)obj).
-              getAccessType().equals(this.accessType);
-  }
+      return ((AuthorizationComponent) obj).
+          getWasbAbsolutePath().equals(this.wasbAbsolutePath)
+          && ((AuthorizationComponent) obj).
+          getAccessType().equals(this.accessType);
+    }
 
-  public String getWasbAbsolutePath() {
-    return this.wasbAbsolutePath;
-  }
+    public String getWasbAbsolutePath() {
+      return this.wasbAbsolutePath;
+    }
 
-  public String getAccessType() {
-    return accessType;
+    public String getAccessType() {
+      return accessType;
+    }
   }
-}
\ No newline at end of file
+}


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


[31/50] [abbrv] hadoop git commit: YARN-7162. Remove XML excludes file format (rkanter)

Posted by ae...@apache.org.
YARN-7162. Remove XML excludes file format (rkanter)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3a8d57a0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3a8d57a0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3a8d57a0

Branch: refs/heads/HDFS-7240
Commit: 3a8d57a0a2e047b34be82f602a2b6cf5593d2125
Parents: de197fc
Author: Robert Kanter <rk...@apache.org>
Authored: Fri Sep 15 12:00:26 2017 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Fri Sep 15 12:00:26 2017 -0700

----------------------------------------------------------------------
 .../hadoop-mapreduce-client-core/pom.xml        |  4 ++
 .../hadoop/mapreduce/JobResourceUploader.java   | 17 ++++++++
 .../apache/hadoop/mapreduce/MRJobConfig.java    |  5 +++
 .../src/main/resources/mapred-default.xml       |  9 ++++
 .../mapreduce/TestJobResourceUploader.java      | 46 ++++++++++++++++++++
 5 files changed, 81 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a8d57a0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
index c34f7bd..ce5fdc8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
@@ -44,6 +44,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-hdfs</artifactId>
       <scope>test</scope>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a8d57a0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
index f1cad57..d9bf988 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
@@ -36,6 +36,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.mapreduce.filecache.ClientDistributedCacheManager;
 import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 
@@ -94,6 +96,11 @@ class JobResourceUploader {
         new FsPermission(JobSubmissionFiles.JOB_DIR_PERMISSION);
     mkdirs(jtFs, submitJobDir, mapredSysPerms);
 
+    if (!conf.getBoolean(MRJobConfig.MR_AM_STAGING_DIR_ERASURECODING_ENABLED,
+        MRJobConfig.DEFAULT_MR_AM_STAGING_ERASURECODING_ENABLED)) {
+      disableErasureCodingForPath(jtFs, submitJobDir);
+    }
+
     Collection<String> files = conf.getStringCollection("tmpfiles");
     Collection<String> libjars = conf.getStringCollection("tmpjars");
     Collection<String> archives = conf.getStringCollection("tmparchives");
@@ -575,4 +582,14 @@ class JobResourceUploader {
     }
     return finalPath;
   }
+
+  private void disableErasureCodingForPath(FileSystem fs, Path path)
+      throws IOException {
+    if (jtFs instanceof DistributedFileSystem) {
+      LOG.info("Disabling Erasure Coding for path: " + path);
+      DistributedFileSystem dfs = (DistributedFileSystem) jtFs;
+      dfs.setErasureCodingPolicy(path,
+          SystemErasureCodingPolicies.getReplicationPolicy().getName());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a8d57a0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index 2023ba3..86abb42 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -1037,4 +1037,9 @@ public interface MRJobConfig {
   String FINISH_JOB_WHEN_REDUCERS_DONE =
       "mapreduce.job.finish-when-all-reducers-done";
   boolean DEFAULT_FINISH_JOB_WHEN_REDUCERS_DONE = true;
+
+  String MR_AM_STAGING_DIR_ERASURECODING_ENABLED =
+      MR_AM_STAGING_DIR + "erasurecoding.enabled";
+
+  boolean DEFAULT_MR_AM_STAGING_ERASURECODING_ENABLED = false;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a8d57a0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index ee9b906..6b6faf2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -1261,6 +1261,15 @@
 </property>
 
 <property>
+  <name>yarn.app.mapreduce.am.staging-dir.erasurecoding.enabled</name>
+  <value>false</value>
+  <description>Whether Erasure Coding should be enabled for
+  files that are copied to the MR staging area. This is a job-level
+  setting.
+  </description>
+</property>
+
+<property>
   <name>mapreduce.am.max-attempts</name>
   <value>2</value>
   <description>The maximum number of application attempts. It is a

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a8d57a0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
index 20b7b7d..d0d7a34 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
@@ -20,6 +20,11 @@ package org.apache.hadoop.mapreduce;
 
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.times;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
 
 import java.io.IOException;
 import java.net.URI;
@@ -36,9 +41,12 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.mapred.JobConf;
 import org.junit.Assert;
 import org.junit.Test;
+import org.mockito.verification.VerificationMode;
 
 /**
  * A class for unit testing JobResourceUploader.
@@ -357,6 +365,40 @@ public class TestJobResourceUploader {
         expectedArchivesWithFrags, expectedJobJar);
   }
 
+  @Test
+  public void testErasureCodingDefault() throws IOException {
+    testErasureCodingSetting(true);
+  }
+
+  @Test
+  public void testErasureCodingDisabled() throws IOException {
+    testErasureCodingSetting(false);
+  }
+
+  private void testErasureCodingSetting(boolean defaultBehavior)
+      throws IOException {
+    JobConf jConf = new JobConf();
+    // don't set to false if EC remains disabled to check default setting
+    if (!defaultBehavior) {
+      jConf.setBoolean(MRJobConfig.MR_AM_STAGING_DIR_ERASURECODING_ENABLED,
+          true);
+    }
+
+    DistributedFileSystem fs = mock(DistributedFileSystem.class);
+    Path path = new Path("/");
+    when(fs.makeQualified(any(Path.class))).thenReturn(path);
+    JobResourceUploader uploader = new StubedUploader(fs, true);
+    Job job = Job.getInstance(jConf);
+
+    uploader.uploadResources(job, new Path("/test"));
+
+    String replicationPolicyName = SystemErasureCodingPolicies
+        .getReplicationPolicy().getName();
+    VerificationMode mode = defaultBehavior ? times(1) : never();
+    verify(fs, mode).setErasureCodingPolicy(eq(path),
+        eq(replicationPolicyName));
+  }
+
   private void runTmpResourcePathTest(JobResourceUploader uploader,
       ResourceConf rConf, JobConf jConf, String[] expectedFiles,
       String[] expectedArchives, String expectedJobJar) throws IOException {
@@ -698,6 +740,10 @@ public class TestJobResourceUploader {
       super(FileSystem.getLocal(conf), useWildcard);
     }
 
+    StubedUploader(FileSystem fs, boolean useWildcard) throws IOException {
+      super(fs, useWildcard);
+    }
+
     @Override
     FileStatus getFileStatus(Map<URI, FileStatus> statCache, Configuration job,
         Path p) throws IOException {


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


[40/50] [abbrv] hadoop git commit: HDFS-12472. Add JUNIT timeout to TestBlockStatsMXBean. Contributed by Bharat Viswanadham.

Posted by ae...@apache.org.
HDFS-12472. Add JUNIT timeout to TestBlockStatsMXBean. Contributed by Bharat Viswanadham.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8d7cc22a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8d7cc22a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8d7cc22a

Branch: refs/heads/HDFS-7240
Commit: 8d7cc22ac286302960c7939bc53574cbfeab1846
Parents: 7618fa9
Author: Arpit Agarwal <ar...@apache.org>
Authored: Sat Sep 16 10:09:27 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Sat Sep 16 10:09:27 2017 -0700

----------------------------------------------------------------------
 .../hdfs/server/blockmanagement/TestBlockStatsMXBean.java       | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d7cc22a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
index bcf38d6..64364cb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
@@ -41,8 +41,10 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.eclipse.jetty.util.ajax.JSON;
+import org.junit.rules.Timeout;
 
 /**
  * Class for testing {@link BlockStatsMXBean} implementation
@@ -51,6 +53,9 @@ public class TestBlockStatsMXBean {
 
   private MiniDFSCluster cluster;
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Before
   public void setup() throws IOException {
     HdfsConfiguration conf = new HdfsConfiguration();


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


[13/50] [abbrv] hadoop git commit: HDFS-12378. TestClientProtocolForPipelineRecovery#testZeroByteBlockRecovery fails on trunk. (Lei (Eddy) Xu)

Posted by ae...@apache.org.
HDFS-12378.  TestClientProtocolForPipelineRecovery#testZeroByteBlockRecovery fails on trunk. (Lei (Eddy) Xu)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/61cee3a0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/61cee3a0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/61cee3a0

Branch: refs/heads/HDFS-7240
Commit: 61cee3a0b9a8ea2e4f6257c17c2d90c7c930cc34
Parents: 390c2b5
Author: Lei Xu <le...@apache.org>
Authored: Thu Sep 14 17:02:48 2017 -0700
Committer: Lei Xu <le...@apache.org>
Committed: Thu Sep 14 17:03:25 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/server/datanode/DataNode.java    | 12 ++++++++++--
 1 file changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/61cee3a0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 837ac07..042a627 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -3000,8 +3000,16 @@ public class DataNode extends ReconfigurableBase
     b.setNumBytes(visible);
 
     if (targets.length > 0) {
-      new Daemon(new DataTransfer(targets, targetStorageTypes,
-          targetStorageIds, b, stage, client)).start();
+      Daemon daemon = new Daemon(threadGroup,
+          new DataTransfer(targets, targetStorageTypes, targetStorageIds, b,
+              stage, client));
+      daemon.start();
+      try {
+        daemon.join();
+      } catch (InterruptedException e) {
+        throw new IOException(
+            "Pipeline recovery for " + b + " is interrupted.", e);
+      }
     }
   }
 


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


[46/50] [abbrv] hadoop git commit: MAPREDUCE-6954. Disable erasure coding for files that are uploaded to the MR staging area (pbacsko via rkanter)

Posted by ae...@apache.org.
MAPREDUCE-6954. Disable erasure coding for files that are uploaded to the MR staging area (pbacsko via rkanter)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0adc0471
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0adc0471
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0adc0471

Branch: refs/heads/HDFS-7240
Commit: 0adc0471d0c06f66a31060f270dcb50a7b4ffafa
Parents: 5f49668
Author: Robert Kanter <rk...@apache.org>
Authored: Mon Sep 18 10:40:06 2017 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Mon Sep 18 10:40:06 2017 -0700

----------------------------------------------------------------------
 .../hadoop-mapreduce-client-core/pom.xml        |  4 ++
 .../hadoop/mapreduce/JobResourceUploader.java   | 17 ++++++++
 .../apache/hadoop/mapreduce/MRJobConfig.java    |  5 +++
 .../src/main/resources/mapred-default.xml       |  9 ++++
 .../mapreduce/TestJobResourceUploader.java      | 46 ++++++++++++++++++++
 5 files changed, 81 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0adc0471/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
index c34f7bd..ce5fdc8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
@@ -44,6 +44,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-hdfs</artifactId>
       <scope>test</scope>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0adc0471/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
index f1cad57..d9bf988 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
@@ -36,6 +36,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.mapreduce.filecache.ClientDistributedCacheManager;
 import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 
@@ -94,6 +96,11 @@ class JobResourceUploader {
         new FsPermission(JobSubmissionFiles.JOB_DIR_PERMISSION);
     mkdirs(jtFs, submitJobDir, mapredSysPerms);
 
+    if (!conf.getBoolean(MRJobConfig.MR_AM_STAGING_DIR_ERASURECODING_ENABLED,
+        MRJobConfig.DEFAULT_MR_AM_STAGING_ERASURECODING_ENABLED)) {
+      disableErasureCodingForPath(jtFs, submitJobDir);
+    }
+
     Collection<String> files = conf.getStringCollection("tmpfiles");
     Collection<String> libjars = conf.getStringCollection("tmpjars");
     Collection<String> archives = conf.getStringCollection("tmparchives");
@@ -575,4 +582,14 @@ class JobResourceUploader {
     }
     return finalPath;
   }
+
+  private void disableErasureCodingForPath(FileSystem fs, Path path)
+      throws IOException {
+    if (jtFs instanceof DistributedFileSystem) {
+      LOG.info("Disabling Erasure Coding for path: " + path);
+      DistributedFileSystem dfs = (DistributedFileSystem) jtFs;
+      dfs.setErasureCodingPolicy(path,
+          SystemErasureCodingPolicies.getReplicationPolicy().getName());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0adc0471/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index 2023ba3..86abb42 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -1037,4 +1037,9 @@ public interface MRJobConfig {
   String FINISH_JOB_WHEN_REDUCERS_DONE =
       "mapreduce.job.finish-when-all-reducers-done";
   boolean DEFAULT_FINISH_JOB_WHEN_REDUCERS_DONE = true;
+
+  String MR_AM_STAGING_DIR_ERASURECODING_ENABLED =
+      MR_AM_STAGING_DIR + "erasurecoding.enabled";
+
+  boolean DEFAULT_MR_AM_STAGING_ERASURECODING_ENABLED = false;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0adc0471/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index ee9b906..6b6faf2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -1261,6 +1261,15 @@
 </property>
 
 <property>
+  <name>yarn.app.mapreduce.am.staging-dir.erasurecoding.enabled</name>
+  <value>false</value>
+  <description>Whether Erasure Coding should be enabled for
+  files that are copied to the MR staging area. This is a job-level
+  setting.
+  </description>
+</property>
+
+<property>
   <name>mapreduce.am.max-attempts</name>
   <value>2</value>
   <description>The maximum number of application attempts. It is a

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0adc0471/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
index 20b7b7d..d0d7a34 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
@@ -20,6 +20,11 @@ package org.apache.hadoop.mapreduce;
 
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.times;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
 
 import java.io.IOException;
 import java.net.URI;
@@ -36,9 +41,12 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.mapred.JobConf;
 import org.junit.Assert;
 import org.junit.Test;
+import org.mockito.verification.VerificationMode;
 
 /**
  * A class for unit testing JobResourceUploader.
@@ -357,6 +365,40 @@ public class TestJobResourceUploader {
         expectedArchivesWithFrags, expectedJobJar);
   }
 
+  @Test
+  public void testErasureCodingDefault() throws IOException {
+    testErasureCodingSetting(true);
+  }
+
+  @Test
+  public void testErasureCodingDisabled() throws IOException {
+    testErasureCodingSetting(false);
+  }
+
+  private void testErasureCodingSetting(boolean defaultBehavior)
+      throws IOException {
+    JobConf jConf = new JobConf();
+    // don't set to false if EC remains disabled to check default setting
+    if (!defaultBehavior) {
+      jConf.setBoolean(MRJobConfig.MR_AM_STAGING_DIR_ERASURECODING_ENABLED,
+          true);
+    }
+
+    DistributedFileSystem fs = mock(DistributedFileSystem.class);
+    Path path = new Path("/");
+    when(fs.makeQualified(any(Path.class))).thenReturn(path);
+    JobResourceUploader uploader = new StubedUploader(fs, true);
+    Job job = Job.getInstance(jConf);
+
+    uploader.uploadResources(job, new Path("/test"));
+
+    String replicationPolicyName = SystemErasureCodingPolicies
+        .getReplicationPolicy().getName();
+    VerificationMode mode = defaultBehavior ? times(1) : never();
+    verify(fs, mode).setErasureCodingPolicy(eq(path),
+        eq(replicationPolicyName));
+  }
+
   private void runTmpResourcePathTest(JobResourceUploader uploader,
       ResourceConf rConf, JobConf jConf, String[] expectedFiles,
       String[] expectedArchives, String expectedJobJar) throws IOException {
@@ -698,6 +740,10 @@ public class TestJobResourceUploader {
       super(FileSystem.getLocal(conf), useWildcard);
     }
 
+    StubedUploader(FileSystem fs, boolean useWildcard) throws IOException {
+      super(fs, useWildcard);
+    }
+
     @Override
     FileStatus getFileStatus(Map<URI, FileStatus> statCache, Configuration job,
         Path p) throws IOException {


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


[23/50] [abbrv] hadoop git commit: HADOOP-14553. Add (parallelized) integration tests to hadoop-azure Contributed by Steve Loughran

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorization.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorization.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorization.java
index a3f2843..4bf6f04 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorization.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorization.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 
 import org.junit.Assume;
-import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 
@@ -49,8 +48,8 @@ public class TestNativeAzureFileSystemAuthorization
   protected MockWasbAuthorizerImpl authorizer;
 
   @Override
-  public Configuration getConfiguration() {
-    Configuration conf = super.getConfiguration();
+  public Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
     conf.set(NativeAzureFileSystem.KEY_AZURE_AUTHORIZATION, "true");
     conf.set(RemoteWasbAuthorizerImpl.KEY_REMOTE_AUTH_SERVICE_URLS, "http://localhost/");
     conf.set(NativeAzureFileSystem.AZURE_CHOWN_USERLIST_PROPERTY_NAME, "user1 , user2");
@@ -59,13 +58,12 @@ public class TestNativeAzureFileSystemAuthorization
 
   @Override
   protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
-    Configuration conf = getConfiguration();
-    return AzureBlobStorageTestAccount.create(conf);
+    return AzureBlobStorageTestAccount.create(createConfiguration());
   }
 
-
-  @Before
-  public void beforeMethod() {
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
     boolean useSecureMode = fs.getConf().getBoolean(KEY_USE_SECURE_MODE, false);
     boolean useAuthorization = fs.getConf().getBoolean(NativeAzureFileSystem.KEY_AZURE_AUTHORIZATION, false);
     Assume.assumeTrue("Test valid when both SecureMode and Authorization are enabled .. skipping",
@@ -76,7 +74,6 @@ public class TestNativeAzureFileSystemAuthorization
     fs.updateWasbAuthorizer(authorizer);
   }
 
-
   @Rule
   public ExpectedException expectedEx = ExpectedException.none();
 
@@ -95,7 +92,7 @@ public class TestNativeAzureFileSystemAuthorization
   }
 
   /**
-   * Setup the expected exception class, and exception message that the test is supposed to fail with
+   * Setup the expected exception class, and exception message that the test is supposed to fail with.
    */
   protected void setExpectedFailureMessage(String operation, Path path) {
     expectedEx.expect(WasbAuthorizationException.class);
@@ -104,7 +101,7 @@ public class TestNativeAzureFileSystemAuthorization
   }
 
   /**
-   * Positive test to verify Create access check
+   * Positive test to verify Create access check.
    * The file is created directly under an existing folder.
    * No intermediate folders need to be created.
    * @throws Throwable
@@ -128,7 +125,7 @@ public class TestNativeAzureFileSystemAuthorization
   }
 
   /**
-   * Positive test to verify Create access check
+   * Positive test to verify Create access check.
    * The test tries to create a file whose parent is non-existent to ensure that
    * the intermediate folders between ancestor and direct parent are being created
    * when proper ranger policies are configured.
@@ -155,7 +152,7 @@ public class TestNativeAzureFileSystemAuthorization
 
 
   /**
-   * Negative test to verify that create fails when trying to overwrite an existing file
+   * Negative test to verify that create fails when trying to overwrite an existing file.
    * without proper write permissions on the file being overwritten.
    * @throws Throwable
    */
@@ -181,7 +178,7 @@ public class TestNativeAzureFileSystemAuthorization
   }
 
   /**
-   * Positive test to verify that create succeeds when trying to overwrite an existing file
+   * Positive test to verify that create succeeds when trying to overwrite an existing file.
    * when proper write permissions on the file being overwritten are provided.
    * @throws Throwable
    */
@@ -232,7 +229,7 @@ public class TestNativeAzureFileSystemAuthorization
   }
 
   /**
-   * Positive test to verify listStatus access check
+   * Positive test to verify listStatus access check.
    * @throws Throwable
    */
   @Test
@@ -257,7 +254,7 @@ public class TestNativeAzureFileSystemAuthorization
   }
 
   /**
-   * Negative test to verify listStatus access check
+   * Negative test to verify listStatus access check.
    * @throws Throwable
    */
 
@@ -342,7 +339,7 @@ public class TestNativeAzureFileSystemAuthorization
   }
 
   /**
-   * Negative test to verify rename access check - the dstFolder disallows rename
+   * Negative test to verify rename access check - the dstFolder disallows rename.
    * @throws Throwable
    */
   @Test //(expected=WasbAuthorizationException.class)
@@ -373,7 +370,7 @@ public class TestNativeAzureFileSystemAuthorization
   }
 
   /**
-   * Positive test to verify rename access check - the dstFolder allows rename
+   * Positive test to verify rename access check - the dstFolder allows rename.
    * @throws Throwable
    */
   @Test
@@ -484,7 +481,7 @@ public class TestNativeAzureFileSystemAuthorization
   }
 
   /**
-   * Positive test to verify file delete access check
+   * Positive test to verify file delete access check.
    * @throws Throwable
    */
   @Test
@@ -506,7 +503,7 @@ public class TestNativeAzureFileSystemAuthorization
   }
 
   /**
-   * Negative test to verify file delete access check
+   * Negative test to verify file delete access check.
    * @throws Throwable
    */
   @Test //(expected=WasbAuthorizationException.class)
@@ -544,7 +541,7 @@ public class TestNativeAzureFileSystemAuthorization
 
   /**
    * Positive test to verify file delete access check, with intermediate folders
-   * Uses wildcard recursive permissions
+   * Uses wildcard recursive permissions.
    * @throws Throwable
    */
   @Test
@@ -582,7 +579,7 @@ public class TestNativeAzureFileSystemAuthorization
   }
 
   /**
-   * Positive test for mkdirs access check
+   * Positive test for mkdirs access check.
    * @throws Throwable
    */
   @Test
@@ -668,7 +665,7 @@ public class TestNativeAzureFileSystemAuthorization
     }
   }
   /**
-   * Negative test for mkdirs access check
+   * Negative test for mkdirs access check.
    * @throws Throwable
    */
   @Test //(expected=WasbAuthorizationException.class)
@@ -692,7 +689,7 @@ public class TestNativeAzureFileSystemAuthorization
   }
 
   /**
-   * Positive test triple slash format (wasb:///) access check
+   * Positive test triple slash format (wasb:///) access check.
    * @throws Throwable
    */
   @Test
@@ -708,7 +705,7 @@ public class TestNativeAzureFileSystemAuthorization
   }
 
   /**
-   * Negative test for setOwner when Authorization is enabled
+   * Negative test for setOwner when Authorization is enabled.
    */
   @Test
   public void testSetOwnerThrowsForUnauthorisedUsers() throws Throwable {
@@ -744,7 +741,7 @@ public class TestNativeAzureFileSystemAuthorization
 
   /**
    * Test for setOwner when Authorization is enabled and
-   * the user is specified in chown allowed user list
+   * the user is specified in chown allowed user list.
    * */
   @Test
   public void testSetOwnerSucceedsForAuthorisedUsers() throws Throwable {
@@ -785,7 +782,7 @@ public class TestNativeAzureFileSystemAuthorization
 
   /**
    * Test for setOwner when Authorization is enabled and
-   * the userlist is specified as '*'
+   * the userlist is specified as '*'.
    * */
   @Test
   public void testSetOwnerSucceedsForAnyUserWhenWildCardIsSpecified() throws Throwable {
@@ -829,7 +826,7 @@ public class TestNativeAzureFileSystemAuthorization
   }
 
   /** Test for setOwner  throws for illegal setup of chown
-   * allowed testSetOwnerSucceedsForAuthorisedUsers
+   * allowed testSetOwnerSucceedsForAuthorisedUsers.
    */
   @Test
   public void testSetOwnerFailsForIllegalSetup() throws Throwable {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorizationWithOwner.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorizationWithOwner.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorizationWithOwner.java
deleted file mode 100644
index 4bd4633..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAuthorizationWithOwner.java
+++ /dev/null
@@ -1,122 +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.azure;
-
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.apache.hadoop.security.UserGroupInformation;
-import java.security.PrivilegedExceptionAction;
-
-import org.apache.hadoop.fs.Path;
-import org.junit.Test;
-import org.junit.Before;
-import static org.junit.Assert.assertEquals;
-
-/**
- * Test class that runs wasb authorization tests with owner check enabled.
- */
-public class TestNativeAzureFileSystemAuthorizationWithOwner
-  extends TestNativeAzureFileSystemAuthorization {
-
-  @Before
-  public void beforeMethod() {
-    super.beforeMethod();
-    authorizer.init(fs.getConf(), true);
-  }
-
-  /**
-   * Test case when owner matches current user
-   */
-  @Test
-  public void testOwnerPermissionPositive() throws Throwable {
-
-    Path parentDir = new Path("/testOwnerPermissionPositive");
-    Path testPath = new Path(parentDir, "test.data");
-
-    authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
-    authorizer.addAuthRule(testPath.toString(), WasbAuthorizationOperations.READ.toString(), true);
-    authorizer.addAuthRule(parentDir.toString(), WasbAuthorizationOperations.WRITE.toString(), true);
-    // additional rule used for assertPathExists
-    authorizer.addAuthRule(parentDir.toString(), WasbAuthorizationOperations.READ.toString(), true);
-    fs.updateWasbAuthorizer(authorizer);
-
-    try {
-      // creates parentDir with owner as current user
-      fs.mkdirs(parentDir);
-      ContractTestUtils.assertPathExists(fs, "parentDir does not exist", parentDir);
-
-      fs.create(testPath);
-      fs.getFileStatus(testPath);
-      ContractTestUtils.assertPathExists(fs, "testPath does not exist", testPath);
-
-    } finally {
-      allowRecursiveDelete(fs, parentDir.toString());
-      fs.delete(parentDir, true);
-    }
-  }
-
-  /**
-   * Negative test case for owner does not match current user
-   */
-  @Test
-  public void testOwnerPermissionNegative() throws Throwable {
-    expectedEx.expect(WasbAuthorizationException.class);
-
-    Path parentDir = new Path("/testOwnerPermissionNegative");
-    Path childDir = new Path(parentDir, "childDir");
-
-    setExpectedFailureMessage("mkdirs", childDir);
-
-    authorizer.addAuthRule("/", WasbAuthorizationOperations.WRITE.toString(), true);
-    authorizer.addAuthRule(parentDir.toString(), WasbAuthorizationOperations.WRITE.toString(), true);
-
-    fs.updateWasbAuthorizer(authorizer);
-
-    try{
-      fs.mkdirs(parentDir);
-      UserGroupInformation ugiSuperUser = UserGroupInformation.createUserForTesting(
-          "testuser", new String[] {});
-
-      ugiSuperUser.doAs(new PrivilegedExceptionAction<Void>() {
-      @Override
-      public Void run() throws Exception {
-          fs.mkdirs(childDir);
-          return null;
-        }
-      });
-
-    } finally {
-       allowRecursiveDelete(fs, parentDir.toString());
-       fs.delete(parentDir, true);
-    }
-  }
-
-  /**
-   * Test to verify that retrieving owner information does not
-   * throw when file/folder does not exist
-   */
-  @Test
-  public void testRetrievingOwnerDoesNotFailWhenFileDoesNotExist() throws Throwable {
-
-    Path testdirectory = new Path("/testDirectory123454565");
-
-    String owner = fs.getOwnerForPath(testdirectory);
-    assertEquals("", owner);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockLocations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockLocations.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockLocations.java
index b2660bb..b280cac 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockLocations.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockLocations.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.fs.azure;
 
-import static org.junit.Assert.assertEquals;
-
 import java.io.OutputStream;
 
 import org.apache.hadoop.conf.Configuration;
@@ -29,7 +27,11 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.Test;
 
-public class TestNativeAzureFileSystemBlockLocations {
+/**
+ * Test block location logic.
+ */
+public class TestNativeAzureFileSystemBlockLocations
+    extends AbstractWasbTestWithTimeout {
   @Test
   public void testNumberOfBlocks() throws Exception {
     Configuration conf = new Configuration();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemClientLogging.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemClientLogging.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemClientLogging.java
deleted file mode 100644
index 4114e60..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemClientLogging.java
+++ /dev/null
@@ -1,140 +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.azure;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.net.URI;
-import java.util.StringTokenizer;
-
-import org.apache.commons.logging.impl.Log4JLogger;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
-import org.apache.log4j.Logger;
-import org.junit.Test;
-
-/**
- * Test to validate Azure storage client side logging. Tests works only when
- * testing with Live Azure storage because Emulator does not have support for
- * client-side logging.
- *
- */
-public class TestNativeAzureFileSystemClientLogging
-    extends AbstractWasbTestBase {
-
-  private AzureBlobStorageTestAccount testAccount;
-
-  // Core-site config controlling Azure Storage Client logging
-  private static final String KEY_LOGGING_CONF_STRING = "fs.azure.storage.client.logging";
-
-  // Temporary directory created using WASB.
-  private static final String TEMP_DIR = "tempDir";
-
-  /*
-   * Helper method to verify the client logging is working. This check primarily
-   * checks to make sure we see a line in the logs corresponding to the entity
-   * that is created during test run.
-   */
-  private boolean verifyStorageClientLogs(String capturedLogs, String entity)
-      throws Exception {
-
-    URI uri = testAccount.getRealAccount().getBlobEndpoint();
-    String container = testAccount.getRealContainer().getName();
-    String validateString = uri + Path.SEPARATOR + container + Path.SEPARATOR
-        + entity;
-    boolean entityFound = false;
-
-    StringTokenizer tokenizer = new StringTokenizer(capturedLogs, "\n");
-
-    while (tokenizer.hasMoreTokens()) {
-      String token = tokenizer.nextToken();
-      if (token.contains(validateString)) {
-        entityFound = true;
-        break;
-      }
-    }
-    return entityFound;
-  }
-
-  /*
-   * Helper method that updates the core-site config to enable/disable logging.
-   */
-  private void updateFileSystemConfiguration(Boolean loggingFlag)
-      throws Exception {
-
-    Configuration conf = fs.getConf();
-    conf.set(KEY_LOGGING_CONF_STRING, loggingFlag.toString());
-    URI uri = fs.getUri();
-    fs.initialize(uri, conf);
-  }
-
-  // Using WASB code to communicate with Azure Storage.
-  private void performWASBOperations() throws Exception {
-
-    Path tempDir = new Path(Path.SEPARATOR + TEMP_DIR);
-    fs.mkdirs(tempDir);
-    fs.delete(tempDir, true);
-  }
-
-  @Test
-  public void testLoggingEnabled() throws Exception {
-
-    LogCapturer logs = LogCapturer.captureLogs(new Log4JLogger(Logger
-        .getRootLogger()));
-
-    // Update configuration based on the Test.
-    updateFileSystemConfiguration(true);
-
-    performWASBOperations();
-
-    String output = getLogOutput(logs);
-    assertTrue("Log entry " + TEMP_DIR + " not found  in " + output,
-        verifyStorageClientLogs(output, TEMP_DIR));
-  }
-
-  protected String getLogOutput(LogCapturer logs) {
-    String output = logs.getOutput();
-    assertTrue("No log created/captured", !output.isEmpty());
-    return output;
-  }
-
-  @Test
-  public void testLoggingDisabled() throws Exception {
-
-    LogCapturer logs = LogCapturer.captureLogs(new Log4JLogger(Logger
-        .getRootLogger()));
-
-    // Update configuration based on the Test.
-    updateFileSystemConfiguration(false);
-
-    performWASBOperations();
-    String output = getLogOutput(logs);
-
-    assertFalse("Log entry " + TEMP_DIR + " found  in " + output,
-        verifyStorageClientLogs(output, TEMP_DIR));
-  }
-
-  @Override
-  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
-    testAccount = AzureBlobStorageTestAccount.create();
-    return testAccount;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemConcurrency.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemConcurrency.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemConcurrency.java
index cbfc563..655ae90 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemConcurrency.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemConcurrency.java
@@ -18,11 +18,6 @@
 
 package org.apache.hadoop.fs.azure;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
 import java.io.OutputStream;
 import java.io.PrintWriter;
 import java.io.StringWriter;
@@ -33,32 +28,30 @@ import java.util.concurrent.ConcurrentLinkedQueue;
 
 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.util.StringUtils;
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 
-public class TestNativeAzureFileSystemConcurrency {
-  private AzureBlobStorageTestAccount testAccount;
-  private FileSystem fs;
+public class TestNativeAzureFileSystemConcurrency extends AbstractWasbTestBase {
   private InMemoryBlockBlobStore backingStore;
 
-  @Before
+  @Override
   public void setUp() throws Exception {
-    testAccount = AzureBlobStorageTestAccount.createMock();
-    fs = testAccount.getFileSystem();
-    backingStore = testAccount.getMockStorage().getBackingStore();
+    super.setUp();
+    backingStore = getTestAccount().getMockStorage().getBackingStore();
   }
 
-  @After
+  @Override
   public void tearDown() throws Exception {
-    testAccount.cleanup();
-    fs = null;
+    super.tearDown();
     backingStore = null;
   }
 
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.createMock();
+  }
+
   @Test
   public void testLinkBlobs() throws Exception {
     Path filePath = new Path("/inProgress");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemConcurrencyLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemConcurrencyLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemConcurrencyLive.java
deleted file mode 100644
index 7c5899d..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemConcurrencyLive.java
+++ /dev/null
@@ -1,184 +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.azure;
-
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-/***
- * Test class to hold all Live Azure storage concurrency tests.
- */
-public class TestNativeAzureFileSystemConcurrencyLive
-    extends AbstractWasbTestBase {
-
-  private static final int THREAD_COUNT = 102;
-  private static final int TEST_EXECUTION_TIMEOUT = 5000;
-  @Override
-  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
-    return AzureBlobStorageTestAccount.create();
-  }
-
-  /**
-   * Validate contract for FileSystem.create when overwrite is true and there
-   * are concurrent callers of FileSystem.delete.  An existing file should be
-   * overwritten, even if the original destination exists but is deleted by an
-   * external agent during the create operation.
-   */
-  @Test(timeout = TEST_EXECUTION_TIMEOUT)
-  public void testConcurrentCreateDeleteFile() throws Exception {
-    Path testFile = new Path("test.dat");
-
-    List<CreateFileTask> tasks = new ArrayList<>(THREAD_COUNT);
-
-    for (int i = 0; i < THREAD_COUNT; i++) {
-      tasks.add(new CreateFileTask(fs, testFile));
-    }
-
-    ExecutorService es = null;
-
-    try {
-      es = Executors.newFixedThreadPool(THREAD_COUNT);
-
-      List<Future<Void>> futures = es.invokeAll(tasks);
-
-      for (Future<Void> future : futures) {
-        Assert.assertTrue(future.isDone());
-
-        // we are using Callable<V>, so if an exception
-        // occurred during the operation, it will be thrown
-        // when we call get
-        Assert.assertEquals(null, future.get());
-      }
-    } finally {
-      if (es != null) {
-        es.shutdownNow();
-      }
-    }
-  }
-
-  /**
-   * Validate contract for FileSystem.delete when invoked concurrently.
-   * One of the threads should successfully delete the file and return true;
-   * all other threads should return false.
-   */
-  @Test(timeout = TEST_EXECUTION_TIMEOUT)
-  public void testConcurrentDeleteFile() throws Exception {
-    Path testFile = new Path("test.dat");
-    fs.create(testFile).close();
-
-    List<DeleteFileTask> tasks = new ArrayList<>(THREAD_COUNT);
-
-    for (int i = 0; i < THREAD_COUNT; i++) {
-      tasks.add(new DeleteFileTask(fs, testFile));
-    }
-
-    ExecutorService es = null;
-    try {
-      es = Executors.newFixedThreadPool(THREAD_COUNT);
-
-      List<Future<Boolean>> futures = es.invokeAll(tasks);
-
-      int successCount = 0;
-      for (Future<Boolean> future : futures) {
-        Assert.assertTrue(future.isDone());
-
-        // we are using Callable<V>, so if an exception
-        // occurred during the operation, it will be thrown
-        // when we call get
-        Boolean success = future.get();
-        if (success) {
-          successCount++;
-        }
-      }
-
-      Assert.assertEquals(
-          "Exactly one delete operation should return true.",
-          1,
-          successCount);
-    } finally {
-      if (es != null) {
-        es.shutdownNow();
-      }
-    }
-  }
-}
-
-abstract class FileSystemTask<V> implements Callable<V> {
-  private final FileSystem fileSystem;
-  private final Path path;
-
-  protected FileSystem getFileSystem() {
-    return this.fileSystem;
-  }
-
-  protected Path getFilePath() {
-    return this.path;
-  }
-
-  FileSystemTask(FileSystem fs, Path p) {
-    this.fileSystem = fs;
-    this.path = p;
-  }
-
-  public abstract V call() throws Exception;
-}
-
-class DeleteFileTask extends FileSystemTask<Boolean> {
-
-  DeleteFileTask(FileSystem fs, Path p) {
-    super(fs, p);
-  }
-
-  @Override
-  public Boolean call() throws Exception {
-    return this.getFileSystem().delete(this.getFilePath(), false);
-  }
-}
-
-class CreateFileTask extends FileSystemTask<Void> {
-  CreateFileTask(FileSystem fs, Path p) {
-    super(fs, p);
-  }
-
-  public Void call() throws Exception {
-    FileSystem fs = getFileSystem();
-    Path p = getFilePath();
-
-    // Create an empty file and close the stream.
-    FSDataOutputStream stream = fs.create(p, true);
-    stream.close();
-
-    // Delete the file.  We don't care if delete returns true or false.
-    // We just want to ensure the file does not exist.
-    this.getFileSystem().delete(this.getFilePath(), false);
-
-    return null;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractEmulator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractEmulator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractEmulator.java
deleted file mode 100644
index 217ca81..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractEmulator.java
+++ /dev/null
@@ -1,48 +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.azure;
-
-import static org.junit.Assume.assumeNotNull;
-
-import org.apache.hadoop.fs.FileSystemContractBaseTest;
-import org.junit.After;
-import org.junit.Before;
-
-public class TestNativeAzureFileSystemContractEmulator extends
-    FileSystemContractBaseTest {
-  private AzureBlobStorageTestAccount testAccount;
-
-  @Before
-  public void setUp() throws Exception {
-    testAccount = AzureBlobStorageTestAccount.createForEmulator();
-    if (testAccount != null) {
-      fs = testAccount.getFileSystem();
-    }
-    assumeNotNull(fs);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (testAccount != null) {
-      testAccount.cleanup();
-      testAccount = null;
-      fs = null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractLive.java
deleted file mode 100644
index b546009..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractLive.java
+++ /dev/null
@@ -1,80 +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.azure;
-
-import static org.junit.Assume.assumeNotNull;
-
-import org.apache.hadoop.fs.FileSystemContractBaseTest;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-public class TestNativeAzureFileSystemContractLive extends
-    FileSystemContractBaseTest {
-  private AzureBlobStorageTestAccount testAccount;
-
-  @Before
-  public void setUp() throws Exception {
-    testAccount = AzureBlobStorageTestAccount.create();
-    if (testAccount != null) {
-      fs = testAccount.getFileSystem();
-    }
-    assumeNotNull(fs);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (testAccount != null) {
-      testAccount.cleanup();
-      testAccount = null;
-      fs = null;
-    }
-  }
-
-  /**
-   * The following tests are failing on Azure and the Azure 
-   * file system code needs to be modified to make them pass.
-   * A separate work item has been opened for this.
-   */
-  @Ignore
-  @Test
-  public void testMoveFileUnderParent() throws Throwable {
-  }
-
-  @Ignore
-  @Test
-  public void testRenameFileToSelf() throws Throwable {
-  }
-
-  @Ignore
-  @Test
-  public void testRenameChildDirForbidden() throws Exception {
-  }
-
-  @Ignore
-  @Test
-  public void testMoveDirUnderParent() throws Throwable {
-  }
-
-  @Ignore
-  @Test
-  public void testRenameDirToSelf() throws Throwable {
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractMocked.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractMocked.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractMocked.java
index f458bb3..2809260 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractMocked.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractMocked.java
@@ -23,6 +23,9 @@ import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
 
+/**
+ * Mocked testing of FileSystemContractBaseTest.
+ */
 public class TestNativeAzureFileSystemContractMocked extends
     FileSystemContractBaseTest {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractPageBlobLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractPageBlobLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractPageBlobLive.java
deleted file mode 100644
index 2a88ad2..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractPageBlobLive.java
+++ /dev/null
@@ -1,93 +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.azure;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystemContractBaseTest;
-import org.junit.After;
-import static org.junit.Assume.assumeNotNull;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-public class TestNativeAzureFileSystemContractPageBlobLive extends
-    FileSystemContractBaseTest {
-  private AzureBlobStorageTestAccount testAccount;
-
-  private AzureBlobStorageTestAccount createTestAccount()
-      throws Exception {
-    Configuration conf = new Configuration();
-
-    // Configure the page blob directories key so every file created is a page blob.
-    conf.set(AzureNativeFileSystemStore.KEY_PAGE_BLOB_DIRECTORIES, "/");
-
-    // Configure the atomic rename directories key so every folder will have
-    // atomic rename applied.
-    conf.set(AzureNativeFileSystemStore.KEY_ATOMIC_RENAME_DIRECTORIES, "/");
-    return AzureBlobStorageTestAccount.create(conf);
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    testAccount = createTestAccount();
-    if (testAccount != null) {
-      fs = testAccount.getFileSystem();
-    }
-    assumeNotNull(fs);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (testAccount != null) {
-      testAccount.cleanup();
-      testAccount = null;
-      fs = null;
-    }
-  }
-
-  /**
-   * The following tests are failing on Azure and the Azure 
-   * file system code needs to be modified to make them pass.
-   * A separate work item has been opened for this.
-   */
-  @Ignore
-  @Test
-  public void testMoveFileUnderParent() throws Throwable {
-  }
-
-  @Ignore
-  @Test
-  public void testRenameFileToSelf() throws Throwable {
-  }
-  
-  @Ignore
-  @Test
-  public void testRenameChildDirForbidden() throws Exception {
-  }
-  
-  @Ignore
-  @Test
-  public void testMoveDirUnderParent() throws Throwable {
-  }
-  
-  @Ignore
-  @Test
-  public void testRenameDirToSelf() throws Throwable {
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemFileNameCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemFileNameCheck.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemFileNameCheck.java
index 82eabaa..0dfbb37 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemFileNameCheck.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemFileNameCheck.java
@@ -18,17 +18,11 @@
 
 package org.apache.hadoop.fs.azure;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.util.HashMap;
 
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.junit.After;
-import org.junit.Before;
+
 import org.junit.Test;
 
 /**
@@ -38,24 +32,18 @@ import org.junit.Test;
  * creation/rename of files/directories through WASB that have colons in the
  * names.
  */
-public class TestNativeAzureFileSystemFileNameCheck {
-  private FileSystem fs = null;
-  private AzureBlobStorageTestAccount testAccount = null;
+public class TestNativeAzureFileSystemFileNameCheck extends AbstractWasbTestBase {
   private String root = null;
 
-  @Before
+  @Override
   public void setUp() throws Exception {
-    testAccount = AzureBlobStorageTestAccount.createMock();
-    fs = testAccount.getFileSystem();
+    super.setUp();
     root = fs.getUri().toString();
   }
 
-  @After
-  public void tearDown() throws Exception {
-    testAccount.cleanup();
-    root = null;
-    fs = null;
-    testAccount = null;
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.createMock();
   }
 
   @Test
@@ -138,4 +126,4 @@ public class TestNativeAzureFileSystemFileNameCheck {
     fsck.run(new String[] { p.toString() });
     return fsck.getPathNameWarning();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemLive.java
deleted file mode 100644
index 6baba33..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemLive.java
+++ /dev/null
@@ -1,242 +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.azure;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.concurrent.CountDownLatch;
-
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-
-import org.junit.Test;
-
-import com.microsoft.azure.storage.StorageException;
-
-/*
- * Tests the Native Azure file system (WASB) against an actual blob store if
- * provided in the environment.
- */
-public class TestNativeAzureFileSystemLive extends
-    NativeAzureFileSystemBaseTest {
-
-  @Override
-  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
-    return AzureBlobStorageTestAccount.create();
-  }
-
-  @Test
-  public void testLazyRenamePendingCanOverwriteExistingFile()
-    throws Exception {
-    final String SRC_FILE_KEY = "srcFile";
-    final String DST_FILE_KEY = "dstFile";
-    Path srcPath = new Path(SRC_FILE_KEY);
-    FSDataOutputStream srcStream = fs.create(srcPath);
-    assertTrue(fs.exists(srcPath));
-    Path dstPath = new Path(DST_FILE_KEY);
-    FSDataOutputStream dstStream = fs.create(dstPath);
-    assertTrue(fs.exists(dstPath));
-    NativeAzureFileSystem nfs = (NativeAzureFileSystem)fs;
-    final String fullSrcKey = nfs.pathToKey(nfs.makeAbsolute(srcPath));
-    final String fullDstKey = nfs.pathToKey(nfs.makeAbsolute(dstPath));
-    nfs.getStoreInterface().rename(fullSrcKey, fullDstKey, true, null);
-    assertTrue(fs.exists(dstPath));
-    assertFalse(fs.exists(srcPath));
-    IOUtils.cleanup(null, srcStream);
-    IOUtils.cleanup(null, dstStream);
-  }
-  /**
-   * Tests fs.delete() function to delete a blob when another blob is holding a
-   * lease on it. Delete if called without a lease should fail if another process
-   * is holding a lease and throw appropriate exception
-   * This is a scenario that would happen in HMaster startup when it tries to
-   * clean up the temp dirs while the HMaster process which was killed earlier
-   * held lease on the blob when doing some DDL operation
-   */
-  @Test
-  public void testDeleteThrowsExceptionWithLeaseExistsErrorMessage()
-      throws Exception {
-    LOG.info("Starting test");
-    final String FILE_KEY = "fileWithLease";
-    // Create the file
-    Path path = new Path(FILE_KEY);
-    fs.create(path);
-    assertTrue(fs.exists(path));
-    NativeAzureFileSystem nfs = (NativeAzureFileSystem)fs;
-    final String fullKey = nfs.pathToKey(nfs.makeAbsolute(path));
-    final AzureNativeFileSystemStore store = nfs.getStore();
-
-    // Acquire the lease on the file in a background thread
-    final CountDownLatch leaseAttemptComplete = new CountDownLatch(1);
-    final CountDownLatch beginningDeleteAttempt = new CountDownLatch(1);
-    Thread t = new Thread() {
-      @Override
-      public void run() {
-        // Acquire the lease and then signal the main test thread.
-        SelfRenewingLease lease = null;
-        try {
-          lease = store.acquireLease(fullKey);
-          LOG.info("Lease acquired: " + lease.getLeaseID());
-        } catch (AzureException e) {
-          LOG.warn("Lease acqusition thread unable to acquire lease", e);
-        } finally {
-          leaseAttemptComplete.countDown();
-        }
-
-        // Wait for the main test thread to signal it will attempt the delete.
-        try {
-          beginningDeleteAttempt.await();
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-        }
-
-        // Keep holding the lease past the lease acquisition retry interval, so
-        // the test covers the case of delete retrying to acquire the lease.
-        try {
-          Thread.sleep(SelfRenewingLease.LEASE_ACQUIRE_RETRY_INTERVAL * 3);
-        } catch (InterruptedException ex) {
-          Thread.currentThread().interrupt();
-        }
-
-        try {
-          if (lease != null){
-            LOG.info("Freeing lease");
-            lease.free();
-          }
-        } catch (StorageException se) {
-          LOG.warn("Unable to free lease.", se);
-        }
-      }
-    };
-
-    // Start the background thread and wait for it to signal the lease is held.
-    t.start();
-    try {
-      leaseAttemptComplete.await();
-    } catch (InterruptedException ex) {
-      Thread.currentThread().interrupt();
-    }
-
-    // Try to delete the same file
-    beginningDeleteAttempt.countDown();
-    store.delete(fullKey);
-
-    // At this point file SHOULD BE DELETED
-    assertFalse(fs.exists(path));
-  }
-
-  /**
-   * Check that isPageBlobKey works as expected. This assumes that
-   * in the test configuration, the list of supported page blob directories
-   * only includes "pageBlobs". That's why this test is made specific
-   * to this subclass.
-   */
-  @Test
-  public void testIsPageBlobKey() {
-    AzureNativeFileSystemStore store = ((NativeAzureFileSystem) fs).getStore();
-
-    // Use literal strings so it's easier to understand the tests.
-    // In case the constant changes, we want to know about it so we can update this test.
-    assertEquals(AzureBlobStorageTestAccount.DEFAULT_PAGE_BLOB_DIRECTORY, "pageBlobs");
-
-    // URI prefix for test environment.
-    String uriPrefix = "file:///";
-
-    // negative tests
-    String[] negativeKeys = { "", "/", "bar", "bar/", "bar/pageBlobs", "bar/pageBlobs/foo",
-        "bar/pageBlobs/foo/", "/pageBlobs/", "/pageBlobs", "pageBlobs", "pageBlobsxyz/" };
-    for (String s : negativeKeys) {
-      assertFalse(store.isPageBlobKey(s));
-      assertFalse(store.isPageBlobKey(uriPrefix + s));
-    }
-
-    // positive tests
-    String[] positiveKeys = { "pageBlobs/", "pageBlobs/foo/", "pageBlobs/foo/bar/" };
-    for (String s : positiveKeys) {
-      assertTrue(store.isPageBlobKey(s));
-      assertTrue(store.isPageBlobKey(uriPrefix + s));
-    }
-  }
-
-  /**
-   * Test that isAtomicRenameKey() works as expected.
-   */
-  @Test
-  public void testIsAtomicRenameKey() {
-
-    AzureNativeFileSystemStore store = ((NativeAzureFileSystem) fs).getStore();
-
-    // We want to know if the default configuration changes so we can fix
-    // this test.
-    assertEquals(AzureBlobStorageTestAccount.DEFAULT_ATOMIC_RENAME_DIRECTORIES,
-        "/atomicRenameDir1,/atomicRenameDir2");
-
-    // URI prefix for test environment.
-    String uriPrefix = "file:///";
-
-    // negative tests
-    String[] negativeKeys = { "", "/", "bar", "bar/", "bar/hbase",
-        "bar/hbase/foo", "bar/hbase/foo/", "/hbase/", "/hbase", "hbase",
-        "hbasexyz/", "foo/atomicRenameDir1/"};
-    for (String s : negativeKeys) {
-      assertFalse(store.isAtomicRenameKey(s));
-      assertFalse(store.isAtomicRenameKey(uriPrefix + s));
-    }
-
-    // Positive tests. The directories for atomic rename are /hbase
-    // plus the ones in the configuration (DEFAULT_ATOMIC_RENAME_DIRECTORIES
-    // for this test).
-    String[] positiveKeys = { "hbase/", "hbase/foo/", "hbase/foo/bar/",
-        "atomicRenameDir1/foo/", "atomicRenameDir2/bar/"};
-    for (String s : positiveKeys) {
-      assertTrue(store.isAtomicRenameKey(s));
-      assertTrue(store.isAtomicRenameKey(uriPrefix + s));
-    }
-  }
-
-  /**
-   * Tests fs.mkdir() function to create a target blob while another thread
-   * is holding the lease on the blob. mkdir should not fail since the blob
-   * already exists.
-   * This is a scenario that would happen in HBase distributed log splitting.
-   * Multiple threads will try to create and update "recovered.edits" folder
-   * under the same path.
-   */
-  @Test
-  public void testMkdirOnExistingFolderWithLease() throws Exception {
-    SelfRenewingLease lease;
-    final String FILE_KEY = "folderWithLease";
-    // Create the folder
-    fs.mkdirs(new Path(FILE_KEY));
-    NativeAzureFileSystem nfs = (NativeAzureFileSystem) fs;
-    String fullKey = nfs.pathToKey(nfs.makeAbsolute(new Path(FILE_KEY)));
-    AzureNativeFileSystemStore store = nfs.getStore();
-    // Acquire the lease on the folder
-    lease = store.acquireLease(fullKey);
-    assertTrue(lease.getLeaseID() != null);
-    // Try to create the same folder
-    store.storeEmptyFolder(fullKey,
-      nfs.createPermissionStatus(FsPermission.getDirDefault()));
-    lease.free();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemMocked.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemMocked.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemMocked.java
index aa1e4f7..20d45b2 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemMocked.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemMocked.java
@@ -21,6 +21,10 @@ package org.apache.hadoop.fs.azure;
 import java.io.IOException;
 import org.junit.Ignore;
 
+/**
+ * Run {@link NativeAzureFileSystemBaseTest} tests against a mocked store,
+ * skipping tests of unsupported features
+ */
 public class TestNativeAzureFileSystemMocked extends
     NativeAzureFileSystemBaseTest {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemUploadLogic.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemUploadLogic.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemUploadLogic.java
index 4c2df8d..7f63295 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemUploadLogic.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemUploadLogic.java
@@ -18,41 +18,27 @@
 
 package org.apache.hadoop.fs.azure;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
 import java.io.ByteArrayInputStream;
 import java.io.InputStream;
 import java.io.OutputStream;
 
 import org.apache.hadoop.fs.Path;
 
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
 
 /**
  * Tests for the upload, buffering and flush logic in WASB.
  */
-public class TestNativeAzureFileSystemUploadLogic {
-  private AzureBlobStorageTestAccount testAccount;
+public class TestNativeAzureFileSystemUploadLogic extends AbstractWasbTestBase {
 
   // Just an arbitrary number so that the values I write have a predictable
   // pattern: 0, 1, 2, .. , 45, 46, 0, 1, 2, ...
   static final int byteValuePeriod = 47;
 
-  @Before
-  public void setUp() throws Exception {
-    testAccount = AzureBlobStorageTestAccount.createMock();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (testAccount != null) {
-      testAccount.cleanup();
-      testAccount = null;
-    }
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.createMock();
   }
 
   /**
@@ -126,9 +112,9 @@ public class TestNativeAzureFileSystemUploadLogic {
    * @param expectedSize The expected size of the data in there.
    */
   private void assertDataInFile(Path file, int expectedSize) throws Exception {
-    InputStream inStream = testAccount.getFileSystem().open(file);
-    assertDataInStream(inStream, expectedSize);
-    inStream.close();
+    try(InputStream inStream = getFileSystem().open(file)) {
+      assertDataInStream(inStream, expectedSize);
+    }
   }
 
   /**
@@ -139,7 +125,7 @@ public class TestNativeAzureFileSystemUploadLogic {
   private void assertDataInTempBlob(int expectedSize) throws Exception {
     // Look for the temporary upload blob in the backing store.
     InMemoryBlockBlobStore backingStore =
-        testAccount.getMockStorage().getBackingStore();
+        getTestAccount().getMockStorage().getBackingStore();
     String tempKey = null;
     for (String key : backingStore.getKeys()) {
       if (key.contains(NativeAzureFileSystem.AZURE_TEMP_FOLDER)) {
@@ -149,9 +135,10 @@ public class TestNativeAzureFileSystemUploadLogic {
       }
     }
     assertNotNull(tempKey);
-    InputStream inStream = new ByteArrayInputStream(backingStore.getContent(tempKey));
-    assertDataInStream(inStream, expectedSize);
-    inStream.close();
+    try (InputStream inStream = new ByteArrayInputStream(
+        backingStore.getContent(tempKey))) {
+      assertDataInStream(inStream, expectedSize);
+    }
   }
 
   /**
@@ -162,25 +149,30 @@ public class TestNativeAzureFileSystemUploadLogic {
    */
   private void testConsistencyAfterManyFlushes(FlushFrequencyVariation variation)
       throws Exception {
-    Path uploadedFile = new Path("/uploadedFile");
-    OutputStream outStream = testAccount.getFileSystem().create(uploadedFile);
-    final int totalSize = 9123;
-    int flushPeriod;
-    switch (variation) {
-      case BeforeSingleBufferFull: flushPeriod = 300; break;
-      case AfterSingleBufferFull: flushPeriod = 600; break;
-      case AfterAllRingBufferFull: flushPeriod = 1600; break;
-      default:
-        throw new IllegalArgumentException("Unknown variation: " + variation);
-    }
-    for (int i = 0; i < totalSize; i++) {
-      outStream.write(i % byteValuePeriod);
-      if ((i + 1) % flushPeriod == 0) {
-        outStream.flush();
-        assertDataInTempBlob(i + 1);
+    Path uploadedFile = methodPath();
+    try {
+      OutputStream outStream = getFileSystem().create(uploadedFile);
+      final int totalSize = 9123;
+      int flushPeriod;
+      switch (variation) {
+        case BeforeSingleBufferFull: flushPeriod = 300; break;
+        case AfterSingleBufferFull: flushPeriod = 600; break;
+        case AfterAllRingBufferFull: flushPeriod = 1600; break;
+        default:
+          throw new IllegalArgumentException("Unknown variation: " + variation);
       }
+      for (int i = 0; i < totalSize; i++) {
+        outStream.write(i % byteValuePeriod);
+        if ((i + 1) % flushPeriod == 0) {
+          outStream.flush();
+          assertDataInTempBlob(i + 1);
+        }
+      }
+      outStream.close();
+      assertDataInFile(uploadedFile, totalSize);
+    } finally {
+      getFileSystem().delete(uploadedFile, false);
+
     }
-    outStream.close();
-    assertDataInFile(uploadedFile, totalSize);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java
index 544d6ab..303a89a 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java
@@ -18,11 +18,6 @@
 
 package org.apache.hadoop.fs.azure;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
 import java.util.HashMap;
 
 import org.apache.hadoop.fs.FileStatus;
@@ -37,7 +32,8 @@ import org.junit.Test;
  * Tests that WASB handles things gracefully when users add blobs to the Azure
  * Storage container from outside WASB's control.
  */
-public class TestOutOfBandAzureBlobOperations {
+public class TestOutOfBandAzureBlobOperations
+    extends AbstractWasbTestWithTimeout {
   private AzureBlobStorageTestAccount testAccount;
   private FileSystem fs;
   private InMemoryBlockBlobStore backingStore;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperationsLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperationsLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperationsLive.java
deleted file mode 100644
index 60b01c6..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperationsLive.java
+++ /dev/null
@@ -1,203 +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.azure;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assume.assumeNotNull;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.microsoft.azure.storage.blob.BlobOutputStream;
-import com.microsoft.azure.storage.blob.CloudBlockBlob;
-
-public class TestOutOfBandAzureBlobOperationsLive {
-  private FileSystem fs;
-  private AzureBlobStorageTestAccount testAccount;
-
-  @Before
-  public void setUp() throws Exception {
-    testAccount = AzureBlobStorageTestAccount.create();
-    if (testAccount != null) {
-      fs = testAccount.getFileSystem();
-    }
-    assumeNotNull(testAccount);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (testAccount != null) {
-      testAccount.cleanup();
-      testAccount = null;
-      fs = null;
-    }
-  }
-
-  // scenario for this particular test described at MONARCH-HADOOP-764
-  // creating a file out-of-band would confuse mkdirs("<oobfilesUncleFolder>")
-  // eg oob creation of "user/<name>/testFolder/a/input/file"
-  // Then wasb creation of "user/<name>/testFolder/a/output" fails
-  @Test
-  public void outOfBandFolder_uncleMkdirs() throws Exception {
-
-    // NOTE: manual use of CloubBlockBlob targets working directory explicitly.
-    // WASB driver methods prepend working directory implicitly.
-    String workingDir = "user/"
-        + UserGroupInformation.getCurrentUser().getShortUserName() + "/";
-
-    CloudBlockBlob blob = testAccount.getBlobReference(workingDir
-        + "testFolder1/a/input/file");
-    BlobOutputStream s = blob.openOutputStream();
-    s.close();
-    assertTrue(fs.exists(new Path("testFolder1/a/input/file")));
-
-    Path targetFolder = new Path("testFolder1/a/output");
-    assertTrue(fs.mkdirs(targetFolder));
-  }
-
-  // scenario for this particular test described at MONARCH-HADOOP-764
-  @Test
-  public void outOfBandFolder_parentDelete() throws Exception {
-
-    // NOTE: manual use of CloubBlockBlob targets working directory explicitly.
-    // WASB driver methods prepend working directory implicitly.
-    String workingDir = "user/"
-        + UserGroupInformation.getCurrentUser().getShortUserName() + "/";
-    CloudBlockBlob blob = testAccount.getBlobReference(workingDir
-        + "testFolder2/a/input/file");
-    BlobOutputStream s = blob.openOutputStream();
-    s.close();
-    assertTrue(fs.exists(new Path("testFolder2/a/input/file")));
-
-    Path targetFolder = new Path("testFolder2/a/input");
-    assertTrue(fs.delete(targetFolder, true));
-  }
-
-  @Test
-  public void outOfBandFolder_rootFileDelete() throws Exception {
-
-    CloudBlockBlob blob = testAccount.getBlobReference("fileY");
-    BlobOutputStream s = blob.openOutputStream();
-    s.close();
-    assertTrue(fs.exists(new Path("/fileY")));
-    assertTrue(fs.delete(new Path("/fileY"), true));
-  }
-
-  @Test
-  public void outOfBandFolder_firstLevelFolderDelete() throws Exception {
-
-    CloudBlockBlob blob = testAccount.getBlobReference("folderW/file");
-    BlobOutputStream s = blob.openOutputStream();
-    s.close();
-    assertTrue(fs.exists(new Path("/folderW")));
-    assertTrue(fs.exists(new Path("/folderW/file")));
-    assertTrue(fs.delete(new Path("/folderW"), true));
-  }
-
-  // scenario for this particular test described at MONARCH-HADOOP-764
-  @Test
-  public void outOfBandFolder_siblingCreate() throws Exception {
-
-    // NOTE: manual use of CloubBlockBlob targets working directory explicitly.
-    // WASB driver methods prepend working directory implicitly.
-    String workingDir = "user/"
-        + UserGroupInformation.getCurrentUser().getShortUserName() + "/";
-    CloudBlockBlob blob = testAccount.getBlobReference(workingDir
-        + "testFolder3/a/input/file");
-    BlobOutputStream s = blob.openOutputStream();
-    s.close();
-    assertTrue(fs.exists(new Path("testFolder3/a/input/file")));
-
-    Path targetFile = new Path("testFolder3/a/input/file2");
-    FSDataOutputStream s2 = fs.create(targetFile);
-    s2.close();
-  }
-
-  // scenario for this particular test described at MONARCH-HADOOP-764
-  // creating a new file in the root folder
-  @Test
-  public void outOfBandFolder_create_rootDir() throws Exception {
-    Path targetFile = new Path("/newInRoot");
-    FSDataOutputStream s2 = fs.create(targetFile);
-    s2.close();
-  }
-
-  // scenario for this particular test described at MONARCH-HADOOP-764
-  @Test
-  public void outOfBandFolder_rename() throws Exception {
-
-    // NOTE: manual use of CloubBlockBlob targets working directory explicitly.
-    // WASB driver methods prepend working directory implicitly.
-    String workingDir = "user/"
-        + UserGroupInformation.getCurrentUser().getShortUserName() + "/";
-    CloudBlockBlob blob = testAccount.getBlobReference(workingDir
-        + "testFolder4/a/input/file");
-    BlobOutputStream s = blob.openOutputStream();
-    s.close();
-
-    Path srcFilePath = new Path("testFolder4/a/input/file");
-    assertTrue(fs.exists(srcFilePath));
-
-    Path destFilePath = new Path("testFolder4/a/input/file2");
-    fs.rename(srcFilePath, destFilePath);
-  }
-
-  // Verify that you can rename a file which is the only file in an implicit folder in the
-  // WASB file system.
-  // scenario for this particular test described at MONARCH-HADOOP-892
-  @Test
-  public void outOfBandSingleFile_rename() throws Exception {
-
-    //NOTE: manual use of CloubBlockBlob targets working directory explicitly.
-    //       WASB driver methods prepend working directory implicitly.
-    String workingDir = "user/" + UserGroupInformation.getCurrentUser().getShortUserName() + "/";
-    CloudBlockBlob blob = testAccount.getBlobReference(workingDir + "testFolder5/a/input/file");
-    BlobOutputStream s = blob.openOutputStream();
-    s.close();
-
-    Path srcFilePath = new Path("testFolder5/a/input/file");
-    assertTrue(fs.exists(srcFilePath));
-
-    Path destFilePath = new Path("testFolder5/file2");
-    fs.rename(srcFilePath, destFilePath);
-  }
-
-  // WASB must force explicit parent directories in create, delete, mkdirs, rename.
-  // scenario for this particular test described at MONARCH-HADOOP-764
-  @Test
-  public void outOfBandFolder_rename_rootLevelFiles() throws Exception {
-
-    // NOTE: manual use of CloubBlockBlob targets working directory explicitly.
-    // WASB driver methods prepend working directory implicitly.
-    CloudBlockBlob blob = testAccount.getBlobReference("fileX");
-    BlobOutputStream s = blob.openOutputStream();
-    s.close();
-
-    Path srcFilePath = new Path("/fileX");
-    assertTrue(fs.exists(srcFilePath));
-
-    Path destFilePath = new Path("/fileXrename");
-    fs.rename(srcFilePath, destFilePath);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestReadAndSeekPageBlobAfterWrite.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestReadAndSeekPageBlobAfterWrite.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestReadAndSeekPageBlobAfterWrite.java
deleted file mode 100644
index 41b8386..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestReadAndSeekPageBlobAfterWrite.java
+++ /dev/null
@@ -1,355 +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.azure;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assume.assumeNotNull;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.Arrays;
-import java.util.Random;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FSDataInputStream;
-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.AzureException;
-import org.apache.hadoop.util.Time;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Write data into a page blob and verify you can read back all of it
- * or just a part of it.
- */
-public class TestReadAndSeekPageBlobAfterWrite {
-  private static final Log LOG = LogFactory.getLog(TestReadAndSeekPageBlobAfterWrite.class);
-
-  private FileSystem fs;
-  private AzureBlobStorageTestAccount testAccount;
-  private byte[] randomData;
-
-  // Page blob physical page size
-  private static final int PAGE_SIZE = PageBlobFormatHelpers.PAGE_SIZE;
-
-  // Size of data on page (excluding header)
-  private static final int PAGE_DATA_SIZE = PAGE_SIZE - PageBlobFormatHelpers.PAGE_HEADER_SIZE;
-  private static final int MAX_BYTES = 33554432; // maximum bytes in a file that we'll test
-  private static final int MAX_PAGES = MAX_BYTES / PAGE_SIZE; // maximum number of pages we'll test
-  private Random rand = new Random();
-
-  // A key with a prefix under /pageBlobs, which for the test file system will
-  // force use of a page blob.
-  private static final String KEY = "/pageBlobs/file.dat";
-  private static final Path PATH = new Path(KEY); // path of page blob file to read and write
-
-  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
-    return AzureBlobStorageTestAccount.create();
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    testAccount = createTestAccount();
-    if (testAccount != null) {
-      fs = testAccount.getFileSystem();
-    }
-    assumeNotNull(testAccount);
-
-    // Make sure we are using an integral number of pages.
-    assertEquals(0, MAX_BYTES % PAGE_SIZE);
-
-    // load an in-memory array of random data
-    randomData = new byte[PAGE_SIZE * MAX_PAGES];
-    rand.nextBytes(randomData);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (testAccount != null) {
-      testAccount.cleanup();
-      testAccount = null;
-      fs = null;
-    }
-  }
-
-  /**
-   * Make sure the file name (key) is a page blob file name. If anybody changes that,
-   * we need to come back and update this test class.
-   */
-  @Test
-  public void testIsPageBlobFileName() {
-    AzureNativeFileSystemStore store = ((NativeAzureFileSystem) fs).getStore();
-    String[] a = KEY.split("/");
-    String key2 = a[1] + "/";
-    assertTrue(store.isPageBlobKey(key2));
-  }
-
-  /**
-   * For a set of different file sizes, write some random data to a page blob,
-   * read it back, and compare that what was read is the same as what was written.
-   */
-  @Test
-  public void testReadAfterWriteRandomData() throws IOException {
-
-    // local shorthand
-    final int PDS = PAGE_DATA_SIZE;
-
-    // Test for sizes at and near page boundaries
-    int[] dataSizes = {
-
-        // on first page
-        0, 1, 2, 3,
-
-        // Near first physical page boundary (because the implementation
-        // stores PDS + the page header size bytes on each page).
-        PDS - 1, PDS, PDS + 1, PDS + 2, PDS + 3,
-
-        // near second physical page boundary
-        (2 * PDS) - 1, (2 * PDS), (2 * PDS) + 1, (2 * PDS) + 2, (2 * PDS) + 3,
-
-        // near tenth physical page boundary
-        (10 * PDS) - 1, (10 * PDS), (10 * PDS) + 1, (10 * PDS) + 2, (10 * PDS) + 3,
-
-        // test one big size, >> 4MB (an internal buffer size in the code)
-        MAX_BYTES
-    };
-
-    for (int i : dataSizes) {
-      testReadAfterWriteRandomData(i);
-    }
-  }
-
-  private void testReadAfterWriteRandomData(int size) throws IOException {
-    writeRandomData(size);
-    readRandomDataAndVerify(size);
-  }
-
-  /**
-   * Read "size" bytes of data and verify that what was read and what was written
-   * are the same.
-   */
-  private void readRandomDataAndVerify(int size) throws AzureException, IOException {
-    byte[] b = new byte[size];
-    FSDataInputStream stream = fs.open(PATH);
-    int bytesRead = stream.read(b);
-    stream.close();
-    assertEquals(bytesRead, size);
-
-    // compare the data read to the data written
-    assertTrue(comparePrefix(randomData, b, size));
-  }
-
-  // return true if the beginning "size" values of the arrays are the same
-  private boolean comparePrefix(byte[] a, byte[] b, int size) {
-    if (a.length < size || b.length < size) {
-      return false;
-    }
-    for (int i = 0; i < size; i++) {
-      if (a[i] != b[i]) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  // Write a specified amount of random data to the file path for this test class.
-  private void writeRandomData(int size) throws IOException {
-    OutputStream output = fs.create(PATH);
-    output.write(randomData, 0, size);
-    output.close();
-  }
-
-  /**
-   * Write data to a page blob, open it, seek, and then read a range of data.
-   * Then compare that the data read from that range is the same as the data originally written.
-   */
-  @Test
-  public void testPageBlobSeekAndReadAfterWrite() throws IOException {
-    writeRandomData(PAGE_SIZE * MAX_PAGES);
-    int recordSize = 100;
-    byte[] b = new byte[recordSize];
-    FSDataInputStream stream = fs.open(PATH);
-
-    // Seek to a boundary around the middle of the 6th page
-    int seekPosition = 5 * PAGE_SIZE + 250;
-    stream.seek(seekPosition);
-
-    // Read a record's worth of bytes and verify results
-    int bytesRead = stream.read(b);
-    verifyReadRandomData(b, bytesRead, seekPosition, recordSize);
-
-    // Seek to another spot and read a record greater than a page
-    seekPosition = 10 * PAGE_SIZE + 250;
-    stream.seek(seekPosition);
-    recordSize = 1000;
-    b = new byte[recordSize];
-    bytesRead = stream.read(b);
-    verifyReadRandomData(b, bytesRead, seekPosition, recordSize);
-
-    // Read the last 100 bytes of the file
-    recordSize = 100;
-    seekPosition = PAGE_SIZE * MAX_PAGES - recordSize;
-    stream.seek(seekPosition);
-    b = new byte[recordSize];
-    bytesRead = stream.read(b);
-    verifyReadRandomData(b, bytesRead, seekPosition, recordSize);
-
-    // Read past the end of the file and we should get only partial data.
-    recordSize = 100;
-    seekPosition = PAGE_SIZE * MAX_PAGES - recordSize + 50;
-    stream.seek(seekPosition);
-    b = new byte[recordSize];
-    bytesRead = stream.read(b);
-    assertEquals(50, bytesRead);
-
-    // compare last 50 bytes written with those read
-    byte[] tail = Arrays.copyOfRange(randomData, seekPosition, randomData.length);
-    assertTrue(comparePrefix(tail, b, 50));
-  }
-
-  // Verify that reading a record of data after seeking gives the expected data.
-  private void verifyReadRandomData(byte[] b, int bytesRead, int seekPosition, int recordSize) {
-    byte[] originalRecordData =
-        Arrays.copyOfRange(randomData, seekPosition, seekPosition + recordSize + 1);
-    assertEquals(recordSize, bytesRead);
-    assertTrue(comparePrefix(originalRecordData, b, recordSize));
-  }
-
-  // Test many small flushed writes interspersed with periodic hflush calls.
-  // For manual testing, increase NUM_WRITES to a large number.
-  // The goal for a long-running manual test is to make sure that it finishes
-  // and the close() call does not time out. It also facilitates debugging into
-  // hflush/hsync.
-  @Test
-  public void testManySmallWritesWithHFlush() throws IOException {
-    writeAndReadOneFile(50, 100, 20);
-  }
-
-  /**
-   * Write a total of numWrites * recordLength data to a file, read it back,
-   * and check to make sure what was read is the same as what was written.
-   * The syncInterval is the number of writes after which to call hflush to
-   * force the data to storage.
-   */
-  private void writeAndReadOneFile(int numWrites, int recordLength, int syncInterval) throws IOException {
-    final int NUM_WRITES = numWrites;
-    final int RECORD_LENGTH = recordLength;
-    final int SYNC_INTERVAL = syncInterval;
-
-    // A lower bound on the minimum time we think it will take to do
-    // a write to Azure storage.
-    final long MINIMUM_EXPECTED_TIME = 20;
-    LOG.info("Writing " + NUM_WRITES * RECORD_LENGTH + " bytes to " + PATH.getName());
-    FSDataOutputStream output = fs.create(PATH);
-    int writesSinceHFlush = 0;
-    try {
-
-      // Do a flush and hflush to exercise case for empty write queue in PageBlobOutputStream,
-      // to test concurrent execution gates.
-      output.flush();
-      output.hflush();
-      for (int i = 0; i < NUM_WRITES; i++) {
-        output.write(randomData, i * RECORD_LENGTH, RECORD_LENGTH);
-        writesSinceHFlush++;
-        output.flush();
-        if ((i % SYNC_INTERVAL) == 0) {
-          output.hflush();
-          writesSinceHFlush = 0;
-        }
-      }
-    } finally {
-      long start = Time.monotonicNow();
-      output.close();
-      long end = Time.monotonicNow();
-      LOG.debug("close duration = " + (end - start) + " msec.");
-      if (writesSinceHFlush > 0) {
-        assertTrue(String.format(
-            "close duration with >= 1 pending write is %d, less than minimum expected of %d",
-            end - start, MINIMUM_EXPECTED_TIME),
-            end - start >= MINIMUM_EXPECTED_TIME);
-        }
-    }
-
-    // Read the data back and check it.
-    FSDataInputStream stream = fs.open(PATH);
-    int SIZE = NUM_WRITES * RECORD_LENGTH;
-    byte[] b = new byte[SIZE];
-    try {
-      stream.seek(0);
-      stream.read(b, 0, SIZE);
-      verifyReadRandomData(b, SIZE, 0, SIZE);
-    } finally {
-      stream.close();
-    }
-
-    // delete the file
-    fs.delete(PATH, false);
-  }
-
-  // Test writing to a large file repeatedly as a stress test.
-  // Set the repetitions to a larger number for manual testing
-  // for a longer stress run.
-  @Test
-  public void testLargeFileStress() throws IOException {
-    int numWrites = 32;
-    int recordSize = 1024 * 1024;
-    int syncInterval = 10;
-    int repetitions = 1;
-    for (int i = 0; i < repetitions; i++) {
-      writeAndReadOneFile(numWrites, recordSize, syncInterval);
-    }
-  }
-  
-  // Write to a file repeatedly to verify that it extends.
-  // The page blob file should start out at 128MB and finish at 256MB.
-  @Test(timeout=300000)
-  public void testFileSizeExtension() throws IOException {
-    final int writeSize = 1024 * 1024;
-    final int numWrites = 129;
-    final byte dataByte = 5;
-    byte[] data = new byte[writeSize];
-    Arrays.fill(data, dataByte);
-    FSDataOutputStream output = fs.create(PATH);
-    try {
-      for (int i = 0; i < numWrites; i++) {
-        output.write(data);
-        output.hflush();
-        LOG.debug("total writes = " + (i + 1));
-      }
-    } finally {
-      output.close();
-    }
-
-    // Show that we wrote more than the default page blob file size.
-    assertTrue(numWrites * writeSize > PageBlobOutputStream.PAGE_BLOB_MIN_SIZE);
-
-    // Verify we can list the new size. That will prove we expanded the file.
-    FileStatus[] status = fs.listStatus(PATH);
-    assertTrue(status[0].getLen() == numWrites * writeSize);
-    LOG.debug("Total bytes written to " + PATH + " = " + status[0].getLen());
-    fs.delete(PATH, false);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestShellDecryptionKeyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestShellDecryptionKeyProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestShellDecryptionKeyProvider.java
index 0bf33d8..0334c39 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestShellDecryptionKeyProvider.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestShellDecryptionKeyProvider.java
@@ -19,20 +19,23 @@
 package org.apache.hadoop.fs.azure;
 
 import static org.apache.hadoop.test.PlatformAssumptions.assumeWindows;
-import static org.junit.Assert.assertEquals;
 
 import java.io.File;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.junit.Assert;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-public class TestShellDecryptionKeyProvider {
-  public static final Log LOG = LogFactory
-      .getLog(TestShellDecryptionKeyProvider.class);
+/**
+ * Windows only tests of shell scripts to provide decryption keys.
+ */
+public class TestShellDecryptionKeyProvider
+    extends AbstractWasbTestWithTimeout {
+  public static final Logger LOG = LoggerFactory
+      .getLogger(TestShellDecryptionKeyProvider.class);
   private static File TEST_ROOT_DIR = new File(System.getProperty(
       "test.build.data", "/tmp"), "TestShellDecryptionKeyProvider");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbFsck.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbFsck.java
index 467424b..9d32fb2 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbFsck.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbFsck.java
@@ -18,10 +18,6 @@
 
 package org.apache.hadoop.fs.azure;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -32,7 +28,10 @@ import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
 
-public class TestWasbFsck {
+/**
+ * Tests which look at fsck recovery.
+ */
+public class TestWasbFsck extends AbstractWasbTestWithTimeout {
   private AzureBlobStorageTestAccount testAccount;
   private FileSystem fs;
   private InMemoryBlockBlobStore backingStore;


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


[05/50] [abbrv] hadoop git commit: HADOOP-14864. FSDataInputStream#unbuffer UOE should include stream class name. Contributed by Bharat Viswanadham.

Posted by ae...@apache.org.
HADOOP-14864. FSDataInputStream#unbuffer UOE should include stream class name. Contributed by Bharat Viswanadham.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7ee02d10
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7ee02d10
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7ee02d10

Branch: refs/heads/HDFS-7240
Commit: 7ee02d1065268511ec10a8b8852d0d6327419b9c
Parents: 09b476e
Author: John Zhuge <jz...@cloudera.com>
Authored: Wed Sep 13 20:55:06 2017 -0700
Committer: John Zhuge <jz...@apache.org>
Committed: Thu Sep 14 11:26:49 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/FSDataInputStream.java |  4 ++--
 .../java/org/apache/hadoop/fs/TestUnbuffer.java | 21 ++++++++++++++++++++
 2 files changed, 23 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ee02d10/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java
index 640db59..a80279d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java
@@ -230,8 +230,8 @@ public class FSDataInputStream extends DataInputStream
     try {
       ((CanUnbuffer)in).unbuffer();
     } catch (ClassCastException e) {
-      throw new UnsupportedOperationException("this stream does not " +
-          "support unbuffering.");
+      throw new UnsupportedOperationException("this stream " +
+          in.getClass().getName() + " does not " + "support unbuffering.");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ee02d10/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java
index e25a754..b112e30 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestUnbuffer.java
@@ -27,12 +27,18 @@ import org.apache.hadoop.hdfs.PeerCache;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.mockito.Mockito;
 
 public class TestUnbuffer {
   private static final Log LOG =
       LogFactory.getLog(TestUnbuffer.class.getName());
 
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
   /**
    * Test that calling Unbuffer closes sockets.
    */
@@ -123,4 +129,19 @@ public class TestUnbuffer {
       }
     }
   }
+
+  /**
+   * Test unbuffer method which throws an Exception with class name included.
+   */
+  @Test
+  public void testUnbufferException() {
+    FSInputStream in = Mockito.mock(FSInputStream.class);
+    FSDataInputStream fs = new FSDataInputStream(in);
+
+    exception.expect(UnsupportedOperationException.class);
+    exception.expectMessage("this stream " + in.getClass().getName()
+        + " does not support unbuffering");
+
+    fs.unbuffer();
+  }
 }


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


[42/50] [abbrv] hadoop git commit: HDFS-12460. Make addErasureCodingPolicy an idempotent operation. Contributed by Sammi Chen

Posted by ae...@apache.org.
HDFS-12460. Make addErasureCodingPolicy an idempotent operation. Contributed by Sammi Chen


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0f9af246
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0f9af246
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0f9af246

Branch: refs/heads/HDFS-7240
Commit: 0f9af246e89e4ad3c4d7ff2c1d7ec9b397494a03
Parents: e81596d
Author: Kai Zheng <ka...@intel.com>
Authored: Mon Sep 18 18:07:12 2017 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Mon Sep 18 18:07:12 2017 +0800

----------------------------------------------------------------------
 .../hdfs/server/namenode/ErasureCodingPolicyManager.java      | 7 ++++---
 .../org/apache/hadoop/hdfs/TestErasureCodingPolicies.java     | 2 +-
 .../hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java   | 4 ++--
 .../hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java  | 4 ++--
 4 files changed, 9 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f9af246/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
index 3a46c30..90699b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
@@ -242,14 +242,15 @@ public final class ErasureCodingPolicyManager {
         policy.getSchema(), policy.getCellSize());
     for (ErasureCodingPolicy p : getPolicies()) {
       if (p.getName().equals(assignedNewName)) {
-        throw new HadoopIllegalArgumentException("The policy name " +
-            assignedNewName + " already exists");
+        LOG.info("The policy name " + assignedNewName + " already exists");
+        return p;
       }
       if (p.getSchema().equals(policy.getSchema()) &&
           p.getCellSize() == policy.getCellSize()) {
-        throw new HadoopIllegalArgumentException("A policy with same schema "
+        LOG.info("A policy with same schema "
             + policy.getSchema().toString() + " and cell size "
             + p.getCellSize() + " already exists");
+        return p;
       }
     }
     policy.setName(assignedNewName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f9af246/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
index 19277c4..4f2040b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
@@ -718,7 +718,7 @@ public class TestErasureCodingPolicies {
     policyArray  = new ErasureCodingPolicy[]{policy0};
     responses = fs.addErasureCodingPolicies(policyArray);
     assertEquals(1, responses.length);
-    assertFalse(responses[0].isSucceed());
+    assertTrue(responses[0].isSucceed());
 
     // Test add policy successfully
     newPolicy =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f9af246/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
index d217813..42ff698 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
@@ -436,7 +436,7 @@ public class TestNamenodeRetryCache {
 
     LightWeightCache<CacheEntry, CacheEntry> cacheSet = 
         (LightWeightCache<CacheEntry, CacheEntry>) namesystem.getRetryCache().getCacheSet();
-    assertEquals("Retry cache size is wrong", 26, cacheSet.size());
+    assertEquals("Retry cache size is wrong", 34, cacheSet.size());
     
     Map<CacheEntry, CacheEntry> oldEntries = 
         new HashMap<CacheEntry, CacheEntry>();
@@ -455,7 +455,7 @@ public class TestNamenodeRetryCache {
     assertTrue(namesystem.hasRetryCache());
     cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) namesystem
         .getRetryCache().getCacheSet();
-    assertEquals("Retry cache size is wrong", 26, cacheSet.size());
+    assertEquals("Retry cache size is wrong", 34, cacheSet.size());
     iter = cacheSet.iterator();
     while (iter.hasNext()) {
       CacheEntry entry = iter.next();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f9af246/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
index b40006b..1d114d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
@@ -166,7 +166,7 @@ public class TestRetryCacheWithHA {
     FSNamesystem fsn0 = cluster.getNamesystem(0);
     LightWeightCache<CacheEntry, CacheEntry> cacheSet = 
         (LightWeightCache<CacheEntry, CacheEntry>) fsn0.getRetryCache().getCacheSet();
-    assertEquals("Retry cache size is wrong", 26, cacheSet.size());
+    assertEquals("Retry cache size is wrong", 34, cacheSet.size());
     
     Map<CacheEntry, CacheEntry> oldEntries = 
         new HashMap<CacheEntry, CacheEntry>();
@@ -187,7 +187,7 @@ public class TestRetryCacheWithHA {
     FSNamesystem fsn1 = cluster.getNamesystem(1);
     cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) fsn1
         .getRetryCache().getCacheSet();
-    assertEquals("Retry cache size is wrong", 26, cacheSet.size());
+    assertEquals("Retry cache size is wrong", 34, cacheSet.size());
     iter = cacheSet.iterator();
     while (iter.hasNext()) {
       CacheEntry entry = iter.next();


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


[48/50] [abbrv] hadoop git commit: YARN-6570. No logs were found for running application, running container. Contributed by Junping Du

Posted by ae...@apache.org.
YARN-6570. No logs were found for running application, running
container. Contributed by Junping Du


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7c732924
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7c732924
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7c732924

Branch: refs/heads/HDFS-7240
Commit: 7c732924a889cd280e972882619a1827877fbafa
Parents: 29dd551
Author: Xuan <xg...@apache.org>
Authored: Mon Sep 18 14:04:05 2017 -0700
Committer: Xuan <xg...@apache.org>
Committed: Mon Sep 18 14:04:05 2017 -0700

----------------------------------------------------------------------
 .../nodemanager/containermanager/container/ContainerImpl.java     | 1 +
 .../org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java  | 3 ++-
 .../nodemanager/containermanager/container/TestContainer.java     | 3 +++
 3 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c732924/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index df107a7..836e70e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -616,6 +616,7 @@ public class ContainerImpl implements Container {
   public org.apache.hadoop.yarn.api.records.ContainerState getCurrentState() {
     switch (stateMachine.getCurrentState()) {
     case NEW:
+      return org.apache.hadoop.yarn.api.records.ContainerState.NEW;
     case LOCALIZING:
     case LOCALIZATION_FAILED:
     case SCHEDULED:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c732924/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
index 8e4522b..9e59449 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
@@ -159,7 +159,8 @@ public class TestEventFlow {
     containerManager.startContainers(allRequests);
 
     BaseContainerManagerTest.waitForContainerState(containerManager, cID,
-        Arrays.asList(ContainerState.RUNNING, ContainerState.SCHEDULED), 20);
+        Arrays.asList(ContainerState.RUNNING, ContainerState.SCHEDULED,
+            ContainerState.NEW), 20);
 
     List<ContainerId> containerIds = new ArrayList<ContainerId>();
     containerIds.add(cID);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c732924/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
index 64e6cf0..b44b500 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
@@ -122,6 +122,8 @@ public class TestContainer {
     try {
       wc = new WrappedContainer(7, 314159265358979L, 4344, "yak");
       assertEquals(ContainerState.NEW, wc.c.getContainerState());
+      ContainerImpl container = (ContainerImpl)wc.c;
+      assertEquals(org.apache.hadoop.yarn.api.records.ContainerState.NEW, container.getCurrentState());
       wc.initContainer();
 
       // Verify request for public/private resources to localizer
@@ -131,6 +133,7 @@ public class TestContainer {
               LocalResourceVisibility.APPLICATION));
       verify(wc.localizerBus).handle(argThat(matchesReq));
       assertEquals(ContainerState.LOCALIZING, wc.c.getContainerState());
+      assertEquals(org.apache.hadoop.yarn.api.records.ContainerState.SCHEDULED, container.getCurrentState());
     }
     finally {
       if (wc != null) {


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


[34/50] [abbrv] hadoop git commit: HDFS-12323. NameNode terminates after full GC thinking QJM unresponsive if full GC is much longer than timeout. Contributed by Erik Krogen.

Posted by ae...@apache.org.
HDFS-12323. NameNode terminates after full GC thinking QJM unresponsive if full GC is much longer than timeout. Contributed by Erik Krogen.

Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/90894c72
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/90894c72
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/90894c72

Branch: refs/heads/HDFS-7240
Commit: 90894c7262df0243e795b675f3ac9f7b322ccd11
Parents: b9b607d
Author: Erik Krogen <ek...@linkedin.com>
Authored: Thu Sep 14 15:53:33 2017 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Fri Sep 15 13:56:27 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/qjournal/client/QuorumCall.java | 65 ++++++++++++++++----
 .../hdfs/qjournal/client/TestQuorumCall.java    | 31 +++++++++-
 2 files changed, 82 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/90894c72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java
index dc32318..dee74e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java
@@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.StopWatch;
-import org.apache.hadoop.util.Time;
+import org.apache.hadoop.util.Timer;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
@@ -35,6 +35,7 @@ import com.google.common.util.concurrent.ListenableFuture;
 import com.google.protobuf.Message;
 import com.google.protobuf.TextFormat;
 
+
 /**
  * Represents a set of calls for which a quorum of results is needed.
  * @param <KEY> a key used to identify each of the outgoing calls
@@ -60,11 +61,12 @@ class QuorumCall<KEY, RESULT> {
    * fraction of the configured timeout for any call.
    */
   private static final float WAIT_PROGRESS_WARN_THRESHOLD = 0.7f;
-  private final StopWatch quorumStopWatch = new StopWatch();
+  private final StopWatch quorumStopWatch;
+  private final Timer timer;
   
   static <KEY, RESULT> QuorumCall<KEY, RESULT> create(
-      Map<KEY, ? extends ListenableFuture<RESULT>> calls) {
-    final QuorumCall<KEY, RESULT> qr = new QuorumCall<KEY, RESULT>();
+      Map<KEY, ? extends ListenableFuture<RESULT>> calls, Timer timer) {
+    final QuorumCall<KEY, RESULT> qr = new QuorumCall<KEY, RESULT>(timer);
     for (final Entry<KEY, ? extends ListenableFuture<RESULT>> e : calls.entrySet()) {
       Preconditions.checkArgument(e.getValue() != null,
           "null future for key: " + e.getKey());
@@ -82,18 +84,53 @@ class QuorumCall<KEY, RESULT> {
     }
     return qr;
   }
-  
+
+  static <KEY, RESULT> QuorumCall<KEY, RESULT> create(
+      Map<KEY, ? extends ListenableFuture<RESULT>> calls) {
+    return create(calls, new Timer());
+  }
+
+  /**
+   * Not intended for outside use.
+   */
   private QuorumCall() {
+    this(new Timer());
+  }
+
+  private QuorumCall(Timer timer) {
     // Only instantiated from factory method above
+    this.timer = timer;
+    this.quorumStopWatch = new StopWatch(timer);
   }
 
+  /**
+   * Used in conjunction with {@link #getQuorumTimeoutIncreaseMillis(long, int)}
+   * to check for pauses.
+   */
   private void restartQuorumStopWatch() {
     quorumStopWatch.reset().start();
   }
 
-  private boolean shouldIncreaseQuorumTimeout(long offset, int millis) {
+  /**
+   * Check for a pause (e.g. GC) since the last time
+   * {@link #restartQuorumStopWatch()} was called. If detected, return the
+   * length of the pause; else, -1.
+   * @param offset Offset the elapsed time by this amount; use if some amount
+   *               of pause was expected
+   * @param millis Total length of timeout in milliseconds
+   * @return Length of pause, if detected, else -1
+   */
+  private long getQuorumTimeoutIncreaseMillis(long offset, int millis) {
     long elapsed = quorumStopWatch.now(TimeUnit.MILLISECONDS);
-    return elapsed + offset > (millis * WAIT_PROGRESS_INFO_THRESHOLD);
+    long pauseTime = elapsed + offset;
+    if (pauseTime > (millis * WAIT_PROGRESS_INFO_THRESHOLD)) {
+      QuorumJournalManager.LOG.info("Pause detected while waiting for " +
+          "QuorumCall response; increasing timeout threshold by pause time " +
+          "of " + pauseTime + " ms.");
+      return pauseTime;
+    } else {
+      return -1;
+    }
   }
 
   
@@ -119,7 +156,7 @@ class QuorumCall<KEY, RESULT> {
       int minResponses, int minSuccesses, int maxExceptions,
       int millis, String operationName)
       throws InterruptedException, TimeoutException {
-    long st = Time.monotonicNow();
+    long st = timer.monotonicNow();
     long nextLogTime = st + (long)(millis * WAIT_PROGRESS_INFO_THRESHOLD);
     long et = st + millis;
     while (true) {
@@ -128,7 +165,7 @@ class QuorumCall<KEY, RESULT> {
       if (minResponses > 0 && countResponses() >= minResponses) return;
       if (minSuccesses > 0 && countSuccesses() >= minSuccesses) return;
       if (maxExceptions >= 0 && countExceptions() > maxExceptions) return;
-      long now = Time.monotonicNow();
+      long now = timer.monotonicNow();
       
       if (now > nextLogTime) {
         long waited = now - st;
@@ -154,8 +191,9 @@ class QuorumCall<KEY, RESULT> {
       long rem = et - now;
       if (rem <= 0) {
         // Increase timeout if a full GC occurred after restarting stopWatch
-        if (shouldIncreaseQuorumTimeout(0, millis)) {
-          et = et + millis;
+        long timeoutIncrease = getQuorumTimeoutIncreaseMillis(0, millis);
+        if (timeoutIncrease > 0) {
+          et += timeoutIncrease;
         } else {
           throw new TimeoutException();
         }
@@ -165,8 +203,9 @@ class QuorumCall<KEY, RESULT> {
       rem = Math.max(rem, 1);
       wait(rem);
       // Increase timeout if a full GC occurred after restarting stopWatch
-      if (shouldIncreaseQuorumTimeout(-rem, millis)) {
-        et = et + millis;
+      long timeoutIncrease = getQuorumTimeoutIncreaseMillis(-rem, millis);
+      if (timeoutIncrease > 0) {
+        et += timeoutIncrease;
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/90894c72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumCall.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumCall.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumCall.java
index 506497e..97cf2f3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumCall.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumCall.java
@@ -23,7 +23,7 @@ import java.util.Map;
 import java.util.TreeMap;
 import java.util.concurrent.TimeoutException;
 
-import org.apache.hadoop.hdfs.qjournal.client.QuorumCall;
+import org.apache.hadoop.util.FakeTimer;
 import org.junit.Test;
 
 import com.google.common.base.Joiner;
@@ -83,4 +83,33 @@ public class TestQuorumCall {
     }
   }
 
+  @Test(timeout=10000)
+  public void testQuorumSucceedsWithLongPause() throws Exception {
+    final Map<String, SettableFuture<String>> futures = ImmutableMap.of(
+        "f1", SettableFuture.<String>create());
+
+    FakeTimer timer = new FakeTimer() {
+      private int callCount = 0;
+      @Override
+      public long monotonicNowNanos() {
+        callCount++;
+        if (callCount == 1) {
+          long old = super.monotonicNowNanos();
+          advance(1000000);
+          return old;
+        } else if (callCount == 10) {
+          futures.get("f1").set("first future");
+          return super.monotonicNowNanos();
+        } else {
+          return super.monotonicNowNanos();
+        }
+      }
+    };
+
+    QuorumCall<String, String> q = QuorumCall.create(futures, timer);
+    assertEquals(0, q.countResponses());
+
+    q.waitFor(1, 0, 0, 3000, "test"); // wait for 1 response
+  }
+
 }


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


[07/50] [abbrv] hadoop git commit: HADOOP-14738 Remove S3N and obsolete bits of S3A; rework docs. Contributed by Steve Loughran.

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
index fe67d69..7957122 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
@@ -107,7 +107,6 @@ It is recommended that you leave the default setting here:
     <name>fs.s3a.metadatastore.authoritative</name>
     <value>false</value>
 </property>
-
 ```
 
 Setting this to `true` is currently an experimental feature.  When true, the
@@ -510,7 +509,6 @@ log4j.logger.com.amazonaws.services.dynamodbv2.AmazonDynamoDB
 # Log all HTTP requests made; includes S3 interaction. This may
 # include sensitive information such as account IDs in HTTP headers.
 log4j.logger.com.amazonaws.request=DEBUG
-
 ```
 
 If all else fails, S3Guard is designed to allow for easy recovery by deleting
@@ -538,7 +536,7 @@ S3Guard tables are created with a version marker, an entry with the primary
 key and child entry of `../VERSION`; the use of a relative path guarantees
 that it will not be resolved.
 
-#### Versioning policy.
+*Versioning policy*
 
 1. The version number of an S3Guard table will only be incremented when
 an incompatible change is made to the table structure —that is, the structure
@@ -557,7 +555,7 @@ in an incompatible manner. The version marker in tables exists to support
 such an option if it ever becomes necessary, by ensuring that all S3Guard
 client can recognise any version mismatch.
 
-### Security
+## Security
 
 All users of the DynamoDB table must have write access to it. This
 effectively means they must have write access to the entire object store.
@@ -569,9 +567,9 @@ are only made after successful file creation, deletion and rename, the
 store is *unlikely* to get out of sync, it is still something which
 merits more testing before it could be considered reliable.
 
-### Troubleshooting
+## Troubleshooting
 
-#### Error: `S3Guard table lacks version marker.`
+### Error: `S3Guard table lacks version marker.`
 
 The table which was intended to be used as a S3guard metadata store
 does not have any version marker indicating that it is a S3Guard table.
@@ -581,7 +579,7 @@ It may be that this is not a S3Guard table.
 * Make sure that this is the correct table name.
 * Delete the table, so it can be rebuilt.
 
-#### Error: `Database table is from an incompatible S3Guard version`
+### Error: `Database table is from an incompatible S3Guard version`
 
 This indicates that the version of S3Guard which created (or possibly updated)
 the database table is from a different version that that expected by the S3A
@@ -596,7 +594,7 @@ bucket. Upgrade the application/library.
 If the expected version is higher than the actual version, then the table
 itself will need upgrading.
 
-#### Error `"DynamoDB table TABLE does not exist in region REGION; auto-creation is turned off"`
+### Error `"DynamoDB table TABLE does not exist in region REGION; auto-creation is turned off"`
 
 S3Guard could not find the DynamoDB table for the Metadata Store,
 and it was not configured to create it. Either the table was missing,
@@ -608,3 +606,8 @@ or the configuration is preventing S3Guard from finding the table.
 1. If the region is not set, verify that the table exists in the same
 region as the bucket being used.
 1. Create the table if necessary.
+
+
+## Other Topis
+
+For details on how to test S3Guard, see [Testing S3Guard](./testing.html#s3guard)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3n.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3n.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3n.md
new file mode 100644
index 0000000..9b59ad1
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3n.md
@@ -0,0 +1,52 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# The S3N Client
+
+<!-- MACRO{toc|fromDepth=0|toDepth=5} -->
+
+S3N was a Hadoop filesystem client which can read or write data stored
+in Amazon S3. It uses URLs with the schema `s3n://`.
+
+- - -
+
+**Hadoop's S3N client for Amazon S3 has been superceded by
+the S3A connector**
+
+**Please upgrade to S3A for a supported, higher-performance S3 Client**
+
+- - -
+
+
+## <a name="migrating"></a> How to migrate to to the S3A client
+
+1. Keep the `hadoop-aws` JAR on your classpath.
+
+1. Add the `aws-java-sdk-bundle.jar` JAR which Hadoop ships
+with to your classpath.
+
+1. Change the authentication keys
+
+    | old key | new key |
+    |---------|---------|
+    | `fs.s3n.awsAccessKeyId` | `fs.s3a.access.key` |
+    | `fs.s3n.awsSecretAccessKey` | `fs.s3a.secret.key` |
+
+    Do make sure the property names are correct. For S3A, they are
+    `fs.s3a.access.key` and `fs.s3a.secret.key` —you cannot just copy the S3N
+    properties and replace `s3n` with `s3a`.
+
+1. Replace URLs which began with `s3n://` with `s3a://`
+
+1. You may now remove the `jets3t` JAR, as it is no longer needed.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
index 19d322d..cf7a2e4 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
@@ -12,9 +12,9 @@
   limitations under the License. See accompanying LICENSE file.
 -->
 
-# Testing the S3 filesystem clients
+# Testing the S3A filesystem client and its features, including S3Guard
 
-<!-- MACRO{toc|fromDepth=0|toDepth=5} -->
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
 
 This module includes both unit tests, which can run in isolation without
 connecting to the S3 service, and integration tests, which require a working
@@ -26,7 +26,7 @@ Due to eventual consistency, integration tests may fail without reason.
 Transient failures, which no longer occur upon rerunning the test, should thus
 be ignored.
 
-## Policy for submitting patches which affect the `hadoop-aws` module.
+## <a name="policy"></a> Policy for submitting patches which affect the `hadoop-aws` module.
 
 The Apache Jenkins infrastucture does not run any S3 integration tests,
 due to the need to keep credentials secure.
@@ -74,7 +74,7 @@ in the production code, that could be a sign of a problem which may surface
 over long-haul connections. Please help us identify and fix these problems
 &mdash; especially as you are the one best placed to verify the fixes work.
 
-## Setting up the tests
+## <a name="setting-up"></a> Setting up the tests
 
 To integration test the S3* filesystem clients, you need to provide
 `auth-keys.xml` which passes in authentication details to the test runner.
@@ -101,16 +101,11 @@ The XML file must contain all the ID/key information needed to connect
 each of the filesystem clients to the object stores, and a URL for
 each filesystem for its testing.
 
-1. `test.fs.s3n.name` : the URL of the bucket for S3n tests
 1. `test.fs.s3a.name` : the URL of the bucket for S3a tests
-1. `fs.contract.test.fs.s3n` : the URL of the bucket for S3n filesystem contract tests
 1. `fs.contract.test.fs.s3a` : the URL of the bucket for S3a filesystem contract tests
 
-*Note* that running s3a and s3n tests in parallel mode, against the same bucket
-is unreliable.  We recommend using separate buckets or testing one connector
-at a time.
 
-The contents of each bucket will be destroyed during the test process:
+The contents of the bucket will be destroyed during the test process:
 do not use the bucket for any purpose other than testing. Furthermore, for
 s3a, all in-progress multi-part uploads to the bucket will be aborted at the
 start of a test (by forcing `fs.s3a.multipart.purge=true`) to clean up the
@@ -122,26 +117,6 @@ Example:
 <configuration>
 
   <property>
-    <name>test.fs.s3n.name</name>
-    <value>s3n://test-aws-s3n/</value>
-  </property>
-
-  <property>
-    <name>fs.contract.test.fs.s3n</name>
-    <value>${test.fs.s3n.name}</value>
-  </property>
-
-  <property>
-    <name>fs.s3n.awsAccessKeyId</name>
-    <value>DONOTPCOMMITTHISKEYTOSCM</value>
-  </property>
-
-  <property>
-    <name>fs.s3n.awsSecretAccessKey</name>
-    <value>DONOTEVERSHARETHISSECRETKEY!</value>
-  </property>
-
-  <property>
     <name>test.fs.s3a.name</name>
     <value>s3a://test-aws-s3a/</value>
   </property>
@@ -172,7 +147,7 @@ Example:
 </configuration>
 ```
 
-### Configuring S3a Encryption
+### <a name="encryption"></a> Configuring S3a Encryption
 
 For S3a encryption tests to run correctly, the
 `fs.s3a.server-side-encryption.key` must be configured in the s3a contract xml
@@ -192,7 +167,7 @@ You can also force all the tests to run with a specific SSE encryption method
 by configuring the property `fs.s3a.server-side-encryption-algorithm` in the s3a
 contract file.
 
-## Running the Tests
+## <a name="running"></a> Running the Tests
 
 After completing the configuration, execute the test run through Maven.
 
@@ -251,7 +226,7 @@ combination with `test` or `it.test`.  If you know that you are specifying only
 tests that can run safely in parallel, then it will work.  For wide patterns,
 like `ITestS3A*` shown above, it may cause unpredictable test failures.
 
-### Testing against different regions
+### <a name="regions"></a> Testing against different regions
 
 S3A can connect to different regions —the tests support this. Simply
 define the target region in `auth-keys.xml`.
@@ -265,7 +240,7 @@ define the target region in `auth-keys.xml`.
 This is used for all tests expect for scale tests using a Public CSV.gz file
 (see below)
 
-### CSV Data source Tests
+### <a name="csv"></a> CSV Data Tests
 
 The `TestS3AInputStreamPerformance` tests require read access to a multi-MB
 text file. The default file for these tests is one published by amazon,
@@ -303,7 +278,7 @@ For the default test dataset, hosted in the `landsat-pds` bucket, this is:
 </property>
 ```
 
-### Viewing Integration Test Reports
+## <a name="reporting"></a> Viewing Integration Test Reports
 
 
 Integration test results and logs are stored in `target/failsafe-reports/`.
@@ -313,7 +288,7 @@ plugin:
 ```bash
 mvn surefire-report:failsafe-report-only
 ```
-### Scale Tests
+## <a name="scale"></a> Scale Tests
 
 There are a set of tests designed to measure the scalability and performance
 at scale of the S3A tests, *Scale Tests*. Tests include: creating
@@ -325,7 +300,7 @@ By their very nature they are slow. And, as their execution time is often
 limited by bandwidth between the computer running the tests and the S3 endpoint,
 parallel execution does not speed these tests up.
 
-#### Enabling the Scale Tests
+### <a name="enabling-scale"></a> Enabling the Scale Tests
 
 The tests are enabled if the `scale` property is set in the maven build
 this can be done regardless of whether or not the parallel test profile
@@ -342,7 +317,7 @@ sequentially; those which are slow due to HTTPS setup costs or server-side
 actionsare included in the set of parallelized tests.
 
 
-#### Maven build tuning options
+### <a name="tuning_scale"></a> Tuning scale optins from Maven
 
 
 Some of the tests can be tuned from the maven build or from the
@@ -373,7 +348,7 @@ Only a few properties can be set this way; more will be added.
 The file and partition sizes are numeric values with a k/m/g/t/p suffix depending
 on the desired size. For example: 128M, 128m, 2G, 2G, 4T or even 1P.
 
-#### Scale test configuration options
+### <a name="scale-config"></a> Scale test configuration options
 
 Some scale tests perform multiple operations (such as creating many directories).
 
@@ -418,7 +393,7 @@ smaller to achieve faster test runs.
 
 S3A specific scale test properties are
 
-##### `fs.s3a.scale.test.huge.filesize`: size in MB for "Huge file tests".
+*`fs.s3a.scale.test.huge.filesize`: size in MB for "Huge file tests".*
 
 The Huge File tests validate S3A's ability to handle large files —the property
 `fs.s3a.scale.test.huge.filesize` declares the file size to use.
@@ -452,13 +427,11 @@ Otherwise, set a large timeout in `fs.s3a.scale.test.timeout`
 </property>
 ```
 
-
 The tests are executed in an order to only clean up created files after
 the end of all the tests. If the tests are interrupted, the test data will remain.
 
 
-
-## Testing against non AWS S3 endpoints.
+## <a name="alternate_s3"></a> Testing against non AWS S3 endpoints.
 
 The S3A filesystem is designed to work with storage endpoints which implement
 the S3 protocols to the extent that the amazon S3 SDK is capable of talking
@@ -527,7 +500,7 @@ An alternate endpoint may be defined in `test.fs.s3a.sts.endpoint`.
 The default is ""; meaning "use the amazon default value".
 
 
-## Debugging Test failures
+## <a name="debugging"></a> Debugging Test failures
 
 Logging at debug level is the standard way to provide more diagnostics output;
 after setting this rerun the tests
@@ -550,7 +523,7 @@ setting the `fs.s3a.user.agent.prefix` to a unique prefix for a specific
 test run, which will enable the specific log entries to be more easily
 located.
 
-## Adding new tests
+## <a name="new_tests"></a> Adding new tests
 
 New tests are always welcome. Bear in mind that we need to keep costs
 and test time down, which is done by
@@ -593,7 +566,7 @@ fail with meaningful diagnostics, so any new problems can be easily debugged
 from test logs.
 
 
-### Requirements of new Tests
+## <a name="requirements"></a> Requirements of new Tests
 
 
 This is what we expect from new tests; they're an extension of the normal
@@ -602,7 +575,7 @@ use requires the presence of secret credentials, where tests may be slow,
 and where finding out why something failed from nothing but the test output
 is critical.
 
-#### Subclasses Existing Shared Base Classes
+### Subclasses Existing Shared Base Classes
 
 Extend `AbstractS3ATestBase` or `AbstractSTestS3AHugeFiles` unless justifiable.
 These set things up for testing against the object stores, provide good threadnames,
@@ -619,12 +592,12 @@ defined in `fs.s3a.contract.test`
 Having shared base classes may help reduce future maintenance too. Please
 use them/
 
-#### Secure
+### Secure
 
 Don't ever log credentials. The credential tests go out of their way to
 not provide meaningful logs or assertion messages precisely to avoid this.
 
-#### Efficient of Time and Money
+### Efficient of Time and Money
 
 This means efficient in test setup/teardown, and, ideally, making use of
 existing public datasets to save setup time and tester cost.
@@ -650,7 +623,7 @@ against other regions, or with third party S3 implementations. Thus the
 URL can be overridden for testing elsewhere.
 
 
-#### Works With Other S3 Endpoints
+### Works With Other S3 Endpoints
 
 Don't assume AWS S3 US-East only, do allow for working with external S3 implementations.
 Those may be behind the latest S3 API features, not support encryption, session
@@ -678,7 +651,7 @@ adds some newlines so as to be easier to spot.
 1. Use `ContractTestUtils.NanoTimer` to measure the duration of operations,
 and log the output.
 
-#### Fails Meaningfully
+### Fails Meaningfully
 
 The `ContractTestUtils` class contains a whole set of assertions for making
 statements about the expected state of a filesystem, e.g.
@@ -705,7 +678,7 @@ get called.
 We really appreciate this &mdash; you will too.
 
 
-## Tips
+## <a name="tips"></a> Tips
 
 ### How to keep your credentials really safe
 
@@ -725,7 +698,7 @@ using an absolute XInclude reference to it.
 </configuration>
 ```
 
-# Failure Injection
+#  <a name="failure-injection"></a>Failure Injection
 
 **Warning do not enable any type of failure injection in production.  The
 following settings are for testing only.**
@@ -858,7 +831,10 @@ The inconsistent client is shipped in the `hadoop-aws` JAR, so it can
 be used in applications which work with S3 to see how they handle
 inconsistent directory listings.
 
-## Testing S3Guard
+##<a name="s3guard"></a> Testing S3Guard
+
+[S3Guard](./s3guard.html) is an extension to S3A which adds consistent metadata
+listings to the S3A client. As it is part of S3A, it also needs to be tested.
 
 The basic strategy for testing S3Guard correctness consists of:
 
@@ -934,13 +910,6 @@ If the `s3guard` profile *is* set,
   overwrite any previously set in the configuration files.
 1. DynamoDB will be configured to create any missing tables.
 
-### Warning About Concurrent Tests
-
-You must not run S3A and S3N tests in parallel on the same bucket.  This is
-especially true when S3Guard is enabled.  S3Guard requires that all clients
-that are modifying the bucket have S3Guard enabled, so having S3N
-integration tests running in parallel with S3A tests will cause strange
-failures.
 
 ### Scale Testing MetadataStore Directly
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
index d79720e..619ffc1 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
@@ -12,27 +12,628 @@
   limitations under the License. See accompanying LICENSE file.
 -->
 
-# Troubleshooting S3A
+# Troubleshooting
 
-Here are some lower level details and hints on troubleshooting and tuning
-the S3A client.
+<!-- MACRO{toc|fromDepth=0|toDepth=5} -->
 
-## Logging at lower levels
+##<a name="introduction"></a>  Introduction
 
-The AWS SDK and the Apache HTTP components can be configured to log at
-more detail, as can S3A itself.
+Common problems working with S3 are
+
+1. Classpath setup
+1. Authentication
+1. S3 Inconsistency side-effects
+
+Classpath is usually the first problem. For the S3x filesystem clients,
+you need the Hadoop-specific filesystem clients, third party S3 client libraries
+compatible with the Hadoop code, and any dependent libraries compatible with
+Hadoop and the specific JVM.
+
+The classpath must be set up for the process talking to S3: if this is code
+running in the Hadoop cluster, the JARs must be on that classpath. That
+includes `distcp` and the `hadoop fs` command.
+
+<!-- MACRO{toc|fromDepth=0|toDepth=2} -->
+
+## <a name="classpath"></a> Classpath Setup
+
+Note that for security reasons, the S3A client does not provide much detail
+on the authentication process (i.e. the secrets used to authenticate).
+
+### `ClassNotFoundException: org.apache.hadoop.fs.s3a.S3AFileSystem`
+
+These is Hadoop filesytem client classes, found in the `hadoop-aws` JAR.
+An exception reporting this class as missing means that this JAR is not on
+the classpath.
+
+### `ClassNotFoundException: com.amazonaws.services.s3.AmazonS3Client`
+
+(or other `com.amazonaws` class.)
+
+This means that the `aws-java-sdk-bundle.jar` JAR is not on the classpath:
+add it.
+
+### Missing method in `com.amazonaws` class
+
+This can be triggered by incompatibilities between the AWS SDK on the classpath
+and the version which Hadoop was compiled with.
+
+The AWS SDK JARs change their signature enough between releases that the only
+way to safely update the AWS SDK version is to recompile Hadoop against the later
+version.
+
+The sole fix is to use the same version of the AWS SDK with which Hadoop
+was built.
+
+
+## <a name="authentication"></a> Authentication Failure
+
+If Hadoop cannot authenticate with the S3 service endpoint,
+the client retries a number of times before eventually failing.
+When it finally gives up, it will report a message about signature mismatch:
+
+```
+com.amazonaws.services.s3.model.AmazonS3Exception:
+ The request signature we calculated does not match the signature you provided.
+ Check your key and signing method.
+  (Service: Amazon S3; Status Code: 403; Error Code: SignatureDoesNotMatch,
+```
+
+The likely cause is that you either have the wrong credentials or somehow
+the credentials were not readable on the host attempting to read or write
+the S3 Bucket.
+
+Enabling debug logging for the package `org.apache.hadoop.fs.s3a`
+can help provide more information.
+
+The most common cause is that you have the wrong credentials for any of the current
+authentication mechanism(s) —or somehow
+the credentials were not readable on the host attempting to read or write
+the S3 Bucket. However, there are a couple of system configuration problems
+(JVM version, system clock) which also need to be checked.
+
+Most common: there's an error in the configuration properties.
+
+1. Make sure that the name of the bucket is the correct one.
+That is: check the URL.
+
+1. If using a private S3 server, make sure endpoint in `fs.s3a.endpoint` has
+been set to this server -and that the client is not accidentally trying to
+authenticate with the public Amazon S3 service.
+
+1. Make sure the property names are correct. For S3A, they are
+`fs.s3a.access.key` and `fs.s3a.secret.key` —you cannot just copy the S3N
+properties and replace `s3n` with `s3a`.
+
+1. Make sure the properties are visible to the process attempting to
+talk to the object store. Placing them in `core-site.xml` is the standard
+mechanism.
+
+1. If using session authentication, the session may have expired.
+Generate a new session token and secret.
+
+1. If using environement variable-based authentication, make sure that the
+relevant variables are set in the environment in which the process is running.
+
+The standard first step is: try to use the AWS command line tools with the same
+credentials, through a command such as:
+
+    hadoop fs -ls s3a://my-bucket/
+
+Note the trailing "/" here; without that the shell thinks you are trying to list
+your home directory under the bucket, which will only exist if explicitly created.
+
+
+Attempting to list a bucket using inline credentials is a
+means of verifying that the key and secret can access a bucket;
+
+    hadoop fs -ls s3a://key:secret@my-bucket/
+
+Do escape any `+` or `/` symbols in the secret, as discussed below, and never
+share the URL, logs generated using it, or use such an inline authentication
+mechanism in production.
+
+Finally, if you set the environment variables, you can take advantage of S3A's
+support of environment-variable authentication by attempting the same ls operation.
+That is: unset the `fs.s3a` secrets and rely on the environment variables.
+
+### Authentication failure due to clock skew
+
+The timestamp is used in signing to S3, so as to
+defend against replay attacks. If the system clock is too far behind *or ahead*
+of Amazon's, requests will be rejected.
+
+This can surface as the situation where
+read requests are allowed, but operations which write to the bucket are denied.
+
+Check the system clock.
+
+### Authentication failure when using URLs with embedded secrets
+
+If using the (strongly discouraged) mechanism of including the
+AWS Key and secret in a URL, then both "+" and "/" symbols need
+to encoded in the URL. As many AWS secrets include these characters,
+encoding problems are not uncommon.
+
+| symbol | encoded  value|
+|-----------|-------------|
+| `+` | `%2B` |
+| `/` | `%2F` |
+
+
+As an example, a URL for `bucket` with AWS ID `user1` and secret `a+b/c` would
+be represented as
 
-```properties
-log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
-log4j.logger.com.amazonaws.request=DEBUG
-log4j.logger.org.apache.http=DEBUG
-log4j.logger.org.apache.http.wire=ERROR
 ```
+s3a://user1:a%2Bb%2Fc@bucket/
+```
+
+This technique is only needed when placing secrets in the URL. Again,
+this is something users are strongly advised against using.
+
+### <a name="bad_request"></a> "Bad Request" exception when working with AWS S3 Frankfurt, Seoul, or other "V4" endpoint
+
+
+S3 Frankfurt and Seoul *only* support
+[the V4 authentication API](http://docs.aws.amazon.com/AmazonS3/latest/API/sig-v4-authenticating-requests.html).
+
+Requests using the V2 API will be rejected with 400 `Bad Request`
+
+```
+$ bin/hadoop fs -ls s3a://frankfurt/
+WARN s3a.S3AFileSystem: Client: Amazon S3 error 400: 400 Bad Request; Bad Request (retryable)
+
+com.amazonaws.services.s3.model.AmazonS3Exception: Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: 923C5D9E75E44C06), S3 Extended Request ID: HDwje6k+ANEeDsM6aJ8+D5gUmNAMguOk2BvZ8PH3g9z0gpH+IuwT7N19oQOnIr5CIx7Vqb/uThE=
+    at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182)
+    at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770)
+    at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
+    at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
+    at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
+    at com.amazonaws.services.s3.AmazonS3Client.headBucket(AmazonS3Client.java:1107)
+    at com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1070)
+    at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:307)
+    at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:284)
+    at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2793)
+    at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:101)
+    at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2830)
+    at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2812)
+    at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:389)
+    at org.apache.hadoop.fs.Path.getFileSystem(Path.java:356)
+    at org.apache.hadoop.fs.shell.PathData.expandAsGlob(PathData.java:325)
+    at org.apache.hadoop.fs.shell.Command.expandArgument(Command.java:235)
+    at org.apache.hadoop.fs.shell.Command.expandArguments(Command.java:218)
+    at org.apache.hadoop.fs.shell.FsCommand.processRawArguments(FsCommand.java:103)
+    at org.apache.hadoop.fs.shell.Command.run(Command.java:165)
+    at org.apache.hadoop.fs.FsShell.run(FsShell.java:315)
+    at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
+    at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90)
+    at org.apache.hadoop.fs.FsShell.main(FsShell.java:373)
+ls: doesBucketExist on frankfurt-new: com.amazonaws.services.s3.model.AmazonS3Exception:
+  Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request;
+```
+
+This happens when trying to work with any S3 service which only supports the
+"V4" signing API —but the client is configured to use the default S3 service
+endpoint.
+
+The S3A client needs to be given the endpoint to use via the `fs.s3a.endpoint`
+property.
+
+As an example, the endpoint for S3 Frankfurt is `s3.eu-central-1.amazonaws.com`:
+
+```xml
+<property>
+  <name>fs.s3a.endpoint</name>
+  <value>s3.eu-central-1.amazonaws.com</value>
+</property>
+```
+
+## <a name="connectivity"></a> Connectivity Problems
+
+### <a name="bad_endpoint"></a> Error message "The bucket you are attempting to access must be addressed using the specified endpoint"
+
+This surfaces when `fs.s3a.endpoint` is configured to use an S3 service endpoint
+which is neither the original AWS one, `s3.amazonaws.com` , nor the one where
+the bucket is hosted.  The error message contains the redirect target returned
+by S3, which can be used to determine the correct value for `fs.s3a.endpoint`.
+
+```
+org.apache.hadoop.fs.s3a.AWSS3IOException: Received permanent redirect response
+  to bucket.s3-us-west-2.amazonaws.com.  This likely indicates that the S3
+  endpoint configured in fs.s3a.endpoint does not match the AWS region
+  containing the bucket.: The bucket you are attempting to access must be
+  addressed using the specified endpoint. Please send all future requests to
+  this endpoint. (Service: Amazon S3; Status Code: 301;
+  Error Code: PermanentRedirect; Request ID: 7D39EC1021C61B11)
+        at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:132)
+        at org.apache.hadoop.fs.s3a.S3AFileSystem.initMultipartUploads(S3AFileSystem.java:287)
+        at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:203)
+        at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2895)
+        at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:102)
+        at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2932)
+        at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2914)
+        at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:390)
+```
+
+1. Use the [Specific endpoint of the bucket's S3 service](http://docs.aws.amazon.com/general/latest/gr/rande.html#s3_region)
+1. If not using "V4" authentication (see above), the original S3 endpoint
+can be used:
+
+```xml
+<property>
+  <name>fs.s3a.endpoint</name>
+  <value>s3.amazonaws.com</value>
+</property>
+```
+
+Using the explicit endpoint for the region is recommended for speed and
+to use the V4 signing API.
+
+
+### <a name="timeout"></a> "Timeout waiting for connection from pool" when writing data
+
+This happens when using the output stream thread pool runs out of capacity.
+
+```
+[s3a-transfer-shared-pool1-t20] INFO  http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) - Unable to execute HTTP request: Timeout waiting for connection from poolorg.apache.http.conn.ConnectionPoolTimeoutException: Timeout waiting for connection from pool
+  at org.apache.http.impl.conn.PoolingClientConnectionManager.leaseConnection(PoolingClientConnectionManager.java:230)
+  at org.apache.http.impl.conn.PoolingClientConnectionManager$1.getConnection(PoolingClientConnectionManager.java:199)
+  at sun.reflect.GeneratedMethodAccessor13.invoke(Unknown Source)
+  at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
+  at java.lang.reflect.Method.invoke(Method.java:498)
+  at com.amazonaws.http.conn.ClientConnectionRequestFactory$Handler.invoke(ClientConnectionRequestFactory.java:70)
+  at com.amazonaws.http.conn.$Proxy10.getConnection(Unknown Source)
+  at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:424)
+  at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884)
+  at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
+  at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
+  at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728)
+  at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
+  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
+  at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
+  at com.amazonaws.services.s3.AmazonS3Client.doUploadPart(AmazonS3Client.java:2921)
+  at com.amazonaws.services.s3.AmazonS3Client.uploadPart(AmazonS3Client.java:2906)
+  at org.apache.hadoop.fs.s3a.S3AFileSystem.uploadPart(S3AFileSystem.java:1025)
+  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:360)
+  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:355)
+  at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239)
+  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
+  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
+  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
+  at java.lang.Thread.run(Thread.java:745)
+```
+
+Make sure that `fs.s3a.connection.maximum` is at least larger
+than `fs.s3a.threads.max`.
+
+```xml
+<property>
+  <name>fs.s3a.threads.max</name>
+  <value>20</value>
+</property>
+
+<property>
+  <name>fs.s3a.connection.maximum</name>
+  <value>30</value>
+</property>
+```
+
+### "Timeout waiting for connection from pool" when reading data
+
+This happens when more threads are trying to read from an S3A system than
+the maximum number of allocated HTTP connections.
+
+Set `fs.s3a.connection.maximum` to a larger value (and at least as large as
+`fs.s3a.threads.max`)
+
+### Out of heap memory when writing with via Fast Upload
+
+This can happen when using the upload buffering mechanism
+uses memory (either `fs.s3a.fast.upload.buffer=array` or
+`fs.s3a.fast.upload.buffer=bytebuffer`).
+
+More data is being generated than in the JVM than it can upload to S3 —and
+so much data has been buffered that the JVM has run out of memory.
 
-Be aware that logging HTTP headers may leak sensitive AWS account information,
-so should not be shared.
+1. Consult [S3A Fast Upload Thread Tuning](./index.html#fast_upload_thread_tuning) for
+detail on this issue and options to address it.
 
-## Advanced: network performance
+1. Switch to buffering to disk, rather than memory.
+
+
+This surfaces if, while a multipart upload was taking place, all outstanding multipart
+uploads were garbage collected. The upload operation cannot complete because
+the data uploaded has been deleted.
+
+Consult [Cleaning up After Incremental Upload Failures](./index.html#multipart_purge) for
+details on how the multipart purge timeout can be set. If multipart uploads
+are failing with the message above, it may be a sign that this value is too low.
+
+### `MultiObjectDeleteException` during delete or rename of files
+
+```
+Exception in thread "main" com.amazonaws.services.s3.model.MultiObjectDeleteException:
+    Status Code: 0, AWS Service: null, AWS Request ID: null, AWS Error Code: null,
+    AWS Error Message: One or more objects could not be deleted, S3 Extended Request ID: null
+  at com.amazonaws.services.s3.AmazonS3Client.deleteObjects(AmazonS3Client.java:1745)
+```
+This happens when trying to delete multiple objects, and one of the objects
+could not be deleted. It *should not occur* just because the object is missing.
+More specifically: at the time this document was written, we could not create
+such a failure.
+
+It will occur if the caller lacks the permission to delete any of the objects.
+
+Consult the log to see the specifics of which objects could not be deleted.
+Do you have permission to do so?
+
+If this operation is failing for reasons other than the caller lacking
+permissions:
+
+1. Try setting `fs.s3a.multiobjectdelete.enable` to `false`.
+1. Consult [HADOOP-11572](https://issues.apache.org/jira/browse/HADOOP-11572)
+for up to date advice.
+
+### "Failed to Sanitize XML document"
+
+```
+org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on test/testname/streaming/:
+  com.amazonaws.AmazonClientException: Failed to sanitize XML document
+  destined for handler class com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$ListBucketHandler:
+  Failed to sanitize XML document destined for handler class com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$ListBucketHandler
+    at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:105)
+    at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1462)
+    at org.apache.hadoop.fs.s3a.S3AFileSystem.innerListStatus(S3AFileSystem.java:1227)
+    at org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:1203)
+    at org.apache.hadoop.fs.s3a.S3AGlobber.listStatus(S3AGlobber.java:69)
+    at org.apache.hadoop.fs.s3a.S3AGlobber.doGlob(S3AGlobber.java:210)
+    at org.apache.hadoop.fs.s3a.S3AGlobber.glob(S3AGlobber.java:125)
+    at org.apache.hadoop.fs.s3a.S3AFileSystem.globStatus(S3AFileSystem.java:1853)
+    at org.apache.hadoop.fs.s3a.S3AFileSystem.globStatus(S3AFileSystem.java:1841)
+```
+
+We believe this is caused by the connection to S3 being broken.
+See [HADOOP-13811](https://issues.apache.org/jira/browse/HADOOP-13811).
+
+It may go away if the operation is retried.
+
+### JSON Parse Error from AWS SDK
+
+Sometimes a JSON Parse error is reported with the stack trace in the `com.amazonaws`,
+
+Again, we believe this is caused by the connection to S3 being broken.
+
+It may go away if the operation is retried.
+
+
+
+## Miscellaneous Errors
+
+### When writing data: "java.io.FileNotFoundException: Completing multi-part upload"
+
+
+```
+java.io.FileNotFoundException: Completing multi-part upload on fork-5/test/multipart/1c397ca6-9dfb-4ac1-9cf7-db666673246b: com.amazonaws.services.s3.model.AmazonS3Exception: The specified upload does not exist. The upload ID may be invalid, or the upload may have been aborted or completed. (Service: Amazon S3; Status Code: 404; Error Code: NoSuchUpload; Request ID: 84FF8057174D9369), S3 Extended Request ID: Ij5Yn6Eq/qIERH4Z6Io3YL2t9/qNZ7z9gjPb1FrTtTovZ8k1MXqh+zCYYjqmfJ/fCY6E1+JR9jA=
+  at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182)
+  at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770)
+  at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
+  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
+  at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
+  at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:2705)
+  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.complete(S3ABlockOutputStream.java:473)
+  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$200(S3ABlockOutputStream.java:382)
+  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:272)
+  at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
+  at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
+```
+
+### Issue: when writing data, HTTP Exceptions logged at info from `AmazonHttpClient`
+
+```
+[s3a-transfer-shared-pool4-t6] INFO  http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) - Unable to execute HTTP request: hwdev-steve-ireland-new.s3.amazonaws.com:443 failed to respond
+org.apache.http.NoHttpResponseException: bucket.s3.amazonaws.com:443 failed to respond
+  at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:143)
+  at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:57)
+  at org.apache.http.impl.io.AbstractMessageParser.parse(AbstractMessageParser.java:261)
+  at org.apache.http.impl.AbstractHttpClientConnection.receiveResponseHeader(AbstractHttpClientConnection.java:283)
+  at org.apache.http.impl.conn.DefaultClientConnection.receiveResponseHeader(DefaultClientConnection.java:259)
+  at org.apache.http.impl.conn.ManagedClientConnectionImpl.receiveResponseHeader(ManagedClientConnectionImpl.java:209)
+  at org.apache.http.protocol.HttpRequestExecutor.doReceiveResponse(HttpRequestExecutor.java:272)
+  at com.amazonaws.http.protocol.SdkHttpRequestExecutor.doReceiveResponse(SdkHttpRequestExecutor.java:66)
+  at org.apache.http.protocol.HttpRequestExecutor.execute(HttpRequestExecutor.java:124)
+  at org.apache.http.impl.client.DefaultRequestDirector.tryExecute(DefaultRequestDirector.java:686)
+  at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:488)
+  at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884)
+  at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
+  at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
+  at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728)
+  at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
+  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
+  at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
+  at com.amazonaws.services.s3.AmazonS3Client.copyPart(AmazonS3Client.java:1731)
+  at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:41)
+  at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:28)
+  at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239)
+  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
+  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
+  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
+  at java.lang.Thread.run(Thread.java:745)
+```
+
+These are HTTP I/O exceptions caught and logged inside the AWS SDK. The client
+will attempt to retry the operation; it may just be a transient event. If there
+are many such exceptions in logs, it may be a symptom of connectivity or network
+problems.
+
+## File System Semantics
+
+These are the issues where S3 does not appear to behave the way a filesystem
+"should".
+
+### Visible S3 Inconsistency
+
+Amazon S3 is *an eventually consistent object store*. That is: not a filesystem.
+
+To reduce visible inconsistencies, use the [S3Guard](./s3guard.html) consistency
+cache.
+
+
+By default, Amazon S3 offers read-after-create consistency: a newly created file
+is immediately visible.
+There is a small quirk: a negative GET may be cached, such
+that even if an object is immediately created, the fact that there "wasn't"
+an object is still remembered.
+
+That means the following sequence on its own will be consistent
+```
+touch(path) -> getFileStatus(path)
+```
+
+But this sequence *may* be inconsistent.
+
+```
+getFileStatus(path) -> touch(path) -> getFileStatus(path)
+```
+
+A common source of visible inconsistencies is that the S3 metadata
+database —the part of S3 which serves list requests— is updated asynchronously.
+Newly added or deleted files may not be visible in the index, even though direct
+operations on the object (`HEAD` and `GET`) succeed.
+
+That means the `getFileStatus()` and `open()` operations are more likely
+to be consistent with the state of the object store, but without S3Guard enabled,
+directory list operations such as `listStatus()`, `listFiles()`, `listLocatedStatus()`,
+and `listStatusIterator()` may not see newly created files, and still list
+old files.
+
+### `FileNotFoundException` even though the file was just written.
+
+This can be a sign of consistency problems. It may also surface if there is some
+asynchronous file write operation still in progress in the client: the operation
+has returned, but the write has not yet completed. While the S3A client code
+does block during the `close()` operation, we suspect that asynchronous writes
+may be taking place somewhere in the stack —this could explain why parallel tests
+fail more often than serialized tests.
+
+### File not found in a directory listing, even though `getFileStatus()` finds it
+
+(Similarly: deleted file found in listing, though `getFileStatus()` reports
+that it is not there)
+
+This is a visible sign of updates to the metadata server lagging
+behind the state of the underlying filesystem.
+
+Fix: Use S3Guard
+
+
+### File not visible/saved
+
+The files in an object store are not visible until the write has been completed.
+In-progress writes are simply saved to a local file/cached in RAM and only uploaded.
+at the end of a write operation. If a process terminated unexpectedly, or failed
+to call the `close()` method on an output stream, the pending data will have
+been lost.
+
+### File `flush()`, `hsync` and `hflush()` calls do not save data to S3
+
+Again, this is due to the fact that the data is cached locally until the
+`close()` operation. The S3A filesystem cannot be used as a store of data
+if it is required that the data is persisted durably after every
+`Syncable.hflush()` or `Syncable.hsync()` call.
+This includes resilient logging, HBase-style journalling
+and the like. The standard strategy here is to save to HDFS and then copy to S3.
+
+## <a name="encryption"></a> S3 Server Side Encryption
+
+### Using SSE-KMS "Invalid arn"
+
+When performing file operations, the user may run into an issue where the KMS
+key arn is invalid.
+```
+com.amazonaws.services.s3.model.AmazonS3Exception:
+Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException; Request ID: 708284CF60EE233F),
+S3 Extended Request ID: iHUUtXUSiNz4kv3Bdk/hf9F+wjPt8GIVvBHx/HEfCBYkn7W6zmpvbA3XT7Y5nTzcZtfuhcqDunw=:
+Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException; Request ID: 708284CF60EE233F)
+```
+
+This is due to either, the KMS key id is entered incorrectly, or the KMS key id
+is in a different region than the S3 bucket being used.
+
+### Using SSE-C "Bad Request"
+
+When performing file operations the user may run into an unexpected 400/403
+error such as
+```
+org.apache.hadoop.fs.s3a.AWSS3IOException: getFileStatus on fork-4/: com.amazonaws.services.s3.model.AmazonS3Exception:
+Bad Request (Service: Amazon S3; Status Code: 400;
+Error Code: 400 Bad Request; Request ID: 42F9A1987CB49A99),
+S3 Extended Request ID: jU2kcwaXnWj5APB14Cgb1IKkc449gu2+dhIsW/+7x9J4D+VUkKvu78mBo03oh9jnOT2eoTLdECU=:
+Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: 42F9A1987CB49A99)
+```
+
+This can happen in the cases of not specifying the correct SSE-C encryption key.
+Such cases can be as follows:
+1. An object is encrypted using SSE-C on S3 and either the wrong encryption type
+is used, no encryption is specified, or the SSE-C specified is incorrect.
+2. A directory is encrypted with a SSE-C keyA and the user is trying to move a
+file using configured SSE-C keyB into that structure.
+
+## <a name="performance"></a> Performance
+
+S3 is slower to read data than HDFS, even on virtual clusters running on
+Amazon EC2.
+
+* HDFS replicates data for faster query performance.
+* HDFS stores the data on the local hard disks, avoiding network traffic
+ if the code can be executed on that host. As EC2 hosts often have their
+ network bandwidth throttled, this can make a tangible difference.
+* HDFS is significantly faster for many "metadata" operations: listing
+the contents of a directory, calling `getFileStatus()` on path,
+creating or deleting directories. (S3Guard reduces but does not eliminate
+the speed gap).
+* On HDFS, Directory renames and deletes are `O(1)` operations. On
+S3 renaming is a very expensive `O(data)` operation which may fail partway through
+in which case the final state depends on where the copy+ delete sequence was when it failed.
+All the objects are copied, then the original set of objects are deleted, so
+a failure should not lose data —it may result in duplicate datasets.
+* Unless fast upload enabled, the write only begins on a `close()` operation.
+This can take so long that some applications can actually time out.
+* File IO involving many seek calls/positioned read calls will encounter
+performance problems due to the size of the HTTP requests made. Enable the
+"random" fadvise policy to alleviate this at the
+expense of sequential read performance and bandwidth.
+
+The slow performance of `rename()` surfaces during the commit phase of work,
+including
+
+* The MapReduce `FileOutputCommitter`. This also used by Apache Spark.
+* DistCp's rename-after-copy operation.
+* The `hdfs fs -rm` command renaming the file under `.Trash` rather than
+deleting it. Use `-skipTrash` to eliminate that step.
+
+These operations can be significantly slower when S3 is the destination
+compared to HDFS or other "real" filesystem.
+
+*Improving S3 load-balancing behavior*
+
+Amazon S3 uses a set of front-end servers to provide access to the underlying data.
+The choice of which front-end server to use is handled via load-balancing DNS
+service: when the IP address of an S3 bucket is looked up, the choice of which
+IP address to return to the client is made based on the the current load
+of the front-end servers.
+
+Over time, the load across the front-end changes, so those servers considered
+"lightly loaded" will change. If the DNS value is cached for any length of time,
+your application may end up talking to an overloaded server. Or, in the case
+of failures, trying to talk to a server that is no longer there.
+
+And by default, for historical security reasons in the era of applets,
+the DNS TTL of a JVM is "infinity".
+
+To work with AWS better, set the DNS time-to-live of an application which
+works with S3 to something lower. See [AWS documentation](http://docs.aws.amazon.com/AWSSdkDocsJava/latest/DeveloperGuide/java-dg-jvm-ttl.html).
+
+## <a name="network_performance"></a>Troubleshooting network performance
 
 An example of this is covered in [HADOOP-13871](https://issues.apache.org/jira/browse/HADOOP-13871).
 
@@ -49,4 +650,74 @@ Consider reducing the connection timeout of the s3a connection.
   <value>15000</value>
 </property>
 ```
-This *may* cause the client to react faster to network pauses.
+This *may* cause the client to react faster to network pauses, so display
+stack traces fast. At the same time, it may be less resilient to
+connectivity problems.
+
+
+## Other Issues
+
+### <a name="logging"></a> Enabling low-level logging
+
+The AWS SDK and the Apache S3 components can be configured to log at
+more detail, as can S3A itself.
+
+```properties
+log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
+log4j.logger.com.amazonaws.request=DEBUG
+log4j.logger.com.amazonaws.thirdparty.apache.http=DEBUG
+```
+
+If using the "unshaded" JAR, then the Apache HttpClient can be directly configured:
+
+```properties
+log4j.logger.org.apache.http=DEBUG
+```
+
+
+This produces a log such as this, wich is for a V4-authenticated PUT of a 0-byte file used
+as an empty directory marker
+
+```
+execchain.MainClientExec (MainClientExec.java:execute(255)) - Executing request PUT /test/ HTTP/1.1
+execchain.MainClientExec (MainClientExec.java:execute(266)) - Proxy auth state: UNCHALLENGED
+http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(135)) - http-outgoing-0 >> PUT /test/ HTTP/1.1
+http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> Host: ireland-new.s3-eu-west-1.amazonaws.com
+http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> x-amz-content-sha256: UNSIGNED-PAYLOAD
+http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> Authorization: AWS4-HMAC-SHA256 Credential=AKIAIYZ5JEEEER/20170904/eu-west-1/s3/aws4_request,  ...
+http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> X-Amz-Date: 20170904T172929Z
+http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> User-Agent: Hadoop 3.0.0-beta-1, aws-sdk-java/1.11.134 ...
+http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> amz-sdk-invocation-id: 75b530f8-ad31-1ad3-13db-9bd53666b30d
+http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> amz-sdk-retry: 0/0/500
+http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> Content-Type: application/octet-stream
+http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> Content-Length: 0
+http.headers (LoggingManagedHttpClientConnection.java:onRequestSubmitted(138)) - http-outgoing-0 >> Connection: Keep-Alive
+http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "PUT /test/ HTTP/1.1[\r][\n]"
+http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "Host: ireland-new.s3-eu-west-1.amazonaws.com[\r][\n]"
+http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "x-amz-content-sha256: UNSIGNED-PAYLOAD[\r][\n]"
+http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "Authorization: AWS4-HMAC-SHA256 Credential=AKIAIYZ5JEEEER/20170904/eu-west-1/s3/aws4_request, ,,,
+http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "X-Amz-Date: 20170904T172929Z[\r][\n]"
+http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "User-Agent: 3.0.0-beta-1, aws-sdk-java/1.11.134  ...
+http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "amz-sdk-invocation-id: 75b530f8-ad31-1ad3-13db-9bd53666b30d[\r][\n]"
+http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "amz-sdk-retry: 0/0/500[\r][\n]"
+http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "Content-Type: application/octet-stream[\r][\n]"
+http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "Content-Length: 0[\r][\n]"
+http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "Connection: Keep-Alive[\r][\n]"
+http.wire (Wire.java:wire(72)) - http-outgoing-0 >> "[\r][\n]"
+http.wire (Wire.java:wire(72)) - http-outgoing-0 << "HTTP/1.1 200 OK[\r][\n]"
+http.wire (Wire.java:wire(72)) - http-outgoing-0 << "x-amz-id-2: mad9GqKztzlL0cdnCKAj9GJOAs+DUjbSC5jRkO7W1E7Nk2BUmFvt81bhSNPGdZmyyKqQI9i/B/A=[\r][\n]"
+http.wire (Wire.java:wire(72)) - http-outgoing-0 << "x-amz-request-id: C953D2FE4ABF5C51[\r][\n]"
+http.wire (Wire.java:wire(72)) - http-outgoing-0 << "Date: Mon, 04 Sep 2017 17:29:30 GMT[\r][\n]"
+http.wire (Wire.java:wire(72)) - http-outgoing-0 << "ETag: "d41d8cd98f00b204e9800998ecf8427e"[\r][\n]"
+http.wire (Wire.java:wire(72)) - http-outgoing-0 << "Content-Length: 0[\r][\n]"
+http.wire (Wire.java:wire(72)) - http-outgoing-0 << "Server: AmazonS3[\r][\n]"
+http.wire (Wire.java:wire(72)) - http-outgoing-0 << "[\r][\n]"
+http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(124)) - http-outgoing-0 << HTTP/1.1 200 OK
+http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 << x-amz-id-2: mad9GqKztzlL0cdnCKAj9GJOAs+DUjbSC5jRkO7W1E7Nk2BUmFvt81bhSNPGdZmyyKqQI9i/B/A=
+http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 << x-amz-request-id: C953D2FE4ABF5C51
+http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 << Date: Mon, 04 Sep 2017 17:29:30 GMT
+http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 << ETag: "d41d8cd98f00b204e9800998ecf8427e"
+http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 << Content-Length: 0
+http.headers (LoggingManagedHttpClientConnection.java:onResponseReceived(127)) - http-outgoing-0 << Server: AmazonS3
+execchain.MainClientExec (MainClientExec.java:execute(284)) - Connection can be kept alive for 60000 MILLISECONDS
+```

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
index 587dbbc..8da8b6a 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java
@@ -47,7 +47,6 @@ public class ITestS3AContractDistCp extends AbstractContractDistCpTest {
   protected Configuration createConfiguration() {
     Configuration newConf = super.createConfiguration();
     newConf.setLong(MULTIPART_SIZE, MULTIPART_SETTING);
-    newConf.setBoolean(FAST_UPLOAD, true);
     newConf.set(FAST_UPLOAD_BUFFER, FAST_UPLOAD_BUFFER_DISK);
     // patch in S3Guard options
     maybeEnableS3Guard(newConf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractCreate.java
deleted file mode 100644
index 502cf5a..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractCreate.java
+++ /dev/null
@@ -1,41 +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.contract.s3n;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-
-/**
- * S3N contract tests creating files.
- */
-public class ITestS3NContractCreate extends AbstractContractCreateTest {
-
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new NativeS3Contract(conf);
-  }
-
-  @Override
-  public void testOverwriteEmptyDirectory() throws Throwable {
-    ContractTestUtils.skip(
-        "blobstores can't distinguish empty directories from files");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractDelete.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractDelete.java
deleted file mode 100644
index 675f979..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractDelete.java
+++ /dev/null
@@ -1,34 +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.contract.s3n;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-/**
- * S3A contract tests covering deletes.
- */
-public class ITestS3NContractDelete extends AbstractContractDeleteTest {
-
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new NativeS3Contract(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractMkdir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractMkdir.java
deleted file mode 100644
index 3c566f3..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractMkdir.java
+++ /dev/null
@@ -1,34 +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.contract.s3n;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-/**
- * Test dir operations on S3.
- */
-public class ITestS3NContractMkdir extends AbstractContractMkdirTest {
-
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new NativeS3Contract(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractOpen.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractOpen.java
deleted file mode 100644
index 7ebfc4e..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractOpen.java
+++ /dev/null
@@ -1,34 +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.contract.s3n;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-/**
- * S3N contract tests opening files.
- */
-public class ITestS3NContractOpen extends AbstractContractOpenTest {
-
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new NativeS3Contract(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractRename.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractRename.java
deleted file mode 100644
index effe9eb..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractRename.java
+++ /dev/null
@@ -1,35 +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.contract.s3n;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-/**
- * S3N contract tests covering rename.
- */
-public class ITestS3NContractRename extends AbstractContractRenameTest {
-
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new NativeS3Contract(conf);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractRootDir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractRootDir.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractRootDir.java
deleted file mode 100644
index 3fdf868..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractRootDir.java
+++ /dev/null
@@ -1,35 +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.contract.s3n;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-/**
- * Root dir operations against an S3 bucket.
- */
-public class ITestS3NContractRootDir extends
-    AbstractContractRootDirectoryTest {
-
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new NativeS3Contract(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractSeek.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractSeek.java
deleted file mode 100644
index 9e1ce73..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/ITestS3NContractSeek.java
+++ /dev/null
@@ -1,34 +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.contract.s3n;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-/**
- * S3N contract tests covering file seek.
- */
-public class ITestS3NContractSeek extends AbstractContractSeekTest {
-
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new NativeS3Contract(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/NativeS3Contract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/NativeS3Contract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/NativeS3Contract.java
deleted file mode 100644
index 5796d88..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3n/NativeS3Contract.java
+++ /dev/null
@@ -1,50 +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.contract.s3n;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
-
-/**
- * The contract of S3N: only enabled if the test bucket is provided.
- */
-public class NativeS3Contract extends AbstractBondedFSContract {
-
-  public static final String CONTRACT_XML = "contract/s3n.xml";
-
-
-  public NativeS3Contract(Configuration conf) {
-    super(conf);
-    //insert the base features
-    addConfResource(CONTRACT_XML);
-  }
-
-  @Override
-  public String getScheme() {
-    return "s3n";
-  }
-
-  @Override
-  public Path getTestPath() {
-    String testUniqueForkId = System.getProperty("test.unique.fork.id");
-    return testUniqueForkId == null ? super.getTestPath() :
-        new Path("/" + testUniqueForkId, "test");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java
index 87f676c..afd3ec2 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java
@@ -56,7 +56,6 @@ public class ITestS3ABlockOutputArray extends AbstractS3ATestBase {
     S3ATestUtils.disableFilesystemCaching(conf);
     conf.setLong(MIN_MULTIPART_THRESHOLD, MULTIPART_MIN_SIZE);
     conf.setInt(MULTIPART_SIZE, MULTIPART_MIN_SIZE);
-    conf.setBoolean(Constants.FAST_UPLOAD, true);
     conf.set(FAST_UPLOAD_BUFFER, getBlockOutputBufferName());
     return conf;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSECBlockOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSECBlockOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSECBlockOutputStream.java
index afa0441..8991bad 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSECBlockOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSECBlockOutputStream.java
@@ -31,7 +31,6 @@ public class ITestS3AEncryptionSSECBlockOutputStream
   @Override
   protected Configuration createConfiguration() {
     Configuration conf = super.createConfiguration();
-    conf.setBoolean(Constants.FAST_UPLOAD, true);
     conf.set(Constants.FAST_UPLOAD_BUFFER,
         Constants.FAST_UPLOAD_BYTEBUFFER);
     conf.set(Constants.SERVER_SIDE_ENCRYPTION_KEY,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEKMSUserDefinedKeyBlockOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEKMSUserDefinedKeyBlockOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEKMSUserDefinedKeyBlockOutputStream.java
index 8ce3a13..4c953bd 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEKMSUserDefinedKeyBlockOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEKMSUserDefinedKeyBlockOutputStream.java
@@ -39,9 +39,7 @@ public class ITestS3AEncryptionSSEKMSUserDefinedKeyBlockOutputStream
       skip(Constants.SERVER_SIDE_ENCRYPTION_KEY+ " is not set for " +
           S3AEncryptionMethods.SSE_KMS.getMethod());
     }
-    conf.setBoolean(Constants.FAST_UPLOAD, true);
-    conf.set(Constants.FAST_UPLOAD_BUFFER,
-        Constants.FAST_UPLOAD_BYTEBUFFER);
+    conf.set(Constants.FAST_UPLOAD_BUFFER, Constants.FAST_UPLOAD_BYTEBUFFER);
     return conf;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSES3BlockOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSES3BlockOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSES3BlockOutputStream.java
index 407601f..ff9c07a 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSES3BlockOutputStream.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSES3BlockOutputStream.java
@@ -29,7 +29,6 @@ public class ITestS3AEncryptionSSES3BlockOutputStream
   @Override
   protected Configuration createConfiguration() {
     Configuration conf = super.createConfiguration();
-    conf.setBoolean(Constants.FAST_UPLOAD, true);
     conf.set(Constants.FAST_UPLOAD_BUFFER,
         Constants.FAST_UPLOAD_BYTEBUFFER);
     //must specify encryption key as empty because SSE-S3 does not allow it,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
index 8b163cb..230dbad 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
@@ -98,7 +98,6 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
     conf.setLong(MIN_MULTIPART_THRESHOLD, partitionSize);
     conf.setInt(MULTIPART_SIZE, partitionSize);
     conf.set(USER_AGENT_PREFIX, "STestS3AHugeFileCreate");
-    conf.setBoolean(FAST_UPLOAD, true);
     conf.set(FAST_UPLOAD_BUFFER, getBlockOutputBufferName());
     return conf;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesClassicOutput.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesClassicOutput.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesClassicOutput.java
deleted file mode 100644
index 551956b..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesClassicOutput.java
+++ /dev/null
@@ -1,41 +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.s3a.scale;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.s3a.Constants;
-
-/**
- * Use classic output for writing things; tweaks the configuration to do
- * this after it has been set up in the superclass.
- * The generator test has been copied and re
- */
-public class ITestS3AHugeFilesClassicOutput extends AbstractSTestS3AHugeFiles {
-
-  @Override
-  protected Configuration createScaleConfiguration() {
-    final Configuration conf = super.createScaleConfiguration();
-    conf.setBoolean(Constants.FAST_UPLOAD, false);
-    return conf;
-  }
-
-  protected String getBlockOutputBufferName() {
-    return "classic";
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestInMemoryNativeS3FileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestInMemoryNativeS3FileSystemContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestInMemoryNativeS3FileSystemContract.java
deleted file mode 100644
index adbf950..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestInMemoryNativeS3FileSystemContract.java
+++ /dev/null
@@ -1,33 +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.s3native;
-
-import java.io.IOException;
-
-/**
- * S3N basic contract tests through mock in-memory S3 implementation.
- */
-public class ITestInMemoryNativeS3FileSystemContract
-    extends NativeS3FileSystemContractBaseTest {
-
-  @Override
-  NativeFileSystemStore getNativeFileSystemStore() throws IOException {
-    return new InMemoryNativeFileSystemStore();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestJets3tNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestJets3tNativeFileSystemStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestJets3tNativeFileSystemStore.java
deleted file mode 100644
index cfe622c..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestJets3tNativeFileSystemStore.java
+++ /dev/null
@@ -1,133 +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.s3native;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-
-import static org.junit.Assert.*;
-import static org.junit.Assume.*;
-
-import org.junit.Before;
-import org.junit.After;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.URI;
-import java.security.DigestInputStream;
-import java.security.DigestOutputStream;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-
-/**
- * S3N tests through live S3 service.
- */
-public class ITestJets3tNativeFileSystemStore {
-  private Configuration conf;
-  private Jets3tNativeFileSystemStore store;
-  private NativeS3FileSystem fs;
-
-  @Before
-  public void setUp() throws Exception {
-    conf = new Configuration();
-    store = new Jets3tNativeFileSystemStore();
-    fs = new NativeS3FileSystem(store);
-    conf.setBoolean("fs.s3n.multipart.uploads.enabled", true);
-    conf.setLong("fs.s3n.multipart.uploads.block.size", 64 * 1024 * 1024);
-    fs.initialize(URI.create(conf.get("test.fs.s3n.name")), conf);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    try {
-      store.purge("test");
-    } catch (Exception e) {}
-  }
-
-  @BeforeClass
-  public static void checkSettings() throws Exception {
-    Configuration conf = new Configuration();
-    assumeNotNull(conf.get("fs.s3n.awsAccessKeyId"));
-    assumeNotNull(conf.get("fs.s3n.awsSecretAccessKey"));
-    assumeNotNull(conf.get("test.fs.s3n.name"));
-  }
-
-  protected void writeRenameReadCompare(Path path, long len)
-      throws IOException, NoSuchAlgorithmException {
-    // If len > fs.s3n.multipart.uploads.block.size,
-    // we'll use a multipart upload copy
-    MessageDigest digest = MessageDigest.getInstance("MD5");
-    OutputStream out = new BufferedOutputStream(
-        new DigestOutputStream(fs.create(path, false), digest));
-    for (long i = 0; i < len; i++) {
-      out.write('Q');
-    }
-    out.flush();
-    out.close();
-
-    assertTrue("Exists", fs.exists(path));
-
-    // Depending on if this file is over 5 GB or not,
-    // rename will cause a multipart upload copy
-    Path copyPath = path.suffix(".copy");
-    fs.rename(path, copyPath);
-
-    assertTrue("Copy exists", fs.exists(copyPath));
-
-    // Download file from S3 and compare the digest against the original
-    MessageDigest digest2 = MessageDigest.getInstance("MD5");
-    InputStream in = new BufferedInputStream(
-        new DigestInputStream(fs.open(copyPath), digest2));
-    long copyLen = 0;
-    while (in.read() != -1) {
-      copyLen++;
-    }
-    in.close();
-
-    assertEquals("Copy length matches original", len, copyLen);
-    assertArrayEquals("Digests match", digest.digest(), digest2.digest());
-  }
-
-  @Test
-  public void testSmallUpload() throws IOException, NoSuchAlgorithmException {
-    // Regular upload, regular copy
-    writeRenameReadCompare(new Path("/test/small"), 16384);
-  }
-
-  @Test
-  public void testMediumUpload() throws IOException, NoSuchAlgorithmException {
-    // Multipart upload, regular copy
-    writeRenameReadCompare(new Path("/test/medium"), 33554432);    // 100 MB
-  }
-
-  /*
-  Enable Multipart upload to run this test
-  @Test
-  public void testExtraLargeUpload()
-      throws IOException, NoSuchAlgorithmException {
-    // Multipart upload, multipart copy
-    writeRenameReadCompare(new Path("/test/xlarge"), 5368709121L); // 5GB+1byte
-  }
-  */
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestJets3tNativeS3FileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestJets3tNativeS3FileSystemContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestJets3tNativeS3FileSystemContract.java
deleted file mode 100644
index e51eaf6..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/ITestJets3tNativeS3FileSystemContract.java
+++ /dev/null
@@ -1,33 +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.s3native;
-
-import java.io.IOException;
-
-/**
- * S3N basic contract tests through live S3 service.
- */
-public class ITestJets3tNativeS3FileSystemContract
-    extends NativeS3FileSystemContractBaseTest {
-
-  @Override
-  NativeFileSystemStore getNativeFileSystemStore() throws IOException {
-    return new Jets3tNativeFileSystemStore();
-  }
-}


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


[47/50] [abbrv] hadoop git commit: YARN-7199. Fix TestAMRMClientContainerRequest.testOpportunisticAndGuaranteedRequests. (Botong Huang via asuresh)

Posted by ae...@apache.org.
YARN-7199. Fix TestAMRMClientContainerRequest.testOpportunisticAndGuaranteedRequests. (Botong Huang via asuresh)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/29dd5515
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/29dd5515
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/29dd5515

Branch: refs/heads/HDFS-7240
Commit: 29dd55153e37471d9c177f4bd173f1d02bc96410
Parents: 0adc047
Author: Arun Suresh <as...@apache.org>
Authored: Mon Sep 18 11:26:44 2017 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Mon Sep 18 11:26:44 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/yarn/client/api/AMRMClient.java  | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/29dd5515/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
index 815915e..e86bd12 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
@@ -118,8 +118,8 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
     private String nodeLabelsExpression;
     private ExecutionTypeRequest executionTypeRequest =
         ExecutionTypeRequest.newInstance();
-    private String resourceProfile;
-    
+    private String resourceProfile = ProfileCapability.DEFAULT_PROFILE;
+
     /**
      * Instantiates a {@link ContainerRequest} with the given constraints and
      * locality relaxation enabled.
@@ -540,6 +540,11 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
         return this;
       }
 
+      public ContainerRequestBuilder resourceProfile(String resourceProfile) {
+        containerRequest.resourceProfile = resourceProfile;
+        return this;
+      }
+
       public ContainerRequest build() {
         containerRequest.sanityCheck();
         return containerRequest;


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


[24/50] [abbrv] hadoop git commit: HADOOP-14553. Add (parallelized) integration tests to hadoop-azure Contributed by Steve Loughran

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestContainerChecks.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestContainerChecks.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestContainerChecks.java
deleted file mode 100644
index f6ab94d..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestContainerChecks.java
+++ /dev/null
@@ -1,185 +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.azure;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assume.assumeNotNull;
-
-import java.io.FileNotFoundException;
-import java.util.EnumSet;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.CreateOptions;
-import org.junit.After;
-import org.junit.Assume;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.microsoft.azure.storage.blob.BlobOutputStream;
-import com.microsoft.azure.storage.blob.CloudBlobContainer;
-import com.microsoft.azure.storage.blob.CloudBlockBlob;
-
-/**
- * Tests that WASB creates containers only if needed.
- */
-public class TestContainerChecks {
-  private AzureBlobStorageTestAccount testAccount;
-  private boolean runningInSASMode = false;
-  @After
-  public void tearDown() throws Exception {
-    if (testAccount != null) {
-      testAccount.cleanup();
-      testAccount = null;
-    }
-  }
-
-  @Before
-  public void setMode() {
-    runningInSASMode = AzureBlobStorageTestAccount.createTestConfiguration().
-        getBoolean(AzureNativeFileSystemStore.KEY_USE_SECURE_MODE, false);
-  }
-
-  @Test
-  public void testContainerExistAfterDoesNotExist() throws Exception {
-    testAccount = AzureBlobStorageTestAccount.create("",
-        EnumSet.noneOf(CreateOptions.class));
-    assumeNotNull(testAccount);
-    CloudBlobContainer container = testAccount.getRealContainer();
-    FileSystem fs = testAccount.getFileSystem();
-
-    // Starting off with the container not there
-    assertFalse(container.exists());
-
-    // A list shouldn't create the container and will set file system store
-    // state to DoesNotExist
-    try {
-      fs.listStatus(new Path("/"));
-      assertTrue("Should've thrown.", false);
-    } catch (FileNotFoundException ex) {
-      assertTrue("Unexpected exception: " + ex,
-          ex.getMessage().contains("does not exist."));
-    }
-    assertFalse(container.exists());
-
-    // Create a container outside of the WASB FileSystem
-    container.create();
-    // Add a file to the container outside of the WASB FileSystem
-    CloudBlockBlob blob = testAccount.getBlobReference("foo");
-    BlobOutputStream outputStream = blob.openOutputStream();
-    outputStream.write(new byte[10]);
-    outputStream.close();
-
-    // Make sure the file is visible
-    assertTrue(fs.exists(new Path("/foo")));
-    assertTrue(container.exists());
-  }
-
-  @Test
-  public void testContainerCreateAfterDoesNotExist() throws Exception {
-    testAccount = AzureBlobStorageTestAccount.create("",
-        EnumSet.noneOf(CreateOptions.class));
-    assumeNotNull(testAccount);
-    CloudBlobContainer container = testAccount.getRealContainer();
-    FileSystem fs = testAccount.getFileSystem();
-
-    // Starting off with the container not there
-    assertFalse(container.exists());
-
-    // A list shouldn't create the container and will set file system store
-    // state to DoesNotExist
-    try {
-      assertNull(fs.listStatus(new Path("/")));
-      assertTrue("Should've thrown.", false);
-    } catch (FileNotFoundException ex) {
-      assertTrue("Unexpected exception: " + ex,
-          ex.getMessage().contains("does not exist."));
-    }
-    assertFalse(container.exists());
-
-    // Create a container outside of the WASB FileSystem
-    container.create();
-
-    // Write should succeed
-    assertTrue(fs.createNewFile(new Path("/foo")));
-    assertTrue(container.exists());
-  }
-
-  @Test
-  public void testContainerCreateOnWrite() throws Exception {
-    testAccount = AzureBlobStorageTestAccount.create("",
-        EnumSet.noneOf(CreateOptions.class));
-    assumeNotNull(testAccount);
-    CloudBlobContainer container = testAccount.getRealContainer();
-    FileSystem fs = testAccount.getFileSystem();
-
-    // Starting off with the container not there
-    assertFalse(container.exists());
-
-    // A list shouldn't create the container.
-    try {
-      fs.listStatus(new Path("/"));
-      assertTrue("Should've thrown.", false);
-    } catch (FileNotFoundException ex) {
-      assertTrue("Unexpected exception: " + ex,
-          ex.getMessage().contains("does not exist."));
-    }
-    assertFalse(container.exists());
-
-    // Neither should a read.
-    try {
-      fs.open(new Path("/foo"));
-      assertFalse("Should've thrown.", true);
-    } catch (FileNotFoundException ex) {
-    }
-    assertFalse(container.exists());
-
-    // Neither should a rename
-    assertFalse(fs.rename(new Path("/foo"), new Path("/bar")));
-    assertFalse(container.exists());
-
-    // But a write should.
-    assertTrue(fs.createNewFile(new Path("/foo")));
-    assertTrue(container.exists());
-  }
-
-  @Test
-  public void testContainerChecksWithSas() throws Exception {
-
-    Assume.assumeFalse(runningInSASMode);
-    testAccount = AzureBlobStorageTestAccount.create("",
-        EnumSet.of(CreateOptions.UseSas));
-    assumeNotNull(testAccount);
-    CloudBlobContainer container = testAccount.getRealContainer();
-    FileSystem fs = testAccount.getFileSystem();
-
-    // The container shouldn't be there
-    assertFalse(container.exists());
-
-    // A write should just fail
-    try {
-      fs.createNewFile(new Path("/foo"));
-      assertFalse("Should've thrown.", true);
-    } catch (AzureException ex) {
-    }
-    assertFalse(container.exists());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationExceptionHandling.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationExceptionHandling.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationExceptionHandling.java
deleted file mode 100644
index 9ac25dd..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationExceptionHandling.java
+++ /dev/null
@@ -1,269 +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.azure;
-
-import java.io.FileNotFoundException;
-
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Test;
-
-
-public class TestFileSystemOperationExceptionHandling
-    extends AbstractWasbTestBase {
-
-  private FSDataInputStream inputStream = null;
-
-  private static Path testPath = new Path("testfile.dat");
-
-  private static Path testFolderPath = new Path("testfolder");
-
-  /*
-   * Helper method that creates a InputStream to validate exceptions
-   * for various scenarios
-   */
-  private void setupInputStreamToTest(AzureBlobStorageTestAccount testAccount)
-      throws Exception {
-
-    FileSystem fs = testAccount.getFileSystem();
-
-    // Step 1: Create a file and write dummy data.
-    Path testFilePath1 = new Path("test1.dat");
-    Path testFilePath2 = new Path("test2.dat");
-    FSDataOutputStream outputStream = fs.create(testFilePath1);
-    String testString = "This is a test string";
-    outputStream.write(testString.getBytes());
-    outputStream.close();
-
-    // Step 2: Open a read stream on the file.
-    inputStream = fs.open(testFilePath1);
-
-    // Step 3: Rename the file
-    fs.rename(testFilePath1, testFilePath2);
-  }
-
-  /*
-   * Tests a basic single threaded read scenario for Page blobs.
-   */
-  @Test(expected=FileNotFoundException.class)
-  public void testSingleThreadedPageBlobReadScenario() throws Throwable {
-    AzureBlobStorageTestAccount testAccount = ExceptionHandlingTestHelper.getPageBlobTestStorageAccount();
-    setupInputStreamToTest(testAccount);
-    byte[] readBuffer = new byte[512];
-    inputStream.read(readBuffer);
-  }
-
-  /*
-   * Tests a basic single threaded seek scenario for Page blobs.
-   */
-  @Test(expected=FileNotFoundException.class)
-  public void testSingleThreadedPageBlobSeekScenario() throws Throwable {
-    AzureBlobStorageTestAccount testAccount = ExceptionHandlingTestHelper.getPageBlobTestStorageAccount();
-    setupInputStreamToTest(testAccount);
-    inputStream.seek(5);
-  }
-
-  /*
-   * Test a basic single thread seek scenario for Block blobs.
-   */
-  @Test(expected=FileNotFoundException.class)
-  public void testSingleThreadBlockBlobSeekScenario() throws Throwable {
-
-    AzureBlobStorageTestAccount testAccount = createTestAccount();
-    setupInputStreamToTest(testAccount);
-    inputStream.seek(5);
-    inputStream.read();
-  }
-
-  /*
-   * Tests a basic single threaded read scenario for Block blobs.
-   */
-  @Test(expected=FileNotFoundException.class)
-  public void testSingledThreadBlockBlobReadScenario() throws Throwable{
-    AzureBlobStorageTestAccount testAccount = createTestAccount();
-    setupInputStreamToTest(testAccount);
-    byte[] readBuffer = new byte[512];
-    inputStream.read(readBuffer);
-  }
-
-  @Test(expected=FileNotFoundException.class)
-  /*
-   * Tests basic single threaded setPermission scenario
-   */
-  public void testSingleThreadedBlockBlobSetPermissionScenario() throws Throwable {
-
-    ExceptionHandlingTestHelper.createEmptyFile(createTestAccount(), testPath);
-    fs.delete(testPath, true);
-    fs.setPermission(testPath, new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
-  }
-
-  @Test(expected=FileNotFoundException.class)
-  /*
-   * Tests basic single threaded setPermission scenario
-   */
-  public void testSingleThreadedPageBlobSetPermissionScenario() throws Throwable {
-    ExceptionHandlingTestHelper.createEmptyFile(ExceptionHandlingTestHelper.getPageBlobTestStorageAccount(),
-        testPath);
-    fs.delete(testPath, true);
-    fs.setOwner(testPath, "testowner", "testgroup");
-  }
-
-  @Test(expected=FileNotFoundException.class)
-  /*
-   * Tests basic single threaded setPermission scenario
-   */
-  public void testSingleThreadedBlockBlobSetOwnerScenario() throws Throwable {
-
-    ExceptionHandlingTestHelper.createEmptyFile(createTestAccount(), testPath);
-    fs.delete(testPath, true);
-    fs.setOwner(testPath, "testowner", "testgroup");
-  }
-
-  @Test(expected=FileNotFoundException.class)
-  /*
-   * Tests basic single threaded setPermission scenario
-   */
-  public void testSingleThreadedPageBlobSetOwnerScenario() throws Throwable {
-    ExceptionHandlingTestHelper.createEmptyFile(ExceptionHandlingTestHelper.getPageBlobTestStorageAccount(),
-        testPath);
-    fs.delete(testPath, true);
-    fs.setPermission(testPath, new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
-  }
-
-  @Test(expected=FileNotFoundException.class)
-  /*
-   * Test basic single threaded listStatus scenario
-   */
-  public void testSingleThreadedBlockBlobListStatusScenario() throws Throwable {
-    ExceptionHandlingTestHelper.createTestFolder(createTestAccount(), testFolderPath);
-    fs.delete(testFolderPath, true);
-    fs.listStatus(testFolderPath);
-  }
-
-  @Test(expected=FileNotFoundException.class)
-  /*
-   * Test basica single threaded listStatus scenario
-   */
-  public void testSingleThreadedPageBlobListStatusScenario() throws Throwable {
-    ExceptionHandlingTestHelper.createTestFolder(ExceptionHandlingTestHelper.getPageBlobTestStorageAccount(),
-        testFolderPath);
-    fs.delete(testFolderPath, true);
-    fs.listStatus(testFolderPath);
-  }
-
-  @Test
-  /*
-   * Test basic single threaded listStatus scenario
-   */
-  public void testSingleThreadedBlockBlobRenameScenario() throws Throwable {
-
-    ExceptionHandlingTestHelper.createEmptyFile(createTestAccount(),
-        testPath);
-    Path dstPath = new Path("dstFile.dat");
-    fs.delete(testPath, true);
-    boolean renameResult = fs.rename(testPath, dstPath);
-    Assert.assertFalse(renameResult);
-  }
-
-  @Test
-  /*
-   * Test basic single threaded listStatus scenario
-   */
-  public void testSingleThreadedPageBlobRenameScenario() throws Throwable {
-
-    ExceptionHandlingTestHelper.createEmptyFile(ExceptionHandlingTestHelper.getPageBlobTestStorageAccount(),
-        testPath);
-    Path dstPath = new Path("dstFile.dat");
-    fs.delete(testPath, true);
-    boolean renameResult = fs.rename(testPath, dstPath);
-    Assert.assertFalse(renameResult);
-  }
-
-  @Test
-  /*
-   * Test basic single threaded listStatus scenario
-   */
-  public void testSingleThreadedBlockBlobDeleteScenario() throws Throwable {
-
-    ExceptionHandlingTestHelper.createEmptyFile(createTestAccount(),
-        testPath);
-    fs.delete(testPath, true);
-    boolean deleteResult = fs.delete(testPath, true);
-    Assert.assertFalse(deleteResult);
-  }
-
-  @Test
-  /*
-   * Test basic single threaded listStatus scenario
-   */
-  public void testSingleThreadedPageBlobDeleteScenario() throws Throwable {
-
-    ExceptionHandlingTestHelper.createEmptyFile(ExceptionHandlingTestHelper.getPageBlobTestStorageAccount(),
-        testPath);
-    fs.delete(testPath, true);
-    boolean deleteResult = fs.delete(testPath, true);
-    Assert.assertFalse(deleteResult);
-  }
-
-  @Test(expected=FileNotFoundException.class)
-  /*
-   * Test basic single threaded listStatus scenario
-   */
-  public void testSingleThreadedBlockBlobOpenScenario() throws Throwable {
-
-    ExceptionHandlingTestHelper.createEmptyFile(createTestAccount(),
-        testPath);
-    fs.delete(testPath, true);
-    inputStream = fs.open(testPath);
-  }
-
-  @Test(expected=FileNotFoundException.class)
-  /*
-   * Test basic single threaded listStatus scenario
-   */
-  public void testSingleThreadedPageBlobOpenScenario() throws Throwable {
-
-    ExceptionHandlingTestHelper.createEmptyFile(ExceptionHandlingTestHelper.getPageBlobTestStorageAccount(),
-        testPath);
-    fs.delete(testPath, true);
-    inputStream = fs.open(testPath);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (inputStream != null) {
-      inputStream.close();
-    }
-
-    if (fs != null && fs.exists(testPath)) {
-      fs.delete(testPath, true);
-    }
-  }
-
-  @Override
-  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
-    return AzureBlobStorageTestAccount.create();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationExceptionMessage.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationExceptionMessage.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationExceptionMessage.java
deleted file mode 100644
index e619817..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationExceptionMessage.java
+++ /dev/null
@@ -1,79 +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.azure;
-import java.net.URI;
-import java.util.UUID;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.test.GenericTestUtils;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.NO_ACCESS_TO_CONTAINER_MSG;
-
-
-public class TestFileSystemOperationExceptionMessage extends
-  NativeAzureFileSystemBaseTest {
-
-  @Test
-  public void testAnonymouseCredentialExceptionMessage() throws Throwable{
-
-    Configuration conf = AzureBlobStorageTestAccount.createTestConfiguration();
-    String testStorageAccount = conf.get("fs.azure.test.account.name");
-    conf = new Configuration();
-    conf.set("fs.AbstractFileSystem.wasb.impl", "org.apache.hadoop.fs.azure.Wasb");
-    conf.set("fs.azure.skip.metrics", "true");
-
-    String testContainer = UUID.randomUUID().toString();
-    String wasbUri = String.format("wasb://%s@%s",
-        testContainer, testStorageAccount);
-
-    fs = new NativeAzureFileSystem();
-    try {
-      fs.initialize(new URI(wasbUri), conf);
-    } catch (Exception ex) {
-
-      Throwable innerException = ex.getCause();
-      while (innerException != null
-             && !(innerException instanceof AzureException)) {
-        innerException = innerException.getCause();
-      }
-
-      if (innerException != null) {
-        String exceptionMessage = innerException.getMessage();
-        if (exceptionMessage == null
-            || exceptionMessage.length() == 0) {
-          Assert.fail();}
-        else {
-          GenericTestUtils.assertExceptionContains(String.format(
-              NO_ACCESS_TO_CONTAINER_MSG, testStorageAccount, testContainer),
-              ex);
-        }
-      } else {
-        Assert.fail();
-      }
-    }
-  }
-
-  @Override
-  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
-    return AzureBlobStorageTestAccount.create();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationsExceptionHandlingMultiThreaded.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationsExceptionHandlingMultiThreaded.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationsExceptionHandlingMultiThreaded.java
deleted file mode 100644
index 1cd18ee..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationsExceptionHandlingMultiThreaded.java
+++ /dev/null
@@ -1,330 +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.azure;
-
-import java.io.FileNotFoundException;
-
-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.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.junit.After;
-import org.junit.Test;
-
-public class TestFileSystemOperationsExceptionHandlingMultiThreaded
-    extends AbstractWasbTestBase {
-
-  FSDataInputStream inputStream = null;
-
-  private static Path testPath = new Path("testfile.dat");
-  private static Path testFolderPath = new Path("testfolder");
-
-
-  /*
-   * Helper method to creates an input stream to test various scenarios.
-   */
-  private void getInputStreamToTest(FileSystem fs, Path testPath) throws Throwable {
-
-    FSDataOutputStream outputStream = fs.create(testPath);
-    String testString = "This is a test string";
-    outputStream.write(testString.getBytes());
-    outputStream.close();
-
-    inputStream = fs.open(testPath);
-  }
-
-  /*
-   * Test to validate correct exception is thrown for Multithreaded read
-   * scenario for block blobs
-   */
-  @Test(expected=FileNotFoundException.class)
-  public void testMultiThreadedBlockBlobReadScenario() throws Throwable {
-
-    AzureBlobStorageTestAccount testAccount = createTestAccount();
-    fs = testAccount.getFileSystem();
-    Path testFilePath1 = new Path("test1.dat");
-
-    getInputStreamToTest(fs, testFilePath1);
-    Thread renameThread = new Thread(new RenameThread(fs, testFilePath1));
-    renameThread.start();
-
-    renameThread.join();
-
-    byte[] readBuffer = new byte[512];
-    inputStream.read(readBuffer);
-  }
-
-  /*
-   * Test to validate correct exception is thrown for Multithreaded seek
-   * scenario for block blobs
-   */
-
-  @Test(expected=FileNotFoundException.class)
-  public void testMultiThreadBlockBlobSeekScenario() throws Throwable {
-
-    AzureBlobStorageTestAccount testAccount = createTestAccount();
-    fs = testAccount.getFileSystem();
-    Path testFilePath1 = new Path("test1.dat");
-
-    getInputStreamToTest(fs, testFilePath1);
-    Thread renameThread = new Thread(new RenameThread(fs, testFilePath1));
-    renameThread.start();
-
-    renameThread.join();
-
-    inputStream.seek(5);
-    inputStream.read();
-  }
-
-  @Test(expected=FileNotFoundException.class)
-  /*
-   * Tests basic multi threaded setPermission scenario
-   */
-  public void testMultiThreadedPageBlobSetPermissionScenario() throws Throwable {
-    ExceptionHandlingTestHelper.createEmptyFile(ExceptionHandlingTestHelper.getPageBlobTestStorageAccount(),
-        testPath);
-    Thread t = new Thread(new DeleteThread(fs, testPath));
-    t.start();
-    while (t.isAlive()) {
-      fs.setPermission(testPath, new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
-    }
-    fs.setPermission(testPath, new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
-  }
-
-  @Test(expected=FileNotFoundException.class)
-  /*
-   * Tests basic multi threaded setPermission scenario
-   */
-  public void testMultiThreadedBlockBlobSetPermissionScenario() throws Throwable {
-    ExceptionHandlingTestHelper.createEmptyFile(createTestAccount(),
-        testPath);
-    Thread t = new Thread(new DeleteThread(fs, testPath));
-    t.start();
-    while (t.isAlive()) {
-      fs.setPermission(testPath, new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
-    }
-    fs.setPermission(testPath, new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
-  }
-
-  @Test(expected=FileNotFoundException.class)
-  /*
-   * Tests basic multi threaded setPermission scenario
-   */
-  public void testMultiThreadedPageBlobOpenScenario() throws Throwable {
-
-    ExceptionHandlingTestHelper.createEmptyFile(createTestAccount(),
-        testPath);
-    Thread t = new Thread(new DeleteThread(fs, testPath));
-    t.start();
-    while (t.isAlive()) {
-      inputStream = fs.open(testPath);
-      inputStream.close();
-    }
-
-    inputStream = fs.open(testPath);
-    inputStream.close();
-  }
-
-  @Test(expected=FileNotFoundException.class)
-  /*
-   * Tests basic multi threaded setPermission scenario
-   */
-  public void testMultiThreadedBlockBlobOpenScenario() throws Throwable {
-
-    ExceptionHandlingTestHelper.createEmptyFile(ExceptionHandlingTestHelper.getPageBlobTestStorageAccount(),
-        testPath);
-    Thread t = new Thread(new DeleteThread(fs, testPath));
-    t.start();
-
-    while (t.isAlive()) {
-      inputStream = fs.open(testPath);
-      inputStream.close();
-    }
-    inputStream = fs.open(testPath);
-    inputStream.close();
-  }
-
-  @Test(expected=FileNotFoundException.class)
-  /*
-   * Tests basic multi threaded setOwner scenario
-   */
-  public void testMultiThreadedBlockBlobSetOwnerScenario() throws Throwable {
-
-    ExceptionHandlingTestHelper.createEmptyFile(createTestAccount(), testPath);
-    Thread t = new Thread(new DeleteThread(fs, testPath));
-    t.start();
-    while (t.isAlive()) {
-      fs.setOwner(testPath, "testowner", "testgroup");
-    }
-    fs.setOwner(testPath, "testowner", "testgroup");
-  }
-
-  @Test(expected=FileNotFoundException.class)
-  /*
-   * Tests basic multi threaded setOwner scenario
-   */
-  public void testMultiThreadedPageBlobSetOwnerScenario() throws Throwable {
-    ExceptionHandlingTestHelper.createEmptyFile(ExceptionHandlingTestHelper.getPageBlobTestStorageAccount(),
-        testPath);
-    Thread t = new Thread(new DeleteThread(fs, testPath));
-    t.start();
-    while (t.isAlive()) {
-      fs.setOwner(testPath, "testowner", "testgroup");
-    }
-    fs.setOwner(testPath, "testowner", "testgroup");
-  }
-
-  @Test(expected=FileNotFoundException.class)
-  /*
-   * Tests basic multi threaded listStatus scenario
-   */
-  public void testMultiThreadedBlockBlobListStatusScenario() throws Throwable {
-
-    ExceptionHandlingTestHelper.createTestFolder(createTestAccount(), testFolderPath);
-    Thread t = new Thread(new DeleteThread(fs, testFolderPath));
-    t.start();
-    while (t.isAlive()) {
-      fs.listStatus(testFolderPath);
-    }
-    fs.listStatus(testFolderPath);
-  }
-
-  @Test(expected=FileNotFoundException.class)
-  /*
-   * Tests basic multi threaded listStatus scenario
-   */
-  public void testMultiThreadedPageBlobListStatusScenario() throws Throwable {
-
-    ExceptionHandlingTestHelper.createTestFolder(ExceptionHandlingTestHelper.getPageBlobTestStorageAccount(),
-        testFolderPath);
-    Thread t = new Thread(new DeleteThread(fs, testFolderPath));
-    t.start();
-    while (t.isAlive()) {
-      fs.listStatus(testFolderPath);
-    }
-    fs.listStatus(testFolderPath);
-  }
-
-  /*
-   * Test to validate correct exception is thrown for Multithreaded read
-   * scenario for page blobs
-   */
-
-  @Test(expected=FileNotFoundException.class)
-  public void testMultiThreadedPageBlobReadScenario() throws Throwable {
-
-    AzureBlobStorageTestAccount testAccount = ExceptionHandlingTestHelper.getPageBlobTestStorageAccount();
-    fs = testAccount.getFileSystem();
-    Path testFilePath1 = new Path("test1.dat");
-
-    getInputStreamToTest(fs, testFilePath1);
-    Thread renameThread = new Thread(new RenameThread(fs, testFilePath1));
-    renameThread.start();
-
-    renameThread.join();
-    byte[] readBuffer = new byte[512];
-    inputStream.read(readBuffer);
-  }
-
-  /*
-   * Test to validate correct exception is thrown for Multithreaded seek
-   * scenario for page blobs
-   */
-
-  @Test(expected=FileNotFoundException.class)
-  public void testMultiThreadedPageBlobSeekScenario() throws Throwable {
-
-    AzureBlobStorageTestAccount testAccount = ExceptionHandlingTestHelper.getPageBlobTestStorageAccount();
-    fs = testAccount.getFileSystem();
-    Path testFilePath1 = new Path("test1.dat");
-
-    getInputStreamToTest(fs, testFilePath1);
-    Thread renameThread = new Thread(new RenameThread(fs, testFilePath1));
-    renameThread.start();
-
-    renameThread.join();
-    inputStream.seek(5);
-  }
-
-  @Override
-  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
-    return AzureBlobStorageTestAccount.create();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-
-    if (inputStream != null) {
-      inputStream.close();
-    }
-
-    if (fs != null && fs.exists(testPath)) {
-      fs.delete(testPath, true);
-    }
-  }
-}
-
-/*
- * Helper thread that just renames the test file.
- */
-class RenameThread implements Runnable {
-
-  private FileSystem fs;
-  private Path testPath;
-  private Path renamePath = new Path("test2.dat");
-
-  public RenameThread(FileSystem fs, Path testPath) {
-    this.fs = fs;
-    this.testPath = testPath;
-  }
-
-  @Override
-  public void run(){
-    try {
-      fs.rename(testPath, renamePath);
-    }catch (Exception e) {
-      // Swallowing the exception as the
-      // correctness of the test is controlled
-      // by the other thread
-    }
-  }
-}
-
-class DeleteThread implements Runnable {
-  private FileSystem fs;
-  private Path testPath;
-
-  public DeleteThread(FileSystem fs, Path testPath) {
-    this.fs = fs;
-    this.testPath = testPath;
-  }
-
-  @Override
-  public void run() {
-    try {
-      fs.delete(testPath, true);
-    } catch (Exception e) {
-      // Swallowing the exception as the
-      // correctness of the test is controlled
-      // by the other thread
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationsWithThreads.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationsWithThreads.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationsWithThreads.java
deleted file mode 100644
index fd3690c..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestFileSystemOperationsWithThreads.java
+++ /dev/null
@@ -1,821 +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.azure;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.logging.impl.Log4JLogger;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azure.NativeAzureFileSystem.FolderRenamePending;
-import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Tests the Native Azure file system (WASB) using parallel threads for rename and delete operations.
- */
-public class TestFileSystemOperationsWithThreads extends AbstractWasbTestBase {
-
-  private final int renameThreads = 10;
-  private final int deleteThreads = 20;
-  private int iterations = 1;
-  private LogCapturer logs = null;
-
-  @Rule
-  public ExpectedException exception = ExpectedException.none();
-
-  @Before
-  public void setUp() throws Exception {
-    super.setUp();
-    Configuration conf = fs.getConf();
-
-    // By default enable parallel threads for rename and delete operations.
-    // Also enable flat listing of blobs for these operations.
-    conf.setInt(NativeAzureFileSystem.AZURE_RENAME_THREADS, renameThreads);
-    conf.setInt(NativeAzureFileSystem.AZURE_DELETE_THREADS, deleteThreads);
-    conf.setBoolean(AzureNativeFileSystemStore.KEY_ENABLE_FLAT_LISTING, true);
-
-    URI uri = fs.getUri();
-    fs.initialize(uri, conf);
-
-    // Capture logs
-    logs = LogCapturer.captureLogs(new Log4JLogger(org.apache.log4j.Logger
-        .getRootLogger()));
-  }
-
-  /*
-   * Helper method to create sub directory and different types of files
-   * for multiple iterations.
-   */
-  private void createFolder(FileSystem fs, String root) throws Exception {
-    fs.mkdirs(new Path(root));
-    for (int i = 0; i < this.iterations; i++) {
-      fs.mkdirs(new Path(root + "/" + i));
-      fs.createNewFile(new Path(root + "/" + i + "/fileToRename"));
-      fs.createNewFile(new Path(root + "/" + i + "/file/to/rename"));
-      fs.createNewFile(new Path(root + "/" + i + "/file+to%rename"));
-      fs.createNewFile(new Path(root + "/fileToRename" + i));
-    }
-  }
-
-  /*
-   * Helper method to do rename operation and validate all files in source folder
-   * doesn't exists and similar files exists in new folder.
-   */
-  private void validateRenameFolder(FileSystem fs, String source, String dest) throws Exception {
-    // Create source folder with files.
-    createFolder(fs, source);
-    Path sourceFolder = new Path(source);
-    Path destFolder = new Path(dest);
-
-    // rename operation
-    assertTrue(fs.rename(sourceFolder, destFolder));
-    assertTrue(fs.exists(destFolder));
-
-    for (int i = 0; i < this.iterations; i++) {
-      // Check destination folder and files exists.
-      assertTrue(fs.exists(new Path(dest + "/" + i)));
-      assertTrue(fs.exists(new Path(dest + "/" + i + "/fileToRename")));
-      assertTrue(fs.exists(new Path(dest + "/" + i + "/file/to/rename")));
-      assertTrue(fs.exists(new Path(dest + "/" + i + "/file+to%rename")));
-      assertTrue(fs.exists(new Path(dest + "/fileToRename" + i)));
-
-      // Check source folder and files doesn't exists.
-      assertFalse(fs.exists(new Path(source + "/" + i)));
-      assertFalse(fs.exists(new Path(source + "/" + i + "/fileToRename")));
-      assertFalse(fs.exists(new Path(source + "/" + i + "/file/to/rename")));
-      assertFalse(fs.exists(new Path(source + "/" + i + "/file+to%rename")));
-      assertFalse(fs.exists(new Path(source + "/fileToRename" + i)));
-    }
-  }
-
-  /*
-   * Test case for rename operation with multiple threads and flat listing enabled.
-   */
-  @Test
-  public void testRenameSmallFolderWithThreads() throws Exception {
-
-    validateRenameFolder(fs, "root", "rootnew");
-
-    // With single iteration, we would have created 7 blobs.
-    int expectedThreadsCreated = Math.min(7, renameThreads);
-
-    // Validate from logs that threads are created.
-    String content = logs.getOutput();
-    assertInLog(content, "ms with threads: " + expectedThreadsCreated);
-
-    // Validate thread executions
-    for (int i = 0; i < expectedThreadsCreated; i++) {
-      assertInLog(content,
-          "AzureBlobRenameThread-" + Thread.currentThread().getName() + "-" + i);
-    }
-
-    // Also ensure that we haven't spawned extra threads.
-    if (expectedThreadsCreated < renameThreads) {
-      for (int i = expectedThreadsCreated; i < renameThreads; i++) {
-        assertNotInLog(content,
-            "AzureBlobRenameThread-" + Thread.currentThread().getName() + "-" + i);
-      }
-    }
-  }
-
-  /*
-   * Test case for rename operation with multiple threads and flat listing enabled.
-   */
-  @Test
-  public void testRenameLargeFolderWithThreads() throws Exception {
-
-    // Populate source folder with large number of files and directories.
-    this.iterations = 10;
-    validateRenameFolder(fs, "root", "rootnew");
-
-    // Validate from logs that threads are created.
-    String content = logs.getOutput();
-    assertInLog(content, "ms with threads: " + renameThreads);
-
-    // Validate thread executions
-    for (int i = 0; i < renameThreads; i++) {
-      assertInLog(content,
-          "AzureBlobRenameThread-" + Thread.currentThread().getName() + "-" + i);
-    }
-  }
-
-  /*
-   * Test case for rename operation with threads disabled and flat listing enabled.
-   */
-  @Test
-  public void testRenameLargeFolderDisableThreads() throws Exception {
-    Configuration conf = fs.getConf();
-
-    // Number of threads set to 0 or 1 disables threads.
-    conf.setInt(NativeAzureFileSystem.AZURE_RENAME_THREADS, 0);
-    URI uri = fs.getUri();
-    fs.initialize(uri, conf);
-
-    // Populate source folder with large number of files and directories.
-    this.iterations = 10;
-    validateRenameFolder(fs, "root", "rootnew");
-
-    // Validate from logs that threads are disabled.
-    String content = logs.getOutput();
-    assertInLog(content,
-        "Disabling threads for Rename operation as thread count 0");
-
-    // Validate no thread executions
-    for (int i = 0; i < renameThreads; i++) {
-      String term = "AzureBlobRenameThread-"
-          + Thread.currentThread().getName()
-          + "-" + i;
-      assertNotInLog(content, term);
-    }
-  }
-
-  /**
-   * Assert that a log contains the given term.
-   * @param content log output
-   * @param term search term
-   */
-  protected void assertInLog(String content, String term) {
-    assertTrue("Empty log", !content.isEmpty());
-    if (!content.contains(term)) {
-      String message = "No " + term + " found in logs";
-      LOG.error(message);
-      System.err.println(content);
-      fail(message);
-    }
-  }
-
-  /**
-   * Assert that a log does not contain the given term.
-   * @param content log output
-   * @param term search term
-   */
-  protected void assertNotInLog(String content, String term) {
-    assertTrue("Empty log", !content.isEmpty());
-    if (content.contains(term)) {
-      String message = term + " found in logs";
-      LOG.error(message);
-      System.err.println(content);
-      fail(message);
-    }
-  }
-
-  /*
-   * Test case for rename operation with threads and flat listing disabled.
-   */
-  @Test
-  public void testRenameSmallFolderDisableThreadsDisableFlatListing() throws Exception {
-    Configuration conf = fs.getConf();
-    conf = fs.getConf();
-
-    // Number of threads set to 0 or 1 disables threads.
-    conf.setInt(NativeAzureFileSystem.AZURE_RENAME_THREADS, 1);
-    conf.setBoolean(AzureNativeFileSystemStore.KEY_ENABLE_FLAT_LISTING, false);
-    URI uri = fs.getUri();
-    fs.initialize(uri, conf);
-
-    validateRenameFolder(fs, "root", "rootnew");
-
-    // Validate from logs that threads are disabled.
-    String content = logs.getOutput();
-    assertInLog(content,
-        "Disabling threads for Rename operation as thread count 1");
-
-    // Validate no thread executions
-    for (int i = 0; i < renameThreads; i++) {
-      assertNotInLog(content,
-          "AzureBlobRenameThread-" + Thread.currentThread().getName() + "-" + i);
-    }
-  }
-
-  /*
-   * Helper method to do delete operation and validate all files in source folder
-   * doesn't exists after delete operation.
-   */
-  private void validateDeleteFolder(FileSystem fs, String source)  throws Exception {
-    // Create folder with files.
-    createFolder(fs, "root");
-    Path sourceFolder = new Path(source);
-
-    // Delete operation
-    assertTrue(fs.delete(sourceFolder, true));
-    assertFalse(fs.exists(sourceFolder));
-
-    for (int i = 0; i < this.iterations; i++) {
-      // check that source folder and files doesn't exists
-      assertFalse(fs.exists(new Path(source + "/" + i)));
-      assertFalse(fs.exists(new Path(source + "/" + i + "/fileToRename")));
-      assertFalse(fs.exists(new Path(source + "/" + i + "/file/to/rename")));
-      assertFalse(fs.exists(new Path(source + "/" + i + "/file+to%rename")));
-      assertFalse(fs.exists(new Path(source + "/fileToRename" + i)));
-    }
-  }
-
-  /*
-   * Test case for delete operation with multiple threads and flat listing enabled.
-   */
-  @Test
-  public void testDeleteSmallFolderWithThreads() throws Exception {
-
-    validateDeleteFolder(fs, "root");
-
-    // With single iteration, we would have created 7 blobs.
-    int expectedThreadsCreated = Math.min(7, deleteThreads);
-
-    // Validate from logs that threads are enabled.
-    String content = logs.getOutput();
-    assertInLog(content, "ms with threads: " + expectedThreadsCreated);
-
-    // Validate thread executions
-    for (int i = 0; i < expectedThreadsCreated; i++) {
-      assertInLog(content,
-          "AzureBlobDeleteThread-" + Thread.currentThread().getName() + "-" + i);
-    }
-
-    // Also ensure that we haven't spawned extra threads.
-    if (expectedThreadsCreated < deleteThreads) {
-      for (int i = expectedThreadsCreated; i < deleteThreads; i++) {
-        assertNotInLog(content,
-            "AzureBlobDeleteThread-" + Thread.currentThread().getName() + "-" + i);
-      }
-    }
-  }
-
-  /*
-   * Test case for delete operation with multiple threads and flat listing enabled.
-   */
-  @Test
-  public void testDeleteLargeFolderWithThreads() throws Exception {
-    // Populate source folder with large number of files and directories.
-    this.iterations = 10;
-    validateDeleteFolder(fs, "root");
-
-    // Validate from logs that threads are enabled.
-    String content = logs.getOutput();
-    assertInLog(content, "ms with threads: " + deleteThreads);
-
-    // Validate thread executions
-    for (int i = 0; i < deleteThreads; i++) {
-      assertInLog(content,
-          "AzureBlobDeleteThread-" + Thread.currentThread().getName() + "-" + i);
-    }
-  }
-
-  /*
-   * Test case for delete operation with threads disabled and flat listing enabled.
-   */
-  @Test
-  public void testDeleteLargeFolderDisableThreads() throws Exception {
-    Configuration conf = fs.getConf();
-    conf.setInt(NativeAzureFileSystem.AZURE_DELETE_THREADS, 0);
-    URI uri = fs.getUri();
-    fs.initialize(uri, conf);
-
-    // Populate source folder with large number of files and directories.
-    this.iterations = 10;
-    validateDeleteFolder(fs, "root");
-
-    // Validate from logs that threads are disabled.
-    String content = logs.getOutput();
-    assertInLog(content,
-        "Disabling threads for Delete operation as thread count 0");
-
-    // Validate no thread executions
-    for (int i = 0; i < deleteThreads; i++) {
-      assertNotInLog(content,
-          "AzureBlobDeleteThread-" + Thread.currentThread().getName() + "-" + i);
-    }
-  }
-
-  /*
-   * Test case for rename operation with threads and flat listing disabled.
-   */
-  @Test
-  public void testDeleteSmallFolderDisableThreadsDisableFlatListing() throws Exception {
-    Configuration conf = fs.getConf();
-
-    // Number of threads set to 0 or 1 disables threads.
-    conf.setInt(NativeAzureFileSystem.AZURE_DELETE_THREADS, 1);
-    conf.setBoolean(AzureNativeFileSystemStore.KEY_ENABLE_FLAT_LISTING, false);
-    URI uri = fs.getUri();
-    fs.initialize(uri, conf);
-
-    validateDeleteFolder(fs, "root");
-
-    // Validate from logs that threads are disabled.
-    String content = logs.getOutput();
-    assertInLog(content,
-        "Disabling threads for Delete operation as thread count 1");
-
-    // Validate no thread executions
-    for (int i = 0; i < deleteThreads; i++) {
-      assertNotInLog(content,
-          "AzureBlobDeleteThread-" + Thread.currentThread().getName() + "-" + i);
-    }
-  }
-
-  /*
-   * Test case for delete operation with multiple threads and flat listing enabled.
-   */
-  @Test
-  public void testDeleteThreadPoolExceptionFailure() throws Exception {
-
-    // Spy azure file system object and raise exception for new thread pool
-    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
-
-    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root")));
-
-    AzureFileSystemThreadPoolExecutor mockThreadPoolExecutor = Mockito.spy(
-        mockFs.getThreadPoolExecutor(deleteThreads, "AzureBlobDeleteThread", "Delete",
-            path, NativeAzureFileSystem.AZURE_DELETE_THREADS));
-    Mockito.when(mockThreadPoolExecutor.getThreadPool(7)).thenThrow(new Exception());
-
-    // With single iteration, we would have created 7 blobs resulting 7 threads.
-    Mockito.when(mockFs.getThreadPoolExecutor(deleteThreads, "AzureBlobDeleteThread", "Delete",
-        path, NativeAzureFileSystem.AZURE_DELETE_THREADS)).thenReturn(mockThreadPoolExecutor);
-
-    validateDeleteFolder(mockFs, "root");
-
-    // Validate from logs that threads are disabled.
-    String content = logs.getOutput();
-    assertInLog(content, "Failed to create thread pool with threads");
-    assertInLog(content, "Serializing the Delete operation");
-  }
-
-  /*
-   * Test case for delete operation with multiple threads and flat listing enabled.
-   */
-  @Test
-  public void testDeleteThreadPoolExecuteFailure() throws Exception {
-
-    // Mock thread pool executor to throw exception for all requests.
-    ThreadPoolExecutor mockThreadExecutor = Mockito.mock(ThreadPoolExecutor.class);
-    Mockito.doThrow(new RejectedExecutionException()).when(mockThreadExecutor).execute(Mockito.any(Runnable.class));
-
-    // Spy azure file system object and return mocked thread pool
-    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
-
-    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root")));
-
-    AzureFileSystemThreadPoolExecutor mockThreadPoolExecutor = Mockito.spy(
-        mockFs.getThreadPoolExecutor(deleteThreads, "AzureBlobDeleteThread", "Delete",
-            path, NativeAzureFileSystem.AZURE_DELETE_THREADS));
-    Mockito.when(mockThreadPoolExecutor.getThreadPool(7)).thenReturn(mockThreadExecutor);
-
-    // With single iteration, we would have created 7 blobs resulting 7 threads.
-    Mockito.when(mockFs.getThreadPoolExecutor(deleteThreads, "AzureBlobDeleteThread", "Delete",
-        path, NativeAzureFileSystem.AZURE_DELETE_THREADS)).thenReturn(mockThreadPoolExecutor);
-
-    validateDeleteFolder(mockFs, "root");
-
-    // Validate from logs that threads are disabled.
-    String content = logs.getOutput();
-    assertInLog(content,
-        "Rejected execution of thread for Delete operation on blob");
-    assertInLog(content, "Serializing the Delete operation");
-  }
-
-  /*
-   * Test case for delete operation with multiple threads and flat listing enabled.
-   */
-  @Test
-  public void testDeleteThreadPoolExecuteSingleThreadFailure() throws Exception {
-
-    // Spy azure file system object and return mocked thread pool
-    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
-
-    // Spy a thread pool executor and link it to azure file system object.
-    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root")));
-    AzureFileSystemThreadPoolExecutor mockThreadPoolExecutor = Mockito.spy(
-        mockFs.getThreadPoolExecutor(deleteThreads, "AzureBlobDeleteThread", "Delete",
-            path, NativeAzureFileSystem.AZURE_DELETE_THREADS));
-
-    // With single iteration, we would have created 7 blobs resulting 7 threads.
-    Mockito.when(mockFs.getThreadPoolExecutor(deleteThreads, "AzureBlobDeleteThread", "Delete",
-        path, NativeAzureFileSystem.AZURE_DELETE_THREADS)).thenReturn(mockThreadPoolExecutor);
-
-    // Create a thread executor and link it to mocked thread pool executor object.
-    ThreadPoolExecutor mockThreadExecutor = Mockito.spy(mockThreadPoolExecutor.getThreadPool(7));
-    Mockito.when(mockThreadPoolExecutor.getThreadPool(7)).thenReturn(mockThreadExecutor);
-
-    // Mock thread executor to throw exception for all requests.
-    Mockito.doCallRealMethod().doThrow(new RejectedExecutionException()).when(mockThreadExecutor).execute(Mockito.any(Runnable.class));
-
-    validateDeleteFolder(mockFs, "root");
-
-    // Validate from logs that threads are enabled and unused threads.
-    String content = logs.getOutput();
-    assertInLog(content,
-        "Using thread pool for Delete operation with threads 7");
-    assertInLog(content,
-        "6 threads not used for Delete operation on blob");
-  }
-
-  /*
-   * Test case for delete operation with multiple threads and flat listing enabled.
-   */
-  @Test
-  public void testDeleteThreadPoolTerminationFailure() throws Exception {
-
-    // Spy azure file system object and return mocked thread pool
-    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
-
-    // Spy a thread pool executor and link it to azure file system object.
-    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root")));
-    AzureFileSystemThreadPoolExecutor mockThreadPoolExecutor = Mockito.spy(
-        ((NativeAzureFileSystem) fs).getThreadPoolExecutor(deleteThreads, "AzureBlobDeleteThread", "Delete",
-            path, NativeAzureFileSystem.AZURE_DELETE_THREADS));
-
-    // Create a thread executor and link it to mocked thread pool executor object.
-    // Mock thread executor to throw exception for terminating threads.
-    ThreadPoolExecutor mockThreadExecutor = Mockito.mock(ThreadPoolExecutor.class);
-    Mockito.doNothing().when(mockThreadExecutor).execute(Mockito.any(Runnable.class));
-    Mockito.when(mockThreadExecutor.awaitTermination(Long.MAX_VALUE, TimeUnit.DAYS)).thenThrow(new InterruptedException());
-
-    Mockito.when(mockThreadPoolExecutor.getThreadPool(7)).thenReturn(mockThreadExecutor);
-
-    // With single iteration, we would have created 7 blobs resulting 7 threads.
-    Mockito.when(mockFs.getThreadPoolExecutor(deleteThreads, "AzureBlobDeleteThread", "Delete",
-        path, NativeAzureFileSystem.AZURE_DELETE_THREADS)).thenReturn(mockThreadPoolExecutor);
-
-    createFolder(mockFs, "root");
-    Path sourceFolder = new Path("root");
-    boolean exception = false;
-    try {
-      mockFs.delete(sourceFolder, true);
-    } catch (IOException e){
-      exception = true;
-    }
-
-    assertTrue(exception);
-    assertTrue(mockFs.exists(sourceFolder));
-
-    // Validate from logs that threads are enabled and delete operation is failed.
-    String content = logs.getOutput();
-    assertInLog(content,
-        "Using thread pool for Delete operation with threads");
-    assertInLog(content, "Threads got interrupted Delete blob operation");
-    assertInLog(content,
-        "Delete failed as operation on subfolders and files failed.");
-  }
-
-  /*
-   * Validate that when a directory is deleted recursively, the operation succeeds
-   * even if a child directory delete fails because the directory does not exist.
-   * This can happen if a child directory is deleted by an external agent while
-   * the parent is in progress of being deleted recursively.
-   */
-  @Test
-  public void testRecursiveDirectoryDeleteWhenChildDirectoryDeleted()
-      throws Exception {
-    testRecusiveDirectoryDelete(true);
-  }
-
-  /*
-   * Validate that when a directory is deleted recursively, the operation succeeds
-   * even if a file delete fails because it does not exist.
-   * This can happen if a file is deleted by an external agent while
-   * the parent directory is in progress of being deleted.
-   */
-  @Test
-  public void testRecursiveDirectoryDeleteWhenDeletingChildFileReturnsFalse()
-      throws Exception {
-    testRecusiveDirectoryDelete(false);
-  }
-
-  private void testRecusiveDirectoryDelete(boolean useDir) throws Exception {
-    String childPathToBeDeletedByExternalAgent = (useDir)
-        ? "root/0"
-        : "root/0/fileToRename";
-    // Spy azure file system object and return false for deleting one file
-    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
-    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path(
-        childPathToBeDeletedByExternalAgent)));
-
-    Answer<Boolean> answer = new Answer<Boolean>() {
-      public Boolean answer(InvocationOnMock invocation) throws Throwable {
-        String path = (String) invocation.getArguments()[0];
-        boolean isDir = (boolean) invocation.getArguments()[1];
-        boolean realResult = fs.deleteFile(path, isDir);
-        assertTrue(realResult);
-        boolean fakeResult = false;
-        return fakeResult;
-      }
-    };
-
-    Mockito.when(mockFs.deleteFile(path, useDir)).thenAnswer(answer);
-
-    createFolder(mockFs, "root");
-    Path sourceFolder = new Path("root");
-
-    assertTrue(mockFs.delete(sourceFolder, true));
-    assertFalse(mockFs.exists(sourceFolder));
-
-    // Validate from logs that threads are enabled, that a child directory was
-    // deleted by an external caller, and the parent delete operation still
-    // succeeds.
-    String content = logs.getOutput();
-    assertInLog(content,
-        "Using thread pool for Delete operation with threads");
-    assertInLog(content, String.format("Attempt to delete non-existent %s %s",
-        useDir ? "directory" : "file", path));
-  }
-
-  /*
-   * Test case for delete operation with multiple threads and flat listing enabled.
-   */
-  @Test
-  public void testDeleteSingleDeleteException() throws Exception {
-
-    // Spy azure file system object and raise exception for deleting one file
-    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
-    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root/0")));
-    Mockito.doThrow(new IOException()).when(mockFs).deleteFile(path, true);
-
-    createFolder(mockFs, "root");
-    Path sourceFolder = new Path("root");
-
-    boolean exception = false;
-    try {
-      mockFs.delete(sourceFolder, true);
-    } catch (IOException e){
-      exception = true;
-    }
-
-    assertTrue(exception);
-    assertTrue(mockFs.exists(sourceFolder));
-
-    // Validate from logs that threads are enabled and delete operation failed.
-    String content = logs.getOutput();
-    assertInLog(content,
-        "Using thread pool for Delete operation with threads");
-    assertInLog(content,
-        "Encountered Exception for Delete operation for file " + path);
-    assertInLog(content,
-        "Terminating execution of Delete operation now as some other thread already got exception or operation failed");
-  }
-
-  /*
-   * Test case for rename operation with multiple threads and flat listing enabled.
-   */
-  @Test
-  public void testRenameThreadPoolExceptionFailure() throws Exception {
-
-    // Spy azure file system object and raise exception for new thread pool
-    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
-
-    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root")));
-    AzureFileSystemThreadPoolExecutor mockThreadPoolExecutor = Mockito.spy(
-        ((NativeAzureFileSystem) fs).getThreadPoolExecutor(renameThreads, "AzureBlobRenameThread", "Rename",
-            path, NativeAzureFileSystem.AZURE_RENAME_THREADS));
-    Mockito.when(mockThreadPoolExecutor.getThreadPool(7)).thenThrow(new Exception());
-
-    // With single iteration, we would have created 7 blobs resulting 7 threads.
-    Mockito.doReturn(mockThreadPoolExecutor).when(mockFs).getThreadPoolExecutor(renameThreads, "AzureBlobRenameThread", "Rename",
-        path, NativeAzureFileSystem.AZURE_RENAME_THREADS);
-
-    validateRenameFolder(mockFs, "root", "rootnew");
-
-    // Validate from logs that threads are disabled.
-    String content = logs.getOutput();
-    assertInLog(content, "Failed to create thread pool with threads");
-    assertInLog(content, "Serializing the Rename operation");
-  }
-
-  /*
-   * Test case for rename operation with multiple threads and flat listing enabled.
-   */
-  @Test
-  public void testRenameThreadPoolExecuteFailure() throws Exception {
-
-    // Mock thread pool executor to throw exception for all requests.
-    ThreadPoolExecutor mockThreadExecutor = Mockito.mock(ThreadPoolExecutor.class);
-    Mockito.doThrow(new RejectedExecutionException()).when(mockThreadExecutor).execute(Mockito.any(Runnable.class));
-
-    // Spy azure file system object and return mocked thread pool
-    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
-
-    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root")));
-    AzureFileSystemThreadPoolExecutor mockThreadPoolExecutor = Mockito.spy(
-        mockFs.getThreadPoolExecutor(renameThreads, "AzureBlobRenameThread", "Rename",
-            path, NativeAzureFileSystem.AZURE_RENAME_THREADS));
-    Mockito.when(mockThreadPoolExecutor.getThreadPool(7)).thenReturn(mockThreadExecutor);
-
-    // With single iteration, we would have created 7 blobs resulting 7 threads.
-    Mockito.when(mockFs.getThreadPoolExecutor(renameThreads, "AzureBlobRenameThread", "Rename",
-        path, NativeAzureFileSystem.AZURE_RENAME_THREADS)).thenReturn(mockThreadPoolExecutor);
-
-    validateRenameFolder(mockFs, "root", "rootnew");
-
-    // Validate from logs that threads are disabled.
-    String content = logs.getOutput();
-    assertInLog(content,
-        "Rejected execution of thread for Rename operation on blob");
-    assertInLog(content, "Serializing the Rename operation");
-  }
-
-  /*
-   * Test case for rename operation with multiple threads and flat listing enabled.
-   */
-  @Test
-  public void testRenameThreadPoolExecuteSingleThreadFailure() throws Exception {
-
-    // Spy azure file system object and return mocked thread pool
-    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
-
-    // Spy a thread pool executor and link it to azure file system object.
-    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root")));
-    AzureFileSystemThreadPoolExecutor mockThreadPoolExecutor = Mockito.spy(
-        mockFs.getThreadPoolExecutor(renameThreads, "AzureBlobRenameThread", "Rename",
-            path, NativeAzureFileSystem.AZURE_RENAME_THREADS));
-
-    // With single iteration, we would have created 7 blobs resulting 7 threads.
-    Mockito.when(mockFs.getThreadPoolExecutor(renameThreads, "AzureBlobRenameThread", "Rename",
-        path, NativeAzureFileSystem.AZURE_RENAME_THREADS)).thenReturn(mockThreadPoolExecutor);
-
-    // Create a thread executor and link it to mocked thread pool executor object.
-    ThreadPoolExecutor mockThreadExecutor = Mockito.spy(mockThreadPoolExecutor.getThreadPool(7));
-    Mockito.when(mockThreadPoolExecutor.getThreadPool(7)).thenReturn(mockThreadExecutor);
-
-    // Mock thread executor to throw exception for all requests.
-    Mockito.doCallRealMethod().doThrow(new RejectedExecutionException()).when(mockThreadExecutor).execute(Mockito.any(Runnable.class));
-
-    validateRenameFolder(mockFs, "root", "rootnew");
-
-    // Validate from logs that threads are enabled and unused threads exists.
-    String content = logs.getOutput();
-    assertInLog(content,
-        "Using thread pool for Rename operation with threads 7");
-    assertInLog(content,
-        "6 threads not used for Rename operation on blob");
-  }
-
-  /*
-   * Test case for rename operation with multiple threads and flat listing enabled.
-   */
-  @Test
-  public void testRenameThreadPoolTerminationFailure() throws Exception {
-
-    // Spy azure file system object and return mocked thread pool
-    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
-
-    // Spy a thread pool executor and link it to azure file system object.
-    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root")));
-    AzureFileSystemThreadPoolExecutor mockThreadPoolExecutor = Mockito.spy(
-        mockFs.getThreadPoolExecutor(renameThreads, "AzureBlobRenameThread", "Rename",
-            path, NativeAzureFileSystem.AZURE_RENAME_THREADS));
-
-    // With single iteration, we would have created 7 blobs resulting 7 threads.
-    Mockito.when(mockFs.getThreadPoolExecutor(renameThreads, "AzureBlobRenameThread", "Rename",
-        path, NativeAzureFileSystem.AZURE_RENAME_THREADS)).thenReturn(mockThreadPoolExecutor);
-
-    // Mock thread executor to throw exception for all requests.
-    ThreadPoolExecutor mockThreadExecutor = Mockito.mock(ThreadPoolExecutor.class);
-    Mockito.doNothing().when(mockThreadExecutor).execute(Mockito.any(Runnable.class));
-    Mockito.when(mockThreadExecutor.awaitTermination(Long.MAX_VALUE, TimeUnit.DAYS)).thenThrow(new InterruptedException());
-    Mockito.when(mockThreadPoolExecutor.getThreadPool(7)).thenReturn(mockThreadExecutor);
-
-
-    createFolder(mockFs, "root");
-    Path sourceFolder = new Path("root");
-    Path destFolder = new Path("rootnew");
-    boolean exception = false;
-    try {
-      mockFs.rename(sourceFolder, destFolder);
-    } catch (IOException e){
-      exception = true;
-    }
-
-    assertTrue(exception);
-    assertTrue(mockFs.exists(sourceFolder));
-
-    // Validate from logs that threads are enabled and rename operation is failed.
-    String content = logs.getOutput();
-    assertInLog(content,
-        "Using thread pool for Rename operation with threads");
-    assertInLog(content, "Threads got interrupted Rename blob operation");
-    assertInLog(content,
-        "Rename failed as operation on subfolders and files failed.");
-  }
-
-  /*
-   * Test case for rename operation with multiple threads and flat listing enabled.
-   */
-  @Test
-  public void testRenameSingleRenameException() throws Exception {
-
-    // Spy azure file system object and raise exception for deleting one file
-    Path sourceFolder = new Path("root");
-    Path destFolder = new Path("rootnew");
-
-    // Spy azure file system object and populate rename pending spy object.
-    NativeAzureFileSystem mockFs = Mockito.spy((NativeAzureFileSystem) fs);
-
-    // Populate data now only such that rename pending spy object would see this data.
-    createFolder(mockFs, "root");
-
-    String srcKey = mockFs.pathToKey(mockFs.makeAbsolute(sourceFolder));
-    String dstKey = mockFs.pathToKey(mockFs.makeAbsolute(destFolder));
-
-    FolderRenamePending mockRenameFs = Mockito.spy(mockFs.prepareAtomicFolderRename(srcKey, dstKey));
-    Mockito.when(mockFs.prepareAtomicFolderRename(srcKey, dstKey)).thenReturn(mockRenameFs);
-    String path = mockFs.pathToKey(mockFs.makeAbsolute(new Path("root/0")));
-    Mockito.doThrow(new IOException()).when(mockRenameFs).renameFile(Mockito.any(FileMetadata.class));
-
-    boolean exception = false;
-    try {
-      mockFs.rename(sourceFolder, destFolder);
-    } catch (IOException e){
-      exception = true;
-    }
-
-    assertTrue(exception);
-    assertTrue(mockFs.exists(sourceFolder));
-
-    // Validate from logs that threads are enabled and delete operation failed.
-    String content = logs.getOutput();
-    assertInLog(content,
-        "Using thread pool for Rename operation with threads");
-    assertInLog(content,
-        "Encountered Exception for Rename operation for file " + path);
-    assertInLog(content,
-        "Terminating execution of Rename operation now as some other thread already got exception or operation failed");
-  }
-
-  @Override
-  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
-    return AzureBlobStorageTestAccount.create();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFSAuthWithBlobSpecificKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFSAuthWithBlobSpecificKeys.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFSAuthWithBlobSpecificKeys.java
deleted file mode 100644
index 6149154..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFSAuthWithBlobSpecificKeys.java
+++ /dev/null
@@ -1,44 +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.azure;
-
-import org.apache.hadoop.conf.Configuration;
-
-import static org.apache.hadoop.fs.azure.SecureStorageInterfaceImpl.KEY_USE_CONTAINER_SASKEY_FOR_ALL_ACCESS;
-
-/**
- * Test class to hold all WASB authorization tests that use blob-specific keys
- * to access storage.
- */
-public class TestNativeAzureFSAuthWithBlobSpecificKeys
-    extends TestNativeAzureFileSystemAuthorizationWithOwner {
-
-  @Override
-  public Configuration getConfiguration() {
-    Configuration conf = super.getConfiguration();
-    conf.set(KEY_USE_CONTAINER_SASKEY_FOR_ALL_ACCESS, "false");
-    return conf;
-  }
-
-  @Override
-  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
-    Configuration conf = getConfiguration();
-    return AzureBlobStorageTestAccount.create(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFSAuthorizationCaching.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFSAuthorizationCaching.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFSAuthorizationCaching.java
deleted file mode 100644
index 84558f8..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFSAuthorizationCaching.java
+++ /dev/null
@@ -1,60 +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.azure;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.junit.Test;
-
-import static org.apache.hadoop.fs.azure.CachingAuthorizer.KEY_AUTH_SERVICE_CACHING_ENABLE;
-
-/**
- * Test class to hold all WASB authorization caching related tests.
- */
-public class TestNativeAzureFSAuthorizationCaching
-    extends TestNativeAzureFileSystemAuthorizationWithOwner {
-
-  private static final int DUMMY_TTL_VALUE = 5000;
-
-  @Override
-  public Configuration getConfiguration() {
-    Configuration conf = super.getConfiguration();
-    conf.set(KEY_AUTH_SERVICE_CACHING_ENABLE, "true");
-    return conf;
-  }
-
-  @Override
-  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
-    Configuration conf = getConfiguration();
-    return AzureBlobStorageTestAccount.create(conf);
-  }
-
-  /**
-   * Test to verify cache behavior -- assert that PUT overwrites value if present
-   */
-  @Test
-  public void testCachePut() throws Throwable {
-    CachingAuthorizer<String, Integer> cache = new CachingAuthorizer<>(DUMMY_TTL_VALUE, "TEST");
-    cache.init(getConfiguration());
-    cache.put("TEST", 1);
-    cache.put("TEST", 3);
-    int result = cache.get("TEST");
-    ContractTestUtils.assertTrue("Cache returned unexpected result", result == 3);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFSPageBlobLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFSPageBlobLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFSPageBlobLive.java
deleted file mode 100644
index 208cff3..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFSPageBlobLive.java
+++ /dev/null
@@ -1,43 +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.azure;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Run the base Azure file system tests strictly on page blobs to make sure fundamental
- * operations on page blob files and folders work as expected.
- * These operations include create, delete, rename, list, and so on.
- */
-public class TestNativeAzureFSPageBlobLive extends
-    NativeAzureFileSystemBaseTest {
-
-  @Override
-  protected AzureBlobStorageTestAccount createTestAccount()
-      throws Exception {
-    Configuration conf = new Configuration();
-
-    // Configure the page blob directories key so every file created is a page blob.
-    conf.set(AzureNativeFileSystemStore.KEY_PAGE_BLOB_DIRECTORIES, "/");
-
-    // Configure the atomic rename directories key so every folder will have
-    // atomic rename applied.
-    conf.set(AzureNativeFileSystemStore.KEY_ATOMIC_RENAME_DIRECTORIES, "/");
-    return AzureBlobStorageTestAccount.create(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAppend.java
deleted file mode 100644
index a2b35cb..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAppend.java
+++ /dev/null
@@ -1,362 +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.azure;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.Arrays;
-
-import org.apache.commons.lang.RandomStringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestNativeAzureFileSystemAppend extends AbstractWasbTestBase {
-
-  private static final String TEST_FILE = "test.dat";
-  private static final Path TEST_PATH = new Path(TEST_FILE);
-
-  private AzureBlobStorageTestAccount testAccount = null;
-
-  @Before
-  public void setUp() throws Exception {
-    super.setUp();
-    testAccount = createTestAccount();
-    fs = testAccount.getFileSystem();
-    Configuration conf = fs.getConf();
-    conf.setBoolean(NativeAzureFileSystem.APPEND_SUPPORT_ENABLE_PROPERTY_NAME, true);
-    URI uri = fs.getUri();
-    fs.initialize(uri, conf);
-  }
-
-  /*
-   * Helper method that creates test data of size provided by the
-   * "size" parameter.
-   */
-  private static byte[] getTestData(int size) {
-    byte[] testData = new byte[size];
-    System.arraycopy(RandomStringUtils.randomAlphabetic(size).getBytes(), 0, testData, 0, size);
-    return testData;
-  }
-
-  // Helper method to create file and write fileSize bytes of data on it.
-  private byte[] createBaseFileWithData(int fileSize, Path testPath) throws Throwable {
-
-    FSDataOutputStream createStream = null;
-    try {
-      createStream = fs.create(testPath);
-      byte[] fileData = null;
-
-      if (fileSize != 0) {
-        fileData = getTestData(fileSize);
-        createStream.write(fileData);
-      }
-      return fileData;
-    } finally {
-      if (createStream != null) {
-        createStream.close();
-      }
-    }
-  }
-
-  /*
-   * Helper method to verify a file data equal to "dataLength" parameter
-   */
-  private boolean verifyFileData(int dataLength, byte[] testData, int testDataIndex,
-      FSDataInputStream srcStream) {
-
-    try {
-
-      byte[] fileBuffer = new byte[dataLength];
-      byte[] testDataBuffer = new byte[dataLength];
-
-      int fileBytesRead = srcStream.read(fileBuffer);
-
-      if (fileBytesRead < dataLength) {
-        return false;
-      }
-
-      System.arraycopy(testData, testDataIndex, testDataBuffer, 0, dataLength);
-
-      if (!Arrays.equals(fileBuffer, testDataBuffer)) {
-        return false;
-      }
-
-      return true;
-
-    } catch (Exception ex) {
-      return false;
-    }
-
-  }
-
-  /*
-   * Helper method to verify Append on a testFile.
-   */
-  private boolean verifyAppend(byte[] testData, Path testFile) {
-
-    FSDataInputStream srcStream = null;
-    try {
-
-      srcStream = fs.open(testFile);
-      int baseBufferSize = 2048;
-      int testDataSize = testData.length;
-      int testDataIndex = 0;
-
-      while (testDataSize > baseBufferSize) {
-
-        if (!verifyFileData(baseBufferSize, testData, testDataIndex, srcStream)) {
-          return false;
-        }
-        testDataIndex += baseBufferSize;
-        testDataSize -= baseBufferSize;
-      }
-
-      if (!verifyFileData(testDataSize, testData, testDataIndex, srcStream)) {
-        return false;
-      }
-
-      return true;
-    } catch(Exception ex) {
-      return false;
-    } finally {
-      if (srcStream != null) {
-        try {
-          srcStream.close();
-        } catch(IOException ioe) {
-          // Swallowing
-        }
-      }
-    }
-  }
-
-  /*
-   * Test case to verify if an append on small size data works. This tests
-   * append E2E
-   */
-  @Test
-  public void testSingleAppend() throws Throwable{
-
-    FSDataOutputStream appendStream = null;
-    try {
-      int baseDataSize = 50;
-      byte[] baseDataBuffer = createBaseFileWithData(baseDataSize, TEST_PATH);
-
-      int appendDataSize = 20;
-      byte[] appendDataBuffer = getTestData(appendDataSize);
-      appendStream = fs.append(TEST_PATH, 10);
-      appendStream.write(appendDataBuffer);
-      appendStream.close();
-      byte[] testData = new byte[baseDataSize + appendDataSize];
-      System.arraycopy(baseDataBuffer, 0, testData, 0, baseDataSize);
-      System.arraycopy(appendDataBuffer, 0, testData, baseDataSize, appendDataSize);
-
-      Assert.assertTrue(verifyAppend(testData, TEST_PATH));
-    } finally {
-      if (appendStream != null) {
-        appendStream.close();
-      }
-    }
-  }
-
-  /*
-   * Test case to verify append to an empty file.
-   */
-  @Test
-  public void testSingleAppendOnEmptyFile() throws Throwable {
-
-    FSDataOutputStream appendStream = null;
-
-    try {
-      createBaseFileWithData(0, TEST_PATH);
-
-      int appendDataSize = 20;
-      byte[] appendDataBuffer = getTestData(appendDataSize);
-      appendStream = fs.append(TEST_PATH, 10);
-      appendStream.write(appendDataBuffer);
-      appendStream.close();
-
-      Assert.assertTrue(verifyAppend(appendDataBuffer, TEST_PATH));
-    } finally {
-      if (appendStream != null) {
-        appendStream.close();
-      }
-    }
-  }
-
-  /*
-   * Test to verify that we can open only one Append stream on a File.
-   */
-  @Test
-  public void testSingleAppenderScenario() throws Throwable {
-
-    FSDataOutputStream appendStream1 = null;
-    FSDataOutputStream appendStream2 = null;
-    IOException ioe = null;
-    try {
-      createBaseFileWithData(0, TEST_PATH);
-      appendStream1 = fs.append(TEST_PATH, 10);
-      boolean encounteredException = false;
-      try {
-        appendStream2 = fs.append(TEST_PATH, 10);
-      } catch(IOException ex) {
-        encounteredException = true;
-        ioe = ex;
-      }
-
-      appendStream1.close();
-
-      Assert.assertTrue(encounteredException);
-      GenericTestUtils.assertExceptionContains("Unable to set Append lease on the Blob", ioe);
-    } finally {
-      if (appendStream1 != null) {
-        appendStream1.close();
-      }
-
-      if (appendStream2 != null) {
-        appendStream2.close();
-      }
-    }
-  }
-
-  /*
-   * Tests to verify multiple appends on a Blob.
-   */
-  @Test
-  public void testMultipleAppends() throws Throwable {
-
-    int baseDataSize = 50;
-    byte[] baseDataBuffer = createBaseFileWithData(baseDataSize, TEST_PATH);
-
-    int appendDataSize = 100;
-    int targetAppendCount = 50;
-    byte[] testData = new byte[baseDataSize + (appendDataSize*targetAppendCount)];
-    int testDataIndex = 0;
-    System.arraycopy(baseDataBuffer, 0, testData, testDataIndex, baseDataSize);
-    testDataIndex += baseDataSize;
-
-    int appendCount = 0;
-
-    FSDataOutputStream appendStream = null;
-
-    try {
-      while (appendCount < targetAppendCount) {
-
-        byte[] appendDataBuffer = getTestData(appendDataSize);
-        appendStream = fs.append(TEST_PATH, 30);
-        appendStream.write(appendDataBuffer);
-        appendStream.close();
-
-        System.arraycopy(appendDataBuffer, 0, testData, testDataIndex, appendDataSize);
-        testDataIndex += appendDataSize;
-        appendCount++;
-      }
-
-      Assert.assertTrue(verifyAppend(testData, TEST_PATH));
-
-    } finally {
-      if (appendStream != null) {
-        appendStream.close();
-      }
-    }
-  }
-
-  /*
-   * Test to verify we multiple appends on the same stream.
-   */
-  @Test
-  public void testMultipleAppendsOnSameStream() throws Throwable {
-
-    int baseDataSize = 50;
-    byte[] baseDataBuffer = createBaseFileWithData(baseDataSize, TEST_PATH);
-    int appendDataSize = 100;
-    int targetAppendCount = 50;
-    byte[] testData = new byte[baseDataSize + (appendDataSize*targetAppendCount)];
-    int testDataIndex = 0;
-    System.arraycopy(baseDataBuffer, 0, testData, testDataIndex, baseDataSize);
-    testDataIndex += baseDataSize;
-    int appendCount = 0;
-
-    FSDataOutputStream appendStream = null;
-
-    try {
-
-      while (appendCount < targetAppendCount) {
-
-        appendStream = fs.append(TEST_PATH, 50);
-
-        int singleAppendChunkSize = 20;
-        int appendRunSize = 0;
-        while (appendRunSize < appendDataSize) {
-
-          byte[] appendDataBuffer = getTestData(singleAppendChunkSize);
-          appendStream.write(appendDataBuffer);
-          System.arraycopy(appendDataBuffer, 0, testData,
-              testDataIndex + appendRunSize, singleAppendChunkSize);
-
-          appendRunSize += singleAppendChunkSize;
-        }
-
-        appendStream.close();
-        testDataIndex += appendDataSize;
-        appendCount++;
-      }
-
-      Assert.assertTrue(verifyAppend(testData, TEST_PATH));
-    } finally {
-      if (appendStream != null) {
-        appendStream.close();
-      }
-    }
-  }
-
-  @Test(expected=UnsupportedOperationException.class)
-  /*
-   * Test to verify the behavior when Append Support configuration flag is set to false
-   */
-  public void testFalseConfigurationFlagBehavior() throws Throwable {
-
-    fs = testAccount.getFileSystem();
-    Configuration conf = fs.getConf();
-    conf.setBoolean(NativeAzureFileSystem.APPEND_SUPPORT_ENABLE_PROPERTY_NAME, false);
-    URI uri = fs.getUri();
-    fs.initialize(uri, conf);
-
-    FSDataOutputStream appendStream = null;
-
-    try {
-      createBaseFileWithData(0, TEST_PATH);
-      appendStream = fs.append(TEST_PATH, 10);
-    } finally {
-      if (appendStream != null) {
-        appendStream.close();
-      }
-    }
-  }
-
-  @Override
-  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
-    return AzureBlobStorageTestAccount.create();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAtomicRenameDirList.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAtomicRenameDirList.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAtomicRenameDirList.java
deleted file mode 100644
index 602c1f7..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemAtomicRenameDirList.java
+++ /dev/null
@@ -1,50 +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.azure;
-
-import java.io.IOException;
-import java.net.URI;
-
-import org.apache.hadoop.conf.Configuration;
-import org.junit.Test;
-
-public class TestNativeAzureFileSystemAtomicRenameDirList
-    extends AbstractWasbTestBase {
-  private AzureBlobStorageTestAccount testAccount;
-
-  // HBase-site config controlling HBase root dir
-  private static final String HBASE_ROOT_DIR_CONF_STRING = "hbase.rootdir";
-  private static final String HBASE_ROOT_DIR_VALUE_ON_DIFFERENT_FS = "wasb://somedifferentfilesystem.blob.core.windows.net/hbase";
-  @Override
-  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
-    testAccount = AzureBlobStorageTestAccount.create();
-    return testAccount;
-  }
-
-  @Test
-  public void testAzureNativeStoreIsAtomicRenameKeyDoesNotThrowNPEOnInitializingWithNonDefaultURI () throws IOException {
-    NativeAzureFileSystem azureFs = (NativeAzureFileSystem)fs;
-    AzureNativeFileSystemStore azureStore = azureFs.getStore();
-    Configuration conf = fs.getConf();
-    conf.set(HBASE_ROOT_DIR_CONF_STRING, HBASE_ROOT_DIR_VALUE_ON_DIFFERENT_FS);
-    URI uri = fs.getUri();
-    fs.initialize(uri, conf);
-    azureStore.isAtomicRenameKey("anyrandomkey");
-  }
-}


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


[21/50] [abbrv] hadoop git commit: HADOOP-14553. Add (parallelized) integration tests to hadoop-azure Contributed by Steve Loughran

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestConstants.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestConstants.java
new file mode 100644
index 0000000..0b72f06
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestConstants.java
@@ -0,0 +1,180 @@
+/*
+ * 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.azure.integration;
+
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Constants for the Azure tests.
+ */
+public interface AzureTestConstants {
+
+  /**
+   * Prefix for any cross-filesystem scale test options.
+   */
+  String SCALE_TEST = "scale.test.";
+
+  /**
+   * Prefix for wasb-specific scale tests.
+   */
+  String AZURE_SCALE_TEST = "fs.azure.scale.test.";
+
+  /**
+   * Prefix for FS wasb tests.
+   */
+  String TEST_FS_WASB = "test.fs.azure.";
+
+  /**
+   * Name of the test filesystem.
+   */
+  String TEST_FS_WASB_NAME = TEST_FS_WASB + "name";
+
+  /**
+   * Tell tests that they are being executed in parallel: {@value}.
+   */
+  String KEY_PARALLEL_TEST_EXECUTION = "test.parallel.execution";
+
+  /**
+   * A property set to true in maven if scale tests are enabled: {@value}.
+   */
+  String KEY_SCALE_TESTS_ENABLED = AZURE_SCALE_TEST + "enabled";
+
+  /**
+   * The number of operations to perform: {@value}.
+   */
+  String KEY_OPERATION_COUNT = SCALE_TEST + "operation.count";
+
+  /**
+   * The number of directory operations to perform: {@value}.
+   */
+  String KEY_DIRECTORY_COUNT = SCALE_TEST + "directory.count";
+
+  /**
+   * The readahead buffer: {@value}.
+   */
+  String KEY_READ_BUFFER_SIZE = AZURE_SCALE_TEST + "read.buffer.size";
+
+  int DEFAULT_READ_BUFFER_SIZE = 16384;
+
+  /**
+   * Key for a multi MB test file: {@value}.
+   */
+  String KEY_CSVTEST_FILE = AZURE_SCALE_TEST + "csvfile";
+
+  /**
+   * Default path for the multi MB test file: {@value}.
+   */
+  String DEFAULT_CSVTEST_FILE = "wasb://datasets@azuremlsampleexperiments.blob.core.windows.net/network_intrusion_detection.csv";
+
+  /**
+   * Name of the property to define the timeout for scale tests: {@value}.
+   * Measured in seconds.
+   */
+  String KEY_TEST_TIMEOUT = AZURE_SCALE_TEST + "timeout";
+
+  /**
+   * Name of the property to define the file size for the huge file
+   * tests: {@value}.
+   * Measured in KB; a suffix like "M", or "G" will change the unit.
+   */
+  String KEY_HUGE_FILESIZE = AZURE_SCALE_TEST + "huge.filesize";
+
+  /**
+   * Name of the property to define the partition size for the huge file
+   * tests: {@value}.
+   * Measured in KB; a suffix like "M", or "G" will change the unit.
+   */
+  String KEY_HUGE_PARTITION_SIZE = AZURE_SCALE_TEST + "huge.partitionsize";
+
+  /**
+   * The default huge size is small —full 5GB+ scale tests are something
+   * to run in long test runs on EC2 VMs. {@value}.
+   */
+  String DEFAULT_HUGE_FILESIZE = "10M";
+
+  /**
+   * The default number of operations to perform: {@value}.
+   */
+  long DEFAULT_OPERATION_COUNT = 2005;
+
+  /**
+   * Default number of directories to create when performing
+   * directory performance/scale tests.
+   */
+  int DEFAULT_DIRECTORY_COUNT = 2;
+
+  /**
+   * Default policy on scale tests: {@value}.
+   */
+  boolean DEFAULT_SCALE_TESTS_ENABLED = false;
+
+  /**
+   * Fork ID passed down from maven if the test is running in parallel.
+   */
+  String TEST_UNIQUE_FORK_ID = "test.unique.fork.id";
+
+  /**
+   * Timeout in Milliseconds for standard tests: {@value}.
+   */
+  int AZURE_TEST_TIMEOUT = 10 * 60 * 1000;
+
+  /**
+   * Timeout in Seconds for Scale Tests: {@value}.
+   */
+  int SCALE_TEST_TIMEOUT_SECONDS = 30 * 60;
+
+  int SCALE_TEST_TIMEOUT_MILLIS = SCALE_TEST_TIMEOUT_SECONDS * 1000;
+
+
+
+  String ACCOUNT_KEY_PROPERTY_NAME
+      = "fs.azure.account.key.";
+  String SAS_PROPERTY_NAME = "fs.azure.sas.";
+  String TEST_CONFIGURATION_FILE_NAME = "azure-test.xml";
+  String TEST_ACCOUNT_NAME_PROPERTY_NAME
+      = "fs.azure.test.account.name";
+  String MOCK_ACCOUNT_NAME
+      = "mockAccount.blob.core.windows.net";
+  String MOCK_CONTAINER_NAME = "mockContainer";
+  String WASB_AUTHORITY_DELIMITER = "@";
+  String WASB_SCHEME = "wasb";
+  String PATH_DELIMITER = "/";
+  String AZURE_ROOT_CONTAINER = "$root";
+  String MOCK_WASB_URI = "wasb://" + MOCK_CONTAINER_NAME
+      + WASB_AUTHORITY_DELIMITER + MOCK_ACCOUNT_NAME + "/";
+  String USE_EMULATOR_PROPERTY_NAME
+      = "fs.azure.test.emulator";
+
+  String KEY_DISABLE_THROTTLING
+      = "fs.azure.disable.bandwidth.throttling";
+  String KEY_READ_TOLERATE_CONCURRENT_APPEND
+      = "fs.azure.io.read.tolerate.concurrent.append";
+  /**
+   * Path for page blobs: {@value}.
+   */
+  String DEFAULT_PAGE_BLOB_DIRECTORY = "pageBlobs";
+
+  String DEFAULT_ATOMIC_RENAME_DIRECTORIES
+      = "/atomicRenameDir1,/atomicRenameDir2";
+
+  /**
+   * Base directory for page blobs.
+   */
+  Path PAGE_BLOB_DIR = new Path("/" + DEFAULT_PAGE_BLOB_DIRECTORY);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestUtils.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestUtils.java
new file mode 100644
index 0000000..2fbbcd1
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AzureTestUtils.java
@@ -0,0 +1,479 @@
+/*
+ * 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.azure.integration;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.internal.AssumptionViolatedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
+import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
+
+import static org.apache.hadoop.fs.azure.integration.AzureTestConstants.*;
+import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
+import static org.apache.hadoop.test.MetricsAsserts.getLongGauge;
+import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
+
+/**
+ * Utilities for the Azure tests. Based on {@code S3ATestUtils}, so
+ * (initially) has unused method.
+ */
+public final class AzureTestUtils extends Assert {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AzureTestUtils.class);
+
+  /**
+   * Value to set a system property to (in maven) to declare that
+   * a property has been unset.
+   */
+  public static final String UNSET_PROPERTY = "unset";
+
+  /**
+   * Create the test filesystem.
+   *
+   * If the test.fs.wasb.name property is not set, this will
+   * raise a JUnit assumption exception
+   *
+   * @param conf configuration
+   * @return the FS
+   * @throws IOException IO Problems
+   * @throws AssumptionViolatedException if the FS is not named
+   */
+  public static NativeAzureFileSystem createTestFileSystem(Configuration conf)
+      throws IOException {
+
+    String fsname = conf.getTrimmed(TEST_FS_WASB_NAME, "");
+
+    boolean liveTest = !StringUtils.isEmpty(fsname);
+    URI testURI = null;
+    if (liveTest) {
+      testURI = URI.create(fsname);
+      liveTest = testURI.getScheme().equals(WASB_SCHEME);
+    }
+    if (!liveTest) {
+      // Skip the test
+      throw new AssumptionViolatedException(
+          "No test filesystem in " + TEST_FS_WASB_NAME);
+    }
+    NativeAzureFileSystem fs1 = new NativeAzureFileSystem();
+    fs1.initialize(testURI, conf);
+    return fs1;
+  }
+
+  /**
+   * Create a file context for tests.
+   *
+   * If the test.fs.wasb.name property is not set, this will
+   * trigger a JUnit failure.
+   *
+   * Multipart purging is enabled.
+   * @param conf configuration
+   * @return the FS
+   * @throws IOException IO Problems
+   * @throws AssumptionViolatedException if the FS is not named
+   */
+  public static FileContext createTestFileContext(Configuration conf)
+      throws IOException {
+    String fsname = conf.getTrimmed(TEST_FS_WASB_NAME, "");
+
+    boolean liveTest = !StringUtils.isEmpty(fsname);
+    URI testURI = null;
+    if (liveTest) {
+      testURI = URI.create(fsname);
+      liveTest = testURI.getScheme().equals(WASB_SCHEME);
+    }
+    if (!liveTest) {
+      // This doesn't work with our JUnit 3 style test cases, so instead we'll
+      // make this whole class not run by default
+      throw new AssumptionViolatedException("No test filesystem in "
+          + TEST_FS_WASB_NAME);
+    }
+    FileContext fc = FileContext.getFileContext(testURI, conf);
+    return fc;
+  }
+
+  /**
+   * Get a long test property.
+   * <ol>
+   *   <li>Look up configuration value (which can pick up core-default.xml),
+   *       using {@code defVal} as the default value (if conf != null).
+   *   </li>
+   *   <li>Fetch the system property.</li>
+   *   <li>If the system property is not empty or "(unset)":
+   *   it overrides the conf value.
+   *   </li>
+   * </ol>
+   * This puts the build properties in charge of everything. It's not a
+   * perfect design; having maven set properties based on a file, as ant let
+   * you do, is better for customization.
+   *
+   * As to why there's a special (unset) value, see
+   * {@link http://stackoverflow.com/questions/7773134/null-versus-empty-arguments-in-maven}
+   * @param conf config: may be null
+   * @param key key to look up
+   * @param defVal default value
+   * @return the evaluated test property.
+   */
+  public static long getTestPropertyLong(Configuration conf,
+      String key, long defVal) {
+    return Long.valueOf(
+        getTestProperty(conf, key, Long.toString(defVal)));
+  }
+  /**
+   * Get a test property value in bytes, using k, m, g, t, p, e suffixes.
+   * {@link org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix#string2long(String)}
+   * <ol>
+   *   <li>Look up configuration value (which can pick up core-default.xml),
+   *       using {@code defVal} as the default value (if conf != null).
+   *   </li>
+   *   <li>Fetch the system property.</li>
+   *   <li>If the system property is not empty or "(unset)":
+   *   it overrides the conf value.
+   *   </li>
+   * </ol>
+   * This puts the build properties in charge of everything. It's not a
+   * perfect design; having maven set properties based on a file, as ant let
+   * you do, is better for customization.
+   *
+   * As to why there's a special (unset) value, see
+   * {@link http://stackoverflow.com/questions/7773134/null-versus-empty-arguments-in-maven}
+   * @param conf config: may be null
+   * @param key key to look up
+   * @param defVal default value
+   * @return the evaluated test property.
+   */
+  public static long getTestPropertyBytes(Configuration conf,
+      String key, String defVal) {
+    return org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix
+        .string2long(getTestProperty(conf, key, defVal));
+  }
+
+  /**
+   * Get an integer test property; algorithm described in
+   * {@link #getTestPropertyLong(Configuration, String, long)}.
+   * @param key key to look up
+   * @param defVal default value
+   * @return the evaluated test property.
+   */
+  public static int getTestPropertyInt(Configuration conf,
+      String key, int defVal) {
+    return (int) getTestPropertyLong(conf, key, defVal);
+  }
+
+  /**
+   * Get a boolean test property; algorithm described in
+   * {@link #getTestPropertyLong(Configuration, String, long)}.
+   * @param key key to look up
+   * @param defVal default value
+   * @return the evaluated test property.
+   */
+  public static boolean getTestPropertyBool(Configuration conf,
+      String key,
+      boolean defVal) {
+    return Boolean.valueOf(
+        getTestProperty(conf, key, Boolean.toString(defVal)));
+  }
+
+  /**
+   * Get a string test property.
+   * <ol>
+   *   <li>Look up configuration value (which can pick up core-default.xml),
+   *       using {@code defVal} as the default value (if conf != null).
+   *   </li>
+   *   <li>Fetch the system property.</li>
+   *   <li>If the system property is not empty or "(unset)":
+   *   it overrides the conf value.
+   *   </li>
+   * </ol>
+   * This puts the build properties in charge of everything. It's not a
+   * perfect design; having maven set properties based on a file, as ant let
+   * you do, is better for customization.
+   *
+   * As to why there's a special (unset) value, see
+   * @see <a href="http://stackoverflow.com/questions/7773134/null-versus-empty-arguments-in-maven">
+   *   Stack Overflow</a>
+   * @param conf config: may be null
+   * @param key key to look up
+   * @param defVal default value
+   * @return the evaluated test property.
+   */
+
+  public static String getTestProperty(Configuration conf,
+      String key,
+      String defVal) {
+    String confVal = conf != null
+        ? conf.getTrimmed(key, defVal)
+        : defVal;
+    String propval = System.getProperty(key);
+    return StringUtils.isNotEmpty(propval) && !UNSET_PROPERTY.equals(propval)
+        ? propval : confVal;
+  }
+
+  /**
+   * Verify the class of an exception. If it is not as expected, rethrow it.
+   * Comparison is on the exact class, not subclass-of inference as
+   * offered by {@code instanceof}.
+   * @param clazz the expected exception class
+   * @param ex the exception caught
+   * @return the exception, if it is of the expected class
+   * @throws Exception the exception passed in.
+   */
+  public static Exception verifyExceptionClass(Class clazz,
+      Exception ex)
+      throws Exception {
+    if (!(ex.getClass().equals(clazz))) {
+      throw ex;
+    }
+    return ex;
+  }
+
+  /**
+   * Turn off FS Caching: use if a filesystem with different options from
+   * the default is required.
+   * @param conf configuration to patch
+   */
+  public static void disableFilesystemCaching(Configuration conf) {
+    conf.setBoolean("fs.wasb.impl.disable.cache", true);
+  }
+
+  /**
+   * Create a test path, using the value of
+   * {@link AzureTestUtils#TEST_UNIQUE_FORK_ID} if it is set.
+   * @param defVal default value
+   * @return a path
+   */
+  public static Path createTestPath(Path defVal) {
+    String testUniqueForkId = System.getProperty(
+        AzureTestConstants.TEST_UNIQUE_FORK_ID);
+    return testUniqueForkId == null
+        ? defVal
+        : new Path("/" + testUniqueForkId, "test");
+  }
+
+  /**
+   * Create a test page blob path using the value of
+   * {@link AzureTestConstants#TEST_UNIQUE_FORK_ID} if it is set.
+   * @param filename filename at the end of the path
+   * @return an absolute path
+   */
+  public static Path blobPathForTests(FileSystem fs, String filename) {
+    String testUniqueForkId = System.getProperty(
+        AzureTestConstants.TEST_UNIQUE_FORK_ID);
+    return fs.makeQualified(new Path(PAGE_BLOB_DIR,
+        testUniqueForkId == null
+            ? filename
+            : (testUniqueForkId + "/" + filename)));
+  }
+
+  /**
+   * Create a test path using the value of
+   * {@link AzureTestConstants#TEST_UNIQUE_FORK_ID} if it is set.
+   * @param filename filename at the end of the path
+   * @return an absolute path
+   */
+  public static Path pathForTests(FileSystem fs, String filename) {
+    String testUniqueForkId = System.getProperty(
+        AzureTestConstants.TEST_UNIQUE_FORK_ID);
+    return fs.makeQualified(new Path(
+        testUniqueForkId == null
+            ? ("/test/" + filename)
+            : (testUniqueForkId + "/" + filename)));
+  }
+
+  /**
+   * Get a unique fork ID.
+   * Returns a default value for non-parallel tests.
+   * @return a string unique for all test VMs running in this maven build.
+   */
+  public static String getForkID() {
+    return System.getProperty(
+        AzureTestConstants.TEST_UNIQUE_FORK_ID, "fork-1");
+  }
+
+  /**
+   * Flag to indicate that this test is being executed in parallel.
+   * This is used by some of the scale tests to validate test time expectations.
+   * @return true if the build indicates this test is being run in parallel.
+   */
+  public static boolean isParallelExecution() {
+    return Boolean.getBoolean(KEY_PARALLEL_TEST_EXECUTION);
+  }
+
+  /**
+   * Asserts that {@code obj} is an instance of {@code expectedClass} using a
+   * descriptive assertion message.
+   * @param expectedClass class
+   * @param obj object to check
+   */
+  public static void assertInstanceOf(Class<?> expectedClass, Object obj) {
+    Assert.assertTrue(String.format("Expected instance of class %s, but is %s.",
+        expectedClass, obj.getClass()),
+        expectedClass.isAssignableFrom(obj.getClass()));
+  }
+
+  /**
+   * Builds a comma-separated list of class names.
+   * @param classes list of classes
+   * @return comma-separated list of class names
+   */
+  public static <T extends Class<?>> String buildClassListString(
+      List<T> classes) {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < classes.size(); ++i) {
+      if (i > 0) {
+        sb.append(',');
+      }
+      sb.append(classes.get(i).getName());
+    }
+    return sb.toString();
+  }
+
+  /**
+   * This class should not be instantiated.
+   */
+  private AzureTestUtils() {
+  }
+
+  /**
+   * Assert that a configuration option matches the expected value.
+   * @param conf configuration
+   * @param key option key
+   * @param expected expected value
+   */
+  public static void assertOptionEquals(Configuration conf,
+      String key,
+      String expected) {
+    assertEquals("Value of " + key, expected, conf.get(key));
+  }
+
+  /**
+   * Assume that a condition is met. If not: log at WARN and
+   * then throw an {@link AssumptionViolatedException}.
+   * @param message message in an assumption
+   * @param condition condition to probe
+   */
+  public static void assume(String message, boolean condition) {
+    if (!condition) {
+      LOG.warn(message);
+    }
+    Assume.assumeTrue(message, condition);
+  }
+
+  /**
+   * Gets the current value of the given gauge.
+   * @param fs filesystem
+   * @param gaugeName gauge name
+   * @return the gauge value
+   */
+  public static long getLongGaugeValue(NativeAzureFileSystem fs,
+      String gaugeName) {
+    return getLongGauge(gaugeName, getMetrics(fs.getInstrumentation()));
+  }
+
+  /**
+   * Gets the current value of the given counter.
+   * @param fs filesystem
+   * @param counterName counter name
+   * @return the counter value
+   */
+  public static long getLongCounterValue(NativeAzureFileSystem fs,
+      String counterName) {
+    return getLongCounter(counterName, getMetrics(fs.getInstrumentation()));
+  }
+
+
+  /**
+   * Delete a path, catching any exception and downgrading to a log message.
+   * @param fs filesystem
+   * @param path path to delete
+   * @param recursive recursive delete?
+   * @throws IOException IO failure.
+   */
+  public static void deleteQuietly(FileSystem fs,
+      Path path,
+      boolean recursive) throws IOException {
+    if (fs != null && path != null) {
+      try {
+        fs.delete(path, recursive);
+      } catch (IOException e) {
+        LOG.warn("When deleting {}", path, e);
+      }
+    }
+  }
+
+
+  /**
+   * Clean up the test account if non-null; return null to put in the
+   * field.
+   * @param testAccount test account to clean up
+   * @return null
+   * @throws Execption cleanup problems
+   */
+  public static AzureBlobStorageTestAccount cleanup(
+      AzureBlobStorageTestAccount testAccount) throws Exception {
+    if (testAccount != null) {
+      testAccount.cleanup();
+      testAccount = null;
+    }
+    return null;
+  }
+
+
+  /**
+   * Clean up the test account; any thrown exceptions are caught and
+   * logged.
+   * @param testAccount test account
+   * @return null, so that any fields can be reset.
+   */
+  public static AzureBlobStorageTestAccount cleanupTestAccount(
+      AzureBlobStorageTestAccount testAccount) {
+    if (testAccount != null) {
+      try {
+        testAccount.cleanup();
+      } catch (Exception e) {
+        LOG.error("While cleaning up test account: ", e);
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Assume that the scale tests are enabled by the relevant system property.
+   */
+  public static void assumeScaleTestsEnabled(Configuration conf) {
+    boolean enabled = getTestPropertyBool(
+        conf,
+        KEY_SCALE_TESTS_ENABLED,
+        DEFAULT_SCALE_TESTS_ENABLED);
+    assume("Scale test disabled: to enable set property "
+            + KEY_SCALE_TESTS_ENABLED,
+        enabled);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/CleanupTestContainers.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/CleanupTestContainers.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/CleanupTestContainers.java
new file mode 100644
index 0000000..059a8c4
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/CleanupTestContainers.java
@@ -0,0 +1,87 @@
+/*
+ * 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.azure.integration;
+
+import java.util.EnumSet;
+
+import com.microsoft.azure.storage.CloudStorageAccount;
+import com.microsoft.azure.storage.blob.CloudBlobClient;
+import com.microsoft.azure.storage.blob.CloudBlobContainer;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.azure.AbstractWasbTestBase;
+import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
+
+/**
+ * This looks like a test, but it is really a command to invoke to
+ * clean up containers created in other test runs.
+ *
+ */
+public class CleanupTestContainers extends AbstractWasbTestBase {
+
+  private static final String CONTAINER_PREFIX = "wasbtests-";
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create(
+        "CleanupTestContainers",
+        EnumSet.noneOf(AzureBlobStorageTestAccount.CreateOptions.class),
+        createConfiguration(),
+        true);
+  }
+
+  @Test
+  public void testEnumContainers() throws Throwable {
+    describe("Enumerating all the WASB test containers");
+
+    int count = 0;
+    CloudStorageAccount storageAccount = getTestAccount().getRealAccount();
+    CloudBlobClient blobClient = storageAccount.createCloudBlobClient();
+    Iterable<CloudBlobContainer> containers
+        = blobClient.listContainers(CONTAINER_PREFIX);
+    for (CloudBlobContainer container : containers) {
+      count++;
+      LOG.info("Container {} URI {}",
+          container.getName(),
+          container.getUri());
+    }
+    LOG.info("Found {} test containers", count);
+  }
+
+  @Test
+  public void testDeleteContainers() throws Throwable {
+    describe("Delete all the WASB test containers");
+    int count = 0;
+    CloudStorageAccount storageAccount = getTestAccount().getRealAccount();
+    CloudBlobClient blobClient = storageAccount.createCloudBlobClient();
+    Iterable<CloudBlobContainer> containers
+        = blobClient.listContainers(CONTAINER_PREFIX);
+    for (CloudBlobContainer container : containers) {
+      LOG.info("Container {} URI {}",
+          container.getName(),
+          container.getUri());
+      if (container.deleteIfExists()) {
+        count++;
+      }
+    }
+    LOG.info("Deleted {} test containers", count);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/ITestAzureHugeFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/ITestAzureHugeFiles.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/ITestAzureHugeFiles.java
new file mode 100644
index 0000000..850aca1
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/ITestAzureHugeFiles.java
@@ -0,0 +1,456 @@
+/*
+ * 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.azure.integration;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.Iterator;
+
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.StorageStatistics;
+import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
+import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.io.IOUtils;
+
+import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.*;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
+
+
+/**
+ * Scale test which creates a huge file.
+ *
+ * <b>Important:</b> the order in which these tests execute is fixed to
+ * alphabetical order. Test cases are numbered {@code test_123_} to impose
+ * an ordering based on the numbers.
+ *
+ * Having this ordering allows the tests to assume that the huge file
+ * exists. Even so: they should all have a {@link #assumeHugeFileExists()}
+ * check at the start, in case an individual test is executed.
+ *
+ * <b>Ignore checkstyle complaints about naming: we need a scheme with visible
+ * ordering.</b>
+ */
+
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class ITestAzureHugeFiles extends AbstractAzureScaleTest {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      ITestAzureHugeFiles.class);
+
+  private Path scaleTestDir;
+  private Path hugefile;
+  private Path hugefileRenamed;
+  private AzureBlobStorageTestAccount testAccountForCleanup;
+
+  private static final int UPLOAD_BLOCKSIZE = 64 * S_1K;
+  private static final byte[] SOURCE_DATA;
+
+  static {
+    SOURCE_DATA = dataset(UPLOAD_BLOCKSIZE, 0, S_256);
+  }
+
+  private Path testPath;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    testPath = path("ITestAzureHugeFiles");
+    scaleTestDir = new Path(testPath, "scale");
+    hugefile = new Path(scaleTestDir, "hugefile");
+    hugefileRenamed = new Path(scaleTestDir, "hugefileRenamed");
+  }
+
+  /**
+   * Only clean up the test account (and delete the container) if the account
+   * is set in the field {@code testAccountForCleanup}.
+   * @throws Exception
+   */
+  @Override
+  public void tearDown() throws Exception {
+    testAccount = null;
+    super.tearDown();
+    if (testAccountForCleanup != null) {
+      cleanupTestAccount(testAccount);
+    }
+  }
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create(
+        "testazurehugefiles",
+        EnumSet.of(AzureBlobStorageTestAccount.CreateOptions.CreateContainer),
+        createConfiguration(),
+        true);
+  }
+
+  /**
+   * Stop the test-case teardown from deleting the test path.
+   * @throws IOException never
+   */
+  protected void deleteTestDirInTeardown() throws IOException {
+    // this is a no-op, so the test file is preserved.
+    // the last test in the suite does the teardown
+  }
+
+  protected void deleteHugeFile() throws IOException {
+    describe("Deleting %s", hugefile);
+    ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+    getFileSystem().delete(hugefile, false);
+    timer.end("time to delete %s", hugefile);
+  }
+
+  /**
+   * Log how long an IOP took, by dividing the total time by the
+   * count of operations, printing in a human-readable form.
+   * @param operation operation being measured
+   * @param timer timing data
+   * @param count IOP count.
+   */
+  protected void logTimePerIOP(String operation,
+      ContractTestUtils.NanoTimer timer,
+      long count) {
+    LOG.info("Time per {}: {} nS",
+        operation, toHuman(timer.duration() / count));
+  }
+
+  /**
+   * Assume that the huge file exists, skip if not/empty.
+   * @return the file status
+   * @throws IOException IO failure
+   */
+  FileStatus assumeHugeFileExists() throws IOException {
+    assertPathExists(getFileSystem(), "huge file not created", hugefile);
+    try {
+      FileStatus status = getFileSystem().getFileStatus(hugefile);
+      Assume.assumeTrue("Not a file: " + status, status.isFile());
+      Assume.assumeTrue("File " + hugefile + " is empty", status.getLen() > 0);
+      return status;
+    } catch (FileNotFoundException e) {
+      skip("huge file not created: " + hugefile);
+    }
+    return null;
+  }
+
+  /**
+   * If/when {@link NativeAzureFileSystem#getStorageStatistics()} returns
+   * statistics, this will be interesting.
+   */
+  private void logFSState() {
+    StorageStatistics statistics = getFileSystem().getStorageStatistics();
+    Iterator<StorageStatistics.LongStatistic> longStatistics
+        = statistics.getLongStatistics();
+    while (longStatistics.hasNext()) {
+      StorageStatistics.LongStatistic next = longStatistics.next();
+      LOG.info("{} = {}", next.getName(), next.getValue());
+    }
+  }
+
+  @Test
+  public void test_010_CreateHugeFile() throws IOException {
+    long filesize = getTestPropertyBytes(getConfiguration(),
+        KEY_HUGE_FILESIZE,
+        DEFAULT_HUGE_FILESIZE);
+    long filesizeMB = filesize / S_1M;
+
+    // clean up from any previous attempts
+    deleteHugeFile();
+
+    describe("Creating file %s of size %d MB", hugefile, filesizeMB);
+
+    // now do a check of available upload time, with a pessimistic bandwidth
+    // (that of remote upload tests). If the test times out then not only is
+    // the test outcome lost, as the follow-on tests continue, they will
+    // overlap with the ongoing upload test, for much confusion.
+/*
+    int timeout = getTestTimeoutSeconds();
+    // assume 1 MB/s upload bandwidth
+    int bandwidth = _1MB;
+    long uploadTime = filesize / bandwidth;
+    assertTrue(String.format("Timeout set in %s seconds is too low;" +
+            " estimating upload time of %d seconds at 1 MB/s." +
+            " Rerun tests with -D%s=%d",
+        timeout, uploadTime, KEY_TEST_TIMEOUT, uploadTime * 2),
+        uploadTime < timeout);
+*/
+    assertEquals("File size set in " + KEY_HUGE_FILESIZE + " = " + filesize
+            + " is not a multiple of " + UPLOAD_BLOCKSIZE,
+        0, filesize % UPLOAD_BLOCKSIZE);
+
+    byte[] data = SOURCE_DATA;
+
+    long blocks = filesize / UPLOAD_BLOCKSIZE;
+    long blocksPerMB = S_1M / UPLOAD_BLOCKSIZE;
+
+    // perform the upload.
+    // there's lots of logging here, so that a tail -f on the output log
+    // can give a view of what is happening.
+    NativeAzureFileSystem fs = getFileSystem();
+
+    ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+    long blocksPer10MB = blocksPerMB * 10;
+    fs.mkdirs(hugefile.getParent());
+    try (FSDataOutputStream out = fs.create(hugefile,
+        true,
+        UPLOAD_BLOCKSIZE,
+        null)) {
+      for (long block = 1; block <= blocks; block++) {
+        out.write(data);
+        long written = block * UPLOAD_BLOCKSIZE;
+        // every 10 MB and on file upload @ 100%, print some stats
+        if (block % blocksPer10MB == 0 || written == filesize) {
+          long percentage = written * 100 / filesize;
+          double elapsedTime = timer.elapsedTime() / NANOSEC;
+          double writtenMB = 1.0 * written / S_1M;
+          LOG.info(String.format("[%02d%%] Buffered %.2f MB out of %d MB;"
+                  + " elapsedTime=%.2fs; write to buffer bandwidth=%.2f MB/s",
+              percentage,
+              writtenMB,
+              filesizeMB,
+              elapsedTime,
+              writtenMB / elapsedTime));
+        }
+      }
+      // now close the file
+      LOG.info("Closing stream {}", out);
+      ContractTestUtils.NanoTimer closeTimer
+          = new ContractTestUtils.NanoTimer();
+      out.close();
+      closeTimer.end("time to close() output stream");
+    }
+
+    timer.end("time to write %d MB in blocks of %d",
+        filesizeMB, UPLOAD_BLOCKSIZE);
+    logFSState();
+    bandwidth(timer, filesize);
+    ContractTestUtils.assertPathExists(fs, "Huge file", hugefile);
+    FileStatus status = fs.getFileStatus(hugefile);
+    ContractTestUtils.assertIsFile(hugefile, status);
+    assertEquals("File size in " + status, filesize, status.getLen());
+  }
+
+  @Test
+  public void test_040_PositionedReadHugeFile() throws Throwable {
+    assumeHugeFileExists();
+    describe("Positioned reads of file %s", hugefile);
+    NativeAzureFileSystem fs = getFileSystem();
+    FileStatus status = fs.getFileStatus(hugefile);
+    long filesize = status.getLen();
+    int ops = 0;
+    final int bufferSize = 8192;
+    byte[] buffer = new byte[bufferSize];
+    long eof = filesize - 1;
+
+    ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+    ContractTestUtils.NanoTimer readAtByte0, readAtByte0Again, readAtEOF;
+    try (FSDataInputStream in = openDataFile()) {
+      readAtByte0 = new ContractTestUtils.NanoTimer();
+      in.readFully(0, buffer);
+      readAtByte0.end("time to read data at start of file");
+      ops++;
+
+      readAtEOF = new ContractTestUtils.NanoTimer();
+      in.readFully(eof - bufferSize, buffer);
+      readAtEOF.end("time to read data at end of file");
+      ops++;
+
+      readAtByte0Again = new ContractTestUtils.NanoTimer();
+      in.readFully(0, buffer);
+      readAtByte0Again.end("time to read data at start of file again");
+      ops++;
+      LOG.info("Final stream state: {}", in);
+    }
+    long mb = Math.max(filesize / S_1M, 1);
+
+    logFSState();
+    timer.end("time to performed positioned reads of %d MB ", mb);
+    LOG.info("Time per positioned read = {} nS",
+        toHuman(timer.nanosPerOperation(ops)));
+  }
+
+  protected FSDataInputStream openDataFile() throws IOException {
+    NanoTimer openTimer = new NanoTimer();
+    FSDataInputStream inputStream = getFileSystem().open(hugefile,
+        UPLOAD_BLOCKSIZE);
+    openTimer.end("open data file");
+    return inputStream;
+  }
+
+
+  /**
+   * Work out the bandwidth in bytes/second.
+   * @param timer timer measuring the duration
+   * @param bytes bytes
+   * @return the number of bytes/second of the recorded operation
+   */
+  public static double bandwidthInBytes(NanoTimer timer, long bytes) {
+    return bytes * NANOSEC / timer.duration();
+  }
+
+  @Test
+  public void test_050_readHugeFile() throws Throwable {
+    assumeHugeFileExists();
+    describe("Reading %s", hugefile);
+    NativeAzureFileSystem fs = getFileSystem();
+    FileStatus status = fs.getFileStatus(hugefile);
+    long filesize = status.getLen();
+    long blocks = filesize / UPLOAD_BLOCKSIZE;
+    byte[] data = new byte[UPLOAD_BLOCKSIZE];
+
+    ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+    try (FSDataInputStream in = openDataFile()) {
+      for (long block = 0; block < blocks; block++) {
+        in.readFully(data);
+      }
+      LOG.info("Final stream state: {}", in);
+    }
+
+    long mb = Math.max(filesize / S_1M, 1);
+    timer.end("time to read file of %d MB ", mb);
+    LOG.info("Time per MB to read = {} nS",
+        toHuman(timer.nanosPerOperation(mb)));
+    bandwidth(timer, filesize);
+    logFSState();
+  }
+
+  @Test
+  public void test_060_openAndReadWholeFileBlocks() throws Throwable {
+    FileStatus status = assumeHugeFileExists();
+    int blockSize = S_1M;
+    describe("Open the test file and read it in blocks of size %d",
+        blockSize);
+    long len =  status.getLen();
+    FSDataInputStream in = openDataFile();
+    NanoTimer timer2 = null;
+    long blockCount = 0;
+    long totalToRead = 0;
+    int resetCount = 0;
+    try {
+      byte[] block = new byte[blockSize];
+      timer2 = new NanoTimer();
+      long count = 0;
+      // implicitly rounding down here
+      blockCount = len / blockSize;
+      totalToRead = blockCount * blockSize;
+      long minimumBandwidth = S_128K;
+      int maxResetCount = 4;
+      resetCount = 0;
+      for (long i = 0; i < blockCount; i++) {
+        int offset = 0;
+        int remaining = blockSize;
+        long blockId = i + 1;
+        NanoTimer blockTimer = new NanoTimer();
+        int reads = 0;
+        while (remaining > 0) {
+          NanoTimer readTimer = new NanoTimer();
+          int bytesRead = in.read(block, offset, remaining);
+          reads++;
+          if (bytesRead == 1) {
+            break;
+          }
+          remaining -= bytesRead;
+          offset += bytesRead;
+          count += bytesRead;
+          readTimer.end();
+          if (bytesRead != 0) {
+            LOG.debug("Bytes in read #{}: {} , block bytes: {},"
+                    + " remaining in block: {}"
+                    + " duration={} nS; ns/byte: {}, bandwidth={} MB/s",
+                reads, bytesRead, blockSize - remaining, remaining,
+                readTimer.duration(),
+                readTimer.nanosPerOperation(bytesRead),
+                readTimer.bandwidthDescription(bytesRead));
+          } else {
+            LOG.warn("0 bytes returned by read() operation #{}", reads);
+          }
+        }
+        blockTimer.end("Reading block %d in %d reads", blockId, reads);
+        String bw = blockTimer.bandwidthDescription(blockSize);
+        LOG.info("Bandwidth of block {}: {} MB/s: ", blockId, bw);
+        if (bandwidthInBytes(blockTimer, blockSize) < minimumBandwidth) {
+          LOG.warn("Bandwidth {} too low on block {}: resetting connection",
+              bw, blockId);
+          Assert.assertTrue("Bandwidth of " + bw + " too low after "
+              + resetCount + " attempts", resetCount <= maxResetCount);
+          resetCount++;
+          // reset the connection
+        }
+      }
+    } finally {
+      IOUtils.closeStream(in);
+    }
+    timer2.end("Time to read %d bytes in %d blocks", totalToRead, blockCount);
+    LOG.info("Overall Bandwidth {} MB/s; reset connections {}",
+        timer2.bandwidth(totalToRead), resetCount);
+  }
+
+  @Test
+  public void test_100_renameHugeFile() throws Throwable {
+    assumeHugeFileExists();
+    describe("renaming %s to %s", hugefile, hugefileRenamed);
+    NativeAzureFileSystem fs = getFileSystem();
+    FileStatus status = fs.getFileStatus(hugefile);
+    long filesize = status.getLen();
+    fs.delete(hugefileRenamed, false);
+    ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+    fs.rename(hugefile, hugefileRenamed);
+    long mb = Math.max(filesize / S_1M, 1);
+    timer.end("time to rename file of %d MB", mb);
+    LOG.info("Time per MB to rename = {} nS",
+        toHuman(timer.nanosPerOperation(mb)));
+    bandwidth(timer, filesize);
+    logFSState();
+    FileStatus destFileStatus = fs.getFileStatus(hugefileRenamed);
+    assertEquals(filesize, destFileStatus.getLen());
+
+    // rename back
+    ContractTestUtils.NanoTimer timer2 = new ContractTestUtils.NanoTimer();
+    fs.rename(hugefileRenamed, hugefile);
+    timer2.end("Renaming back");
+    LOG.info("Time per MB to rename = {} nS",
+        toHuman(timer2.nanosPerOperation(mb)));
+    bandwidth(timer2, filesize);
+  }
+
+  @Test
+  public void test_999_deleteHugeFiles() throws IOException {
+    // mark the test account for cleanup after this test
+    testAccountForCleanup = testAccount;
+    deleteHugeFile();
+    ContractTestUtils.NanoTimer timer2 = new ContractTestUtils.NanoTimer();
+    NativeAzureFileSystem fs = getFileSystem();
+    fs.delete(hugefileRenamed, false);
+    timer2.end("time to delete %s", hugefileRenamed);
+    rm(fs, testPath, true, false);
+    assertPathDoesNotExist(fs, "deleted huge file", testPath);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/Sizes.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/Sizes.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/Sizes.java
new file mode 100644
index 0000000..92b10cf
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/Sizes.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure.integration;
+
+/**
+ * Sizes of data.
+ * Checkstyle doesn't like the naming scheme or the fact its an interface.
+ */
+public interface Sizes {
+
+  int S_256 = 256;
+  int S_512 = 512;
+  int S_1K = 1024;
+  int S_4K = 4 * S_1K;
+  int S_8K = 8 * S_1K;
+  int S_16K = 16 * S_1K;
+  int S_32K = 32 * S_1K;
+  int S_64K = 64 * S_1K;
+  int S_128K = 128 * S_1K;
+  int S_256K = 256 * S_1K;
+  int S_1M = S_1K * S_1K;
+  int S_2M = 2 * S_1M;
+  int S_5M = 5 * S_1M;
+  int S_10M = 10* S_1M;
+  double NANOSEC = 1.0e9;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/ITestAzureFileSystemInstrumentation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/ITestAzureFileSystemInstrumentation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/ITestAzureFileSystemInstrumentation.java
new file mode 100644
index 0000000..60e24ee
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/ITestAzureFileSystemInstrumentation.java
@@ -0,0 +1,586 @@
+/**
+ * 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.azure.metrics;
+
+import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_CLIENT_ERRORS;
+import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_DIRECTORIES_CREATED;
+import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_DOWNLOAD_LATENCY;
+import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_DOWNLOAD_RATE;
+import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_FILES_CREATED;
+import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_FILES_DELETED;
+import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_SERVER_ERRORS;
+import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_UPLOAD_LATENCY;
+import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_UPLOAD_RATE;
+import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_WEB_RESPONSES;
+import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
+import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Mockito.verify;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Date;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azure.AbstractWasbTestBase;
+import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
+import org.apache.hadoop.fs.azure.AzureException;
+import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore;
+import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsTag;
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Instrumentation test, changing state of time and verifying metrics are
+ * consistent.
+ */
+public class ITestAzureFileSystemInstrumentation extends AbstractWasbTestBase {
+
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(ITestAzureFileSystemInstrumentation.class);
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create();
+  }
+
+  @Test
+  public void testMetricTags() throws Exception {
+    String accountName =
+        getTestAccount().getRealAccount().getBlobEndpoint()
+        .getAuthority();
+    String containerName =
+        getTestAccount().getRealContainer().getName();
+    MetricsRecordBuilder myMetrics = getMyMetrics();
+    verify(myMetrics).add(argThat(
+        new TagMatcher("accountName", accountName)
+        ));
+    verify(myMetrics).add(argThat(
+        new TagMatcher("containerName", containerName)
+        ));
+    verify(myMetrics).add(argThat(
+        new TagMatcher("Context", "azureFileSystem")
+        ));
+    verify(myMetrics).add(argThat(
+        new TagExistsMatcher("wasbFileSystemId")
+        ));
+  }
+  
+
+  @Test
+  public void testMetricsOnMkdirList() throws Exception {
+    long base = getBaseWebResponses();
+
+    // Create a directory
+    assertTrue(fs.mkdirs(new Path("a")));
+    // At the time of writing
+    // getAncestor uses 2 calls for each folder level /user/<name>/a
+    // plus 1 call made by checkContainer
+    // mkdir checks the hierarchy with 2 calls per level
+    // mkdirs calls storeEmptyDir to create the empty folder, which makes 5 calls
+    // For a total of 7 + 6 + 5 = 18 web responses
+    base = assertWebResponsesInRange(base, 1, 18);
+    assertEquals(1,
+        AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_DIRECTORIES_CREATED));
+
+    // List the root contents
+    assertEquals(1, getFileSystem().listStatus(new Path("/")).length);
+    base = assertWebResponsesEquals(base, 1);
+
+    assertNoErrors();
+  }
+
+  private BandwidthGaugeUpdater getBandwidthGaugeUpdater() {
+    NativeAzureFileSystem azureFs = (NativeAzureFileSystem) getFileSystem();
+    AzureNativeFileSystemStore azureStore = azureFs.getStore();
+    return azureStore.getBandwidthGaugeUpdater();
+  }
+
+  private static byte[] nonZeroByteArray(int size) {
+    byte[] data = new byte[size];
+    Arrays.fill(data, (byte)5);
+    return data;
+  }
+
+  @Test
+  public void testMetricsOnFileCreateRead() throws Exception {
+    long base = getBaseWebResponses();
+
+    assertEquals(0, AzureMetricsTestUtil.getCurrentBytesWritten(getInstrumentation()));
+
+    Path filePath = new Path("/metricsTest_webResponses");
+    final int FILE_SIZE = 1000;
+
+    // Suppress auto-update of bandwidth metrics so we get
+    // to update them exactly when we want to.
+    getBandwidthGaugeUpdater().suppressAutoUpdate();
+
+    // Create a file
+    Date start = new Date();
+    OutputStream outputStream = getFileSystem().create(filePath);
+    outputStream.write(nonZeroByteArray(FILE_SIZE));
+    outputStream.close();
+    long uploadDurationMs = new Date().getTime() - start.getTime();
+
+    // The exact number of requests/responses that happen to create a file
+    // can vary  - at the time of writing this code it takes 10
+    // requests/responses for the 1000 byte file (33 for 100 MB),
+    // plus the initial container-check request but that
+    // can very easily change in the future. Just assert that we do roughly
+    // more than 2 but less than 15.
+    logOpResponseCount("Creating a 1K file", base);
+    base = assertWebResponsesInRange(base, 2, 15);
+    getBandwidthGaugeUpdater().triggerUpdate(true);
+    long bytesWritten = AzureMetricsTestUtil.getCurrentBytesWritten(getInstrumentation());
+    assertTrue("The bytes written in the last second " + bytesWritten +
+        " is pretty far from the expected range of around " + FILE_SIZE +
+        " bytes plus a little overhead.",
+        bytesWritten > (FILE_SIZE / 2) && bytesWritten < (FILE_SIZE * 2));
+    long totalBytesWritten = AzureMetricsTestUtil.getCurrentTotalBytesWritten(getInstrumentation());
+    assertTrue("The total bytes written  " + totalBytesWritten +
+        " is pretty far from the expected range of around " + FILE_SIZE +
+        " bytes plus a little overhead.",
+        totalBytesWritten >= FILE_SIZE && totalBytesWritten < (FILE_SIZE * 2));
+    long uploadRate = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(), WASB_UPLOAD_RATE);
+    LOG.info("Upload rate: " + uploadRate + " bytes/second.");
+    long expectedRate = (FILE_SIZE * 1000L) / uploadDurationMs;
+    assertTrue("The upload rate " + uploadRate +
+        " is below the expected range of around " + expectedRate +
+        " bytes/second that the unit test observed. This should never be" +
+        " the case since the test underestimates the rate by looking at " +
+        " end-to-end time instead of just block upload time.",
+        uploadRate >= expectedRate);
+    long uploadLatency = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(),
+        WASB_UPLOAD_LATENCY);
+    LOG.info("Upload latency: {}", uploadLatency);
+    long expectedLatency = uploadDurationMs; // We're uploading less than a block.
+    assertTrue("The upload latency " + uploadLatency +
+        " should be greater than zero now that I've just uploaded a file.",
+        uploadLatency > 0);
+    assertTrue("The upload latency " + uploadLatency +
+        " is more than the expected range of around " + expectedLatency +
+        " milliseconds that the unit test observed. This should never be" +
+        " the case since the test overestimates the latency by looking at " +
+        " end-to-end time instead of just block upload time.",
+        uploadLatency <= expectedLatency);
+
+    // Read the file
+    start = new Date();
+    InputStream inputStream = getFileSystem().open(filePath);
+    int count = 0;
+    while (inputStream.read() >= 0) {
+      count++;
+    }
+    inputStream.close();
+    long downloadDurationMs = new Date().getTime() - start.getTime();
+    assertEquals(FILE_SIZE, count);
+
+    // Again, exact number varies. At the time of writing this code
+    // it takes 4 request/responses, so just assert a rough range between
+    // 1 and 10.
+    logOpResponseCount("Reading a 1K file", base);
+    base = assertWebResponsesInRange(base, 1, 10);
+    getBandwidthGaugeUpdater().triggerUpdate(false);
+    long totalBytesRead = AzureMetricsTestUtil.getCurrentTotalBytesRead(getInstrumentation());
+    assertEquals(FILE_SIZE, totalBytesRead);
+    long bytesRead = AzureMetricsTestUtil.getCurrentBytesRead(getInstrumentation());
+    assertTrue("The bytes read in the last second " + bytesRead +
+        " is pretty far from the expected range of around " + FILE_SIZE +
+        " bytes plus a little overhead.",
+        bytesRead > (FILE_SIZE / 2) && bytesRead < (FILE_SIZE * 2));
+    long downloadRate = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(), WASB_DOWNLOAD_RATE);
+    LOG.info("Download rate: " + downloadRate + " bytes/second.");
+    expectedRate = (FILE_SIZE * 1000L) / downloadDurationMs;
+    assertTrue("The download rate " + downloadRate +
+        " is below the expected range of around " + expectedRate +
+        " bytes/second that the unit test observed. This should never be" +
+        " the case since the test underestimates the rate by looking at " +
+        " end-to-end time instead of just block download time.",
+        downloadRate >= expectedRate);
+    long downloadLatency = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(),
+        WASB_DOWNLOAD_LATENCY);
+    LOG.info("Download latency: " + downloadLatency);
+    expectedLatency = downloadDurationMs; // We're downloading less than a block.
+    assertTrue("The download latency " + downloadLatency +
+        " should be greater than zero now that I've just downloaded a file.",
+        downloadLatency > 0);
+    assertTrue("The download latency " + downloadLatency +
+        " is more than the expected range of around " + expectedLatency +
+        " milliseconds that the unit test observed. This should never be" +
+        " the case since the test overestimates the latency by looking at " +
+        " end-to-end time instead of just block download time.",
+        downloadLatency <= expectedLatency);
+
+    assertNoErrors();
+  }
+
+  @Test
+  public void testMetricsOnBigFileCreateRead() throws Exception {
+    long base = getBaseWebResponses();
+
+    assertEquals(0, AzureMetricsTestUtil.getCurrentBytesWritten(getInstrumentation()));
+
+    Path filePath = new Path("/metricsTest_webResponses");
+    final int FILE_SIZE = 100 * 1024 * 1024;
+
+    // Suppress auto-update of bandwidth metrics so we get
+    // to update them exactly when we want to.
+    getBandwidthGaugeUpdater().suppressAutoUpdate();
+
+    // Create a file
+    OutputStream outputStream = getFileSystem().create(filePath);
+    outputStream.write(new byte[FILE_SIZE]);
+    outputStream.close();
+
+    // The exact number of requests/responses that happen to create a file
+    // can vary  - at the time of writing this code it takes 34
+    // requests/responses for the 100 MB file,
+    // plus the initial container check request, but that
+    // can very easily change in the future. Just assert that we do roughly
+    // more than 20 but less than 50.
+    logOpResponseCount("Creating a 100 MB file", base);
+    base = assertWebResponsesInRange(base, 20, 50);
+    getBandwidthGaugeUpdater().triggerUpdate(true);
+    long totalBytesWritten = AzureMetricsTestUtil.getCurrentTotalBytesWritten(getInstrumentation());
+    assertTrue("The total bytes written  " + totalBytesWritten +
+        " is pretty far from the expected range of around " + FILE_SIZE +
+        " bytes plus a little overhead.",
+        totalBytesWritten >= FILE_SIZE && totalBytesWritten < (FILE_SIZE * 2));
+    long uploadRate = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(), WASB_UPLOAD_RATE);
+    LOG.info("Upload rate: " + uploadRate + " bytes/second.");
+    long uploadLatency = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(),
+        WASB_UPLOAD_LATENCY);
+    LOG.info("Upload latency: " + uploadLatency);
+    assertTrue("The upload latency " + uploadLatency +
+        " should be greater than zero now that I've just uploaded a file.",
+        uploadLatency > 0);
+
+    // Read the file
+    InputStream inputStream = getFileSystem().open(filePath);
+    int count = 0;
+    while (inputStream.read() >= 0) {
+      count++;
+    }
+    inputStream.close();
+    assertEquals(FILE_SIZE, count);
+
+    // Again, exact number varies. At the time of writing this code
+    // it takes 27 request/responses, so just assert a rough range between
+    // 20 and 40.
+    logOpResponseCount("Reading a 100 MB file", base);
+    base = assertWebResponsesInRange(base, 20, 40);
+    getBandwidthGaugeUpdater().triggerUpdate(false);
+    long totalBytesRead = AzureMetricsTestUtil.getCurrentTotalBytesRead(getInstrumentation());
+    assertEquals(FILE_SIZE, totalBytesRead);
+    long downloadRate = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(), WASB_DOWNLOAD_RATE);
+    LOG.info("Download rate: " + downloadRate + " bytes/second.");
+    long downloadLatency = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(),
+        WASB_DOWNLOAD_LATENCY);
+    LOG.info("Download latency: " + downloadLatency);
+    assertTrue("The download latency " + downloadLatency +
+        " should be greater than zero now that I've just downloaded a file.",
+        downloadLatency > 0);
+  }
+
+  @Test
+  public void testMetricsOnFileRename() throws Exception {
+    long base = getBaseWebResponses();
+
+    Path originalPath = new Path("/metricsTest_RenameStart");
+    Path destinationPath = new Path("/metricsTest_RenameFinal");
+
+    // Create an empty file
+    assertEquals(0, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_FILES_CREATED));
+    assertTrue(getFileSystem().createNewFile(originalPath));
+    logOpResponseCount("Creating an empty file", base);
+    base = assertWebResponsesInRange(base, 2, 20);
+    assertEquals(1, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_FILES_CREATED));
+
+    // Rename the file
+    assertTrue(
+        ((FileSystem) getFileSystem()).rename(originalPath, destinationPath));
+    // Varies: at the time of writing this code it takes 7 requests/responses.
+    logOpResponseCount("Renaming a file", base);
+    base = assertWebResponsesInRange(base, 2, 15);
+
+    assertNoErrors();
+  }
+
+  @Test
+  public void testMetricsOnFileExistsDelete() throws Exception {
+    long base = getBaseWebResponses();
+
+    Path filePath = new Path("/metricsTest_delete");
+
+    // Check existence
+    assertFalse(getFileSystem().exists(filePath));
+    // At the time of writing this code it takes 2 requests/responses to
+    // check existence, which seems excessive, plus initial request for
+    // container check.
+    logOpResponseCount("Checking file existence for non-existent file", base);
+    base = assertWebResponsesInRange(base, 1, 3);
+
+    // Create an empty file
+    assertTrue(getFileSystem().createNewFile(filePath));
+    base = getCurrentWebResponses();
+
+    // Check existence again
+    assertTrue(getFileSystem().exists(filePath));
+    logOpResponseCount("Checking file existence for existent file", base);
+    base = assertWebResponsesInRange(base, 1, 2);
+
+    // Delete the file
+    assertEquals(0, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_FILES_DELETED));
+    assertTrue(getFileSystem().delete(filePath, false));
+    // At the time of writing this code it takes 4 requests/responses to
+    // delete, which seems excessive. Check for range 1-4 for now.
+    logOpResponseCount("Deleting a file", base);
+    base = assertWebResponsesInRange(base, 1, 4);
+    assertEquals(1, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_FILES_DELETED));
+
+    assertNoErrors();
+  }
+
+  @Test
+  public void testMetricsOnDirRename() throws Exception {
+    long base = getBaseWebResponses();
+
+    Path originalDirName = new Path("/metricsTestDirectory_RenameStart");
+    Path innerFileName = new Path(originalDirName, "innerFile");
+    Path destDirName = new Path("/metricsTestDirectory_RenameFinal");
+
+    // Create an empty directory
+    assertTrue(getFileSystem().mkdirs(originalDirName));
+    base = getCurrentWebResponses();
+
+    // Create an inner file
+    assertTrue(getFileSystem().createNewFile(innerFileName));
+    base = getCurrentWebResponses();
+
+    // Rename the directory
+    assertTrue(getFileSystem().rename(originalDirName, destDirName));
+
+    // At the time of writing this code it takes 11 requests/responses
+    // to rename the directory with one file. Check for range 1-20 for now.
+    logOpResponseCount("Renaming a directory", base);
+    base = assertWebResponsesInRange(base, 1, 20);
+
+    assertNoErrors();
+  }
+
+  /**
+   * Recursive discovery of path depth
+   * @param path path to measure.
+   * @return depth, where "/" == 0.
+   */
+  int depth(Path path) {
+    if (path.isRoot()) {
+      return 0;
+    } else {
+      return 1 + depth(path.getParent());
+    }
+  }
+
+  @Test
+  public void testClientErrorMetrics() throws Exception {
+    String fileName = "metricsTestFile_ClientError";
+    Path filePath = new Path("/"+fileName);
+    final int FILE_SIZE = 100;
+    OutputStream outputStream = null;
+    String leaseID = null;
+    try {
+      // Create a file
+      outputStream = getFileSystem().create(filePath);
+      leaseID = getTestAccount().acquireShortLease(fileName);
+      try {
+        outputStream.write(new byte[FILE_SIZE]);
+        outputStream.close();
+        assertTrue("Should've thrown", false);
+      } catch (AzureException ex) {
+        assertTrue("Unexpected exception: " + ex,
+          ex.getMessage().contains("lease"));
+      }
+      assertEquals(1, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_CLIENT_ERRORS));
+      assertEquals(0, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_SERVER_ERRORS));
+    } finally {
+      if(leaseID != null){
+        getTestAccount().releaseLease(leaseID, fileName);
+      }
+      IOUtils.closeStream(outputStream);
+    }
+  }
+
+  private void logOpResponseCount(String opName, long base) {
+    LOG.info("{}  took {} web responses to complete.",
+        opName, getCurrentWebResponses() - base);
+  }
+
+  /**
+   * Gets (and asserts) the value of the wasb_web_responses counter just
+   * after the creation of the file system object.
+   */
+  private long getBaseWebResponses() {
+    // The number of requests should start at 0
+    return assertWebResponsesEquals(0, 0);
+  }
+
+  /**
+   * Gets the current value of the wasb_web_responses counter.
+   */
+  private long getCurrentWebResponses() {
+      return AzureMetricsTestUtil.getCurrentWebResponses(getInstrumentation());
+  }
+
+  /**
+   * Checks that the wasb_web_responses counter is at the given value.
+   * @param base The base value (before the operation of interest).
+   * @param expected The expected value for the operation of interest.
+   * @return The new base value now.
+   */
+  private long assertWebResponsesEquals(long base, long expected) {
+    assertCounter(WASB_WEB_RESPONSES, base + expected, getMyMetrics());
+    return base + expected;
+  }
+
+  private void assertNoErrors() {
+    assertEquals(0, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_CLIENT_ERRORS));
+    assertEquals(0, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_SERVER_ERRORS));
+  }
+
+  /**
+   * Checks that the wasb_web_responses counter is in the given range.
+   * @param base The base value (before the operation of interest).
+   * @param inclusiveLowerLimit The lower limit for what it should increase by.
+   * @param inclusiveUpperLimit The upper limit for what it should increase by.
+   * @return The new base value now.
+   */
+  private long assertWebResponsesInRange(long base,
+      long inclusiveLowerLimit,
+      long inclusiveUpperLimit) {
+    long currentResponses = getCurrentWebResponses();
+    long justOperation = currentResponses - base;
+    assertTrue(String.format(
+        "Web responses expected in range [%d, %d], but was %d.",
+        inclusiveLowerLimit, inclusiveUpperLimit, justOperation),
+        justOperation >= inclusiveLowerLimit &&
+        justOperation <= inclusiveUpperLimit);
+    return currentResponses;
+  }  
+
+  /**
+   * Gets the metrics for the file system object.
+   * @return The metrics record.
+   */
+  private MetricsRecordBuilder getMyMetrics() {
+    return getMetrics(getInstrumentation());
+  }
+
+  private AzureFileSystemInstrumentation getInstrumentation() {
+    return getFileSystem().getInstrumentation();
+  }
+
+  /**
+   * A matcher class for asserting that we got a tag with a given
+   * value.
+   */
+  private static class TagMatcher extends TagExistsMatcher {
+    private final String tagValue;
+
+    public TagMatcher(String tagName, String tagValue) {
+      super(tagName);
+      this.tagValue = tagValue;
+    }
+
+    @Override
+    public boolean matches(MetricsTag toMatch) {
+      return toMatch.value().equals(tagValue);
+    }
+
+    @Override
+    public void describeTo(Description desc) {
+      super.describeTo(desc);
+      desc.appendText(" with value " + tagValue);
+    }
+  }
+
+  /**
+   * A matcher class for asserting that we got a tag with any value.
+   */
+  private static class TagExistsMatcher extends BaseMatcher<MetricsTag> {
+    private final String tagName;
+
+    public TagExistsMatcher(String tagName) {
+      this.tagName = tagName;
+    }
+
+    @Override
+    public boolean matches(Object toMatch) {
+      MetricsTag asTag = (MetricsTag)toMatch;
+      return asTag.name().equals(tagName) && matches(asTag);
+    }
+
+    protected boolean matches(MetricsTag toMatch) {
+      return true;
+    }
+
+    @Override
+    public void describeTo(Description desc) {
+      desc.appendText("Has tag " + tagName);
+    }
+  }
+
+  /**
+   * A matcher class for asserting that a long value is in a
+   * given range.
+   */
+  private static class InRange extends BaseMatcher<Long> {
+    private final long inclusiveLowerLimit;
+    private final long inclusiveUpperLimit;
+    private long obtained;
+
+    public InRange(long inclusiveLowerLimit, long inclusiveUpperLimit) {
+      this.inclusiveLowerLimit = inclusiveLowerLimit;
+      this.inclusiveUpperLimit = inclusiveUpperLimit;
+    }
+
+    @Override
+    public boolean matches(Object number) {
+      obtained = (Long)number;
+      return obtained >= inclusiveLowerLimit &&
+          obtained <= inclusiveUpperLimit;
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description.appendText("Between " + inclusiveLowerLimit +
+          " and " + inclusiveUpperLimit + " inclusively");
+    }
+  }
+}


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


[28/50] [abbrv] hadoop git commit: HADOOP-14553. Add (parallelized) integration tests to hadoop-azure Contributed by Steve Loughran

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestBlobDataValidation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestBlobDataValidation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestBlobDataValidation.java
new file mode 100644
index 0000000..0aa9393
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestBlobDataValidation.java
@@ -0,0 +1,244 @@
+/**
+ * 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.azure;
+
+import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.KEY_CHECK_BLOCK_MD5;
+import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.KEY_STORE_BLOB_MD5;
+import static org.junit.Assume.assumeNotNull;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.util.Arrays;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.TestHookOperationContext;
+import org.apache.hadoop.fs.azure.integration.AzureTestUtils;
+
+import org.junit.After;
+import org.junit.Test;
+
+import com.microsoft.azure.storage.Constants;
+import com.microsoft.azure.storage.OperationContext;
+import com.microsoft.azure.storage.ResponseReceivedEvent;
+import com.microsoft.azure.storage.StorageErrorCodeStrings;
+import com.microsoft.azure.storage.StorageEvent;
+import com.microsoft.azure.storage.StorageException;
+import com.microsoft.azure.storage.blob.BlockEntry;
+import com.microsoft.azure.storage.blob.BlockSearchMode;
+import com.microsoft.azure.storage.blob.CloudBlockBlob;
+import com.microsoft.azure.storage.core.Base64;
+
+/**
+ * Test that we do proper data integrity validation with MD5 checks as
+ * configured.
+ */
+public class ITestBlobDataValidation extends AbstractWasbTestWithTimeout {
+  private AzureBlobStorageTestAccount testAccount;
+
+  @After
+  public void tearDown() throws Exception {
+    testAccount = AzureTestUtils.cleanupTestAccount(testAccount);
+  }
+
+  /**
+   * Test that by default we don't store the blob-level MD5.
+   */
+  @Test
+  public void testBlobMd5StoreOffByDefault() throws Exception {
+    testAccount = AzureBlobStorageTestAccount.create();
+    testStoreBlobMd5(false);
+  }
+
+  /**
+   * Test that we get blob-level MD5 storage and validation if we specify that
+   * in the configuration.
+   */
+  @Test
+  public void testStoreBlobMd5() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(KEY_STORE_BLOB_MD5, true);
+    testAccount = AzureBlobStorageTestAccount.create(conf);
+    testStoreBlobMd5(true);
+  }
+
+  /**
+   * Trims a suffix/prefix from the given string. For example if
+   * s is given as "/xy" and toTrim is "/", this method returns "xy"
+   */
+  private static String trim(String s, String toTrim) {
+    return StringUtils.removeEnd(StringUtils.removeStart(s, toTrim),
+        toTrim);
+  }
+
+  private void testStoreBlobMd5(boolean expectMd5Stored) throws Exception {
+    assumeNotNull(testAccount);
+    // Write a test file.
+    NativeAzureFileSystem fs = testAccount.getFileSystem();
+    Path testFilePath = AzureTestUtils.pathForTests(fs,
+        methodName.getMethodName());
+    String testFileKey = trim(testFilePath.toUri().getPath(), "/");
+    OutputStream outStream = fs.create(testFilePath);
+    outStream.write(new byte[] { 5, 15 });
+    outStream.close();
+
+    // Check that we stored/didn't store the MD5 field as configured.
+    CloudBlockBlob blob = testAccount.getBlobReference(testFileKey);
+    blob.downloadAttributes();
+    String obtainedMd5 = blob.getProperties().getContentMD5();
+    if (expectMd5Stored) {
+      assertNotNull(obtainedMd5);
+    } else {
+      assertNull("Expected no MD5, found: " + obtainedMd5, obtainedMd5);
+    }
+
+    // Mess with the content so it doesn't match the MD5.
+    String newBlockId = Base64.encode(new byte[] { 55, 44, 33, 22 });
+    blob.uploadBlock(newBlockId,
+        new ByteArrayInputStream(new byte[] { 6, 45 }), 2);
+    blob.commitBlockList(Arrays.asList(new BlockEntry[] { new BlockEntry(
+        newBlockId, BlockSearchMode.UNCOMMITTED) }));
+
+    // Now read back the content. If we stored the MD5 for the blob content
+    // we should get a data corruption error.
+    InputStream inStream = fs.open(testFilePath);
+    try {
+      byte[] inBuf = new byte[100];
+      while (inStream.read(inBuf) > 0){
+        //nothing;
+      }
+      inStream.close();
+      if (expectMd5Stored) {
+        fail("Should've thrown because of data corruption.");
+      }
+    } catch (IOException ex) {
+      if (!expectMd5Stored) {
+        throw ex;
+      }
+      StorageException cause = (StorageException)ex.getCause();
+      assertNotNull(cause);
+      assertEquals("Unexpected cause: " + cause,
+          StorageErrorCodeStrings.INVALID_MD5, cause.getErrorCode());
+    }
+  }
+
+  /**
+   * Test that by default we check block-level MD5.
+   */
+  @Test
+  public void testCheckBlockMd5() throws Exception {
+    testAccount = AzureBlobStorageTestAccount.create();
+    testCheckBlockMd5(true);
+  }
+
+  /**
+   * Test that we don't check block-level MD5 if we specify that in the
+   * configuration.
+   */
+  @Test
+  public void testDontCheckBlockMd5() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(KEY_CHECK_BLOCK_MD5, false);
+    testAccount = AzureBlobStorageTestAccount.create(conf);
+    testCheckBlockMd5(false);
+  }
+
+  /**
+   * Connection inspector to check that MD5 fields for content is set/not set as
+   * expected.
+   */
+  private static class ContentMD5Checker extends
+      StorageEvent<ResponseReceivedEvent> {
+    private final boolean expectMd5;
+
+    public ContentMD5Checker(boolean expectMd5) {
+      this.expectMd5 = expectMd5;
+    }
+
+    @Override
+    public void eventOccurred(ResponseReceivedEvent eventArg) {
+      HttpURLConnection connection = (HttpURLConnection) eventArg
+          .getConnectionObject();
+      if (isGetRange(connection)) {
+        checkObtainedMd5(connection
+            .getHeaderField(Constants.HeaderConstants.CONTENT_MD5));
+      } else if (isPutBlock(connection)) {
+        checkObtainedMd5(connection
+            .getRequestProperty(Constants.HeaderConstants.CONTENT_MD5));
+      }
+    }
+
+    private void checkObtainedMd5(String obtainedMd5) {
+      if (expectMd5) {
+        assertNotNull(obtainedMd5);
+      } else {
+        assertNull("Expected no MD5, found: " + obtainedMd5, obtainedMd5);
+      }
+    }
+
+    private static boolean isPutBlock(HttpURLConnection connection) {
+      return connection.getRequestMethod().equals("PUT")
+          && connection.getURL().getQuery() != null
+          && connection.getURL().getQuery().contains("blockid");
+    }
+
+    private static boolean isGetRange(HttpURLConnection connection) {
+      return connection.getRequestMethod().equals("GET")
+          && connection
+              .getHeaderField(Constants.HeaderConstants.STORAGE_RANGE_HEADER) != null;
+    }
+  }
+
+  private void testCheckBlockMd5(final boolean expectMd5Checked)
+      throws Exception {
+    assumeNotNull(testAccount);
+    Path testFilePath = new Path("/testFile");
+
+    // Add a hook to check that for GET/PUT requests we set/don't set
+    // the block-level MD5 field as configured. I tried to do clever
+    // testing by also messing with the raw data to see if we actually
+    // validate the data as expected, but the HttpURLConnection wasn't
+    // pluggable enough for me to do that.
+    testAccount.getFileSystem().getStore()
+    .addTestHookToOperationContext(new TestHookOperationContext() {
+    @Override
+          public OperationContext modifyOperationContext(
+              OperationContext original) {
+      original.getResponseReceivedEventHandler().addListener(
+          new ContentMD5Checker(expectMd5Checked));
+      return original;
+          }
+        });
+
+    OutputStream outStream = testAccount.getFileSystem().create(testFilePath);
+    outStream.write(new byte[] { 5, 15 });
+    outStream.close();
+
+    InputStream inStream = testAccount.getFileSystem().open(testFilePath);
+    byte[] inBuf = new byte[100];
+    while (inStream.read(inBuf) > 0){
+      //nothing;
+    }
+    inStream.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestBlobTypeSpeedDifference.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestBlobTypeSpeedDifference.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestBlobTypeSpeedDifference.java
new file mode 100644
index 0000000..b46ad5b
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestBlobTypeSpeedDifference.java
@@ -0,0 +1,163 @@
+/**
+ * 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.azure;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Date;
+
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azure.integration.AzureTestUtils;
+import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
+
+
+/**
+ * A simple benchmark to find out the difference in speed between block
+ * and page blobs.
+ */
+public class ITestBlobTypeSpeedDifference extends AbstractWasbTestBase {
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create();
+  }
+
+  /**
+   * Writes data to the given stream of the given size, flushing every
+   * x bytes.
+   */
+  private static void writeTestFile(OutputStream writeStream,
+      long size, long flushInterval) throws IOException {
+    int bufferSize = (int) Math.min(1000, flushInterval);
+    byte[] buffer = new byte[bufferSize];
+    Arrays.fill(buffer, (byte) 7);
+    int bytesWritten = 0;
+    int bytesUnflushed = 0;
+    while (bytesWritten < size) {
+      int numberToWrite = (int) Math.min(bufferSize, size - bytesWritten);
+      writeStream.write(buffer, 0, numberToWrite);
+      bytesWritten += numberToWrite;
+      bytesUnflushed += numberToWrite;
+      if (bytesUnflushed >= flushInterval) {
+        writeStream.flush();
+        bytesUnflushed = 0;
+      }
+    }
+  }
+
+  private static class TestResult {
+    final long timeTakenInMs;
+    final long totalNumberOfRequests;
+
+    TestResult(long timeTakenInMs, long totalNumberOfRequests) {
+      this.timeTakenInMs = timeTakenInMs;
+      this.totalNumberOfRequests = totalNumberOfRequests;
+    }
+  }
+
+  /**
+   * Writes data to the given file of the given size, flushing every
+   * x bytes. Measure performance of that and return it.
+   */
+  private static TestResult writeTestFile(NativeAzureFileSystem fs, Path path,
+      long size, long flushInterval) throws IOException {
+    AzureFileSystemInstrumentation instrumentation =
+        fs.getInstrumentation();
+    long initialRequests = instrumentation.getCurrentWebResponses();
+    Date start = new Date();
+    OutputStream output = fs.create(path);
+    writeTestFile(output, size, flushInterval);
+    output.close();
+    long finalRequests = instrumentation.getCurrentWebResponses();
+    return new TestResult(new Date().getTime() - start.getTime(),
+        finalRequests - initialRequests);
+  }
+
+  /**
+   * Writes data to a block blob of the given size, flushing every
+   * x bytes. Measure performance of that and return it.
+   */
+  private static TestResult writeBlockBlobTestFile(NativeAzureFileSystem fs,
+      long size, long flushInterval) throws IOException {
+    return writeTestFile(fs, new Path("/blockBlob"), size, flushInterval);
+  }
+
+  /**
+   * Writes data to a page blob of the given size, flushing every
+   * x bytes. Measure performance of that and return it.
+   */
+  private static TestResult writePageBlobTestFile(NativeAzureFileSystem fs,
+      long size, long flushInterval) throws IOException {
+    Path testFile = AzureTestUtils.blobPathForTests(fs,
+        "writePageBlobTestFile");
+    return writeTestFile(fs,
+        testFile,
+        size, flushInterval);
+  }
+
+  /**
+   * Runs the benchmark over a small 10 KB file, flushing every 500 bytes.
+   */
+  @Test
+  public void testTenKbFileFrequentFlush() throws Exception {
+    testForSizeAndFlushInterval(getFileSystem(), 10 * 1000, 500);
+  }
+
+  /**
+   * Runs the benchmark for the given file size and flush frequency.
+   */
+  private static void testForSizeAndFlushInterval(NativeAzureFileSystem fs,
+      final long size, final long flushInterval) throws IOException {
+    for (int i = 0; i < 5; i++) {
+      TestResult pageBlobResults = writePageBlobTestFile(fs, size, flushInterval);
+      System.out.printf(
+          "Page blob upload took %d ms. Total number of requests: %d.\n",
+          pageBlobResults.timeTakenInMs, pageBlobResults.totalNumberOfRequests);
+      TestResult blockBlobResults = writeBlockBlobTestFile(fs, size, flushInterval);
+      System.out.printf(
+          "Block blob upload took %d ms. Total number of requests: %d.\n",
+          blockBlobResults.timeTakenInMs, blockBlobResults.totalNumberOfRequests);
+    }
+  }
+
+  /**
+   * Runs the benchmark for the given file size and flush frequency from the
+   * command line.
+   */
+  public static void main(String[] argv) throws Exception {
+    Configuration conf = new Configuration();
+    long size = 10 * 1000 * 1000;
+    long flushInterval = 2000;
+    if (argv.length > 0) {
+      size = Long.parseLong(argv[0]);
+    }
+    if (argv.length > 1) {
+      flushInterval = Long.parseLong(argv[1]);
+    }
+    testForSizeAndFlushInterval(
+        (NativeAzureFileSystem) FileSystem.get(conf),
+        size,
+        flushInterval);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestBlockBlobInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestBlockBlobInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestBlockBlobInputStream.java
new file mode 100644
index 0000000..07a13df
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestBlockBlobInputStream.java
@@ -0,0 +1,874 @@
+/*
+ * 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.azure;
+
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.Random;
+import java.util.concurrent.Callable;
+
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azure.integration.AbstractAzureScaleTest;
+import org.apache.hadoop.fs.azure.integration.AzureTestUtils;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.contract.ContractTestUtils.NanoTimer;
+
+import static org.junit.Assume.assumeNotNull;
+
+import static org.apache.hadoop.test.LambdaTestUtils.*;
+
+/**
+ * Test semantics and performance of the original block blob input stream
+ * (KEY_INPUT_STREAM_VERSION=1) and the new
+ * <code>BlockBlobInputStream</code> (KEY_INPUT_STREAM_VERSION=2).
+ */
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+
+public class ITestBlockBlobInputStream extends AbstractAzureScaleTest {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      ITestBlockBlobInputStream.class);
+  private static final int KILOBYTE = 1024;
+  private static final int MEGABYTE = KILOBYTE * KILOBYTE;
+  private static final int TEST_FILE_SIZE = 6 * MEGABYTE;
+  private static final Path TEST_FILE_PATH = new Path(
+      "TestBlockBlobInputStream.txt");
+
+  private AzureBlobStorageTestAccount accountUsingInputStreamV1;
+  private AzureBlobStorageTestAccount accountUsingInputStreamV2;
+  private long testFileLength;
+
+
+
+  private FileStatus testFileStatus;
+  private Path hugefile;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    Configuration conf = new Configuration();
+    conf.setInt(AzureNativeFileSystemStore.KEY_INPUT_STREAM_VERSION, 1);
+
+    accountUsingInputStreamV1 = AzureBlobStorageTestAccount.create(
+        "testblockblobinputstream",
+        EnumSet.of(AzureBlobStorageTestAccount.CreateOptions.CreateContainer),
+        conf,
+        true);
+
+    accountUsingInputStreamV2 = AzureBlobStorageTestAccount.create(
+        "testblockblobinputstream",
+        EnumSet.noneOf(AzureBlobStorageTestAccount.CreateOptions.class),
+        null,
+        true);
+
+    assumeNotNull(accountUsingInputStreamV1);
+    assumeNotNull(accountUsingInputStreamV2);
+    hugefile = fs.makeQualified(TEST_FILE_PATH);
+    try {
+      testFileStatus = fs.getFileStatus(TEST_FILE_PATH);
+      testFileLength = testFileStatus.getLen();
+    } catch (FileNotFoundException e) {
+      // file doesn't exist
+      testFileLength = 0;
+    }
+  }
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(AzureNativeFileSystemStore.KEY_INPUT_STREAM_VERSION, 1);
+
+    accountUsingInputStreamV1 = AzureBlobStorageTestAccount.create(
+        "testblockblobinputstream",
+        EnumSet.of(AzureBlobStorageTestAccount.CreateOptions.CreateContainer),
+        conf,
+        true);
+
+    accountUsingInputStreamV2 = AzureBlobStorageTestAccount.create(
+        "testblockblobinputstream",
+        EnumSet.noneOf(AzureBlobStorageTestAccount.CreateOptions.class),
+        null,
+        true);
+
+    assumeNotNull(accountUsingInputStreamV1);
+    assumeNotNull(accountUsingInputStreamV2);
+    return accountUsingInputStreamV1;
+  }
+
+  /**
+   * Create a test file by repeating the characters in the alphabet.
+   * @throws IOException
+   */
+  private void createTestFileAndSetLength() throws IOException {
+    FileSystem fs = accountUsingInputStreamV1.getFileSystem();
+
+    // To reduce test run time, the test file can be reused.
+    if (fs.exists(TEST_FILE_PATH)) {
+      testFileStatus = fs.getFileStatus(TEST_FILE_PATH);
+      testFileLength = testFileStatus.getLen();
+      LOG.info("Reusing test file: {}", testFileStatus);
+      return;
+    }
+
+    int sizeOfAlphabet = ('z' - 'a' + 1);
+    byte[] buffer = new byte[26 * KILOBYTE];
+    char character = 'a';
+    for (int i = 0; i < buffer.length; i++) {
+      buffer[i] = (byte) character;
+      character = (character == 'z') ? 'a' : (char) ((int) character + 1);
+    }
+
+    LOG.info("Creating test file {} of size: {}", TEST_FILE_PATH,
+        TEST_FILE_SIZE);
+    ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
+
+    try(FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) {
+      int bytesWritten = 0;
+      while (bytesWritten < TEST_FILE_SIZE) {
+        outputStream.write(buffer);
+        bytesWritten += buffer.length;
+      }
+      LOG.info("Closing stream {}", outputStream);
+      ContractTestUtils.NanoTimer closeTimer
+          = new ContractTestUtils.NanoTimer();
+      outputStream.close();
+      closeTimer.end("time to close() output stream");
+    }
+    timer.end("time to write %d KB", TEST_FILE_SIZE / 1024);
+    testFileLength = fs.getFileStatus(TEST_FILE_PATH).getLen();
+  }
+
+  void assumeHugeFileExists() throws IOException {
+    ContractTestUtils.assertPathExists(fs, "huge file not created", hugefile);
+    FileStatus status = fs.getFileStatus(hugefile);
+    ContractTestUtils.assertIsFile(hugefile, status);
+    assertTrue("File " + hugefile + " is empty", status.getLen() > 0);
+  }
+
+  /**
+   * Calculate megabits per second from the specified values for bytes and
+   * milliseconds.
+   * @param bytes The number of bytes.
+   * @param milliseconds The number of milliseconds.
+   * @return The number of megabits per second.
+   */
+  private static double toMbps(long bytes, long milliseconds) {
+    return bytes / 1000.0 * 8 / milliseconds;
+  }
+
+  @Test
+  public void test_0100_CreateHugeFile() throws IOException {
+    createTestFileAndSetLength();
+  }
+
+  @Test
+  public void test_0200_BasicReadTest() throws Exception {
+    assumeHugeFileExists();
+
+    try (
+        FSDataInputStream inputStreamV1
+            = accountUsingInputStreamV1.getFileSystem().open(TEST_FILE_PATH);
+
+        FSDataInputStream inputStreamV2
+            = accountUsingInputStreamV2.getFileSystem().open(TEST_FILE_PATH);
+    ) {
+      byte[] bufferV1 = new byte[3 * MEGABYTE];
+      byte[] bufferV2 = new byte[bufferV1.length];
+
+      // v1 forward seek and read a kilobyte into first kilobyte of bufferV1
+      inputStreamV1.seek(5 * MEGABYTE);
+      int numBytesReadV1 = inputStreamV1.read(bufferV1, 0, KILOBYTE);
+      assertEquals(KILOBYTE, numBytesReadV1);
+
+      // v2 forward seek and read a kilobyte into first kilobyte of bufferV2
+      inputStreamV2.seek(5 * MEGABYTE);
+      int numBytesReadV2 = inputStreamV2.read(bufferV2, 0, KILOBYTE);
+      assertEquals(KILOBYTE, numBytesReadV2);
+
+      assertArrayEquals(bufferV1, bufferV2);
+
+      int len = MEGABYTE;
+      int offset = bufferV1.length - len;
+
+      // v1 reverse seek and read a megabyte into last megabyte of bufferV1
+      inputStreamV1.seek(3 * MEGABYTE);
+      numBytesReadV1 = inputStreamV1.read(bufferV1, offset, len);
+      assertEquals(len, numBytesReadV1);
+
+      // v2 reverse seek and read a megabyte into last megabyte of bufferV2
+      inputStreamV2.seek(3 * MEGABYTE);
+      numBytesReadV2 = inputStreamV2.read(bufferV2, offset, len);
+      assertEquals(len, numBytesReadV2);
+
+      assertArrayEquals(bufferV1, bufferV2);
+    }
+  }
+
+  @Test
+  public void test_0201_RandomReadTest() throws Exception {
+    assumeHugeFileExists();
+
+    try (
+        FSDataInputStream inputStreamV1
+            = accountUsingInputStreamV1.getFileSystem().open(TEST_FILE_PATH);
+
+        FSDataInputStream inputStreamV2
+            = accountUsingInputStreamV2.getFileSystem().open(TEST_FILE_PATH);
+    ) {
+      final int bufferSize = 4 * KILOBYTE;
+      byte[] bufferV1 = new byte[bufferSize];
+      byte[] bufferV2 = new byte[bufferV1.length];
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      inputStreamV1.seek(0);
+      inputStreamV2.seek(0);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      int seekPosition = 2 * KILOBYTE;
+      inputStreamV1.seek(seekPosition);
+      inputStreamV2.seek(seekPosition);
+
+      inputStreamV1.seek(0);
+      inputStreamV2.seek(0);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      seekPosition = 5 * KILOBYTE;
+      inputStreamV1.seek(seekPosition);
+      inputStreamV2.seek(seekPosition);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      seekPosition = 10 * KILOBYTE;
+      inputStreamV1.seek(seekPosition);
+      inputStreamV2.seek(seekPosition);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+
+      seekPosition = 4100 * KILOBYTE;
+      inputStreamV1.seek(seekPosition);
+      inputStreamV2.seek(seekPosition);
+
+      verifyConsistentReads(inputStreamV1, inputStreamV2, bufferV1, bufferV2);
+    }
+  }
+
+  private void verifyConsistentReads(FSDataInputStream inputStreamV1,
+      FSDataInputStream inputStreamV2,
+      byte[] bufferV1,
+      byte[] bufferV2) throws IOException {
+    int size = bufferV1.length;
+    final int numBytesReadV1 = inputStreamV1.read(bufferV1, 0, size);
+    assertEquals("Bytes read from V1 stream", size, numBytesReadV1);
+
+    final int numBytesReadV2 = inputStreamV2.read(bufferV2, 0, size);
+    assertEquals("Bytes read from V2 stream", size, numBytesReadV2);
+
+    assertArrayEquals("Mismatch in read data", bufferV1, bufferV2);
+  }
+
+  /**
+   * Validates the implementation of InputStream.markSupported.
+   * @throws IOException
+   */
+  @Test
+  public void test_0301_MarkSupportedV1() throws IOException {
+    validateMarkSupported(accountUsingInputStreamV1.getFileSystem());
+  }
+
+  /**
+   * Validates the implementation of InputStream.markSupported.
+   * @throws IOException
+   */
+  @Test
+  public void test_0302_MarkSupportedV2() throws IOException {
+    validateMarkSupported(accountUsingInputStreamV1.getFileSystem());
+  }
+
+  private void validateMarkSupported(FileSystem fs) throws IOException {
+    assumeHugeFileExists();
+    try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
+      assertTrue("mark is not supported", inputStream.markSupported());
+    }
+  }
+
+  /**
+   * Validates the implementation of InputStream.mark and reset
+   * for version 1 of the block blob input stream.
+   * @throws Exception
+   */
+  @Test
+  public void test_0303_MarkAndResetV1() throws Exception {
+    validateMarkAndReset(accountUsingInputStreamV1.getFileSystem());
+  }
+
+  /**
+   * Validates the implementation of InputStream.mark and reset
+   * for version 2 of the block blob input stream.
+   * @throws Exception
+   */
+  @Test
+  public void test_0304_MarkAndResetV2() throws Exception {
+    validateMarkAndReset(accountUsingInputStreamV2.getFileSystem());
+  }
+
+  private void validateMarkAndReset(FileSystem fs) throws Exception {
+    assumeHugeFileExists();
+    try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
+      inputStream.mark(KILOBYTE - 1);
+
+      byte[] buffer = new byte[KILOBYTE];
+      int bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+
+      inputStream.reset();
+      assertEquals("rest -> pos 0", 0, inputStream.getPos());
+
+      inputStream.mark(8 * KILOBYTE - 1);
+
+      buffer = new byte[8 * KILOBYTE];
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+
+      intercept(IOException.class,
+          "Resetting to invalid mark",
+          new Callable<FSDataInputStream>() {
+            @Override
+            public FSDataInputStream call() throws Exception {
+              inputStream.reset();
+              return inputStream;
+            }
+          }
+      );
+    }
+  }
+
+  /**
+   * Validates the implementation of Seekable.seekToNewSource, which should
+   * return false for version 1 of the block blob input stream.
+   * @throws IOException
+   */
+  @Test
+  public void test_0305_SeekToNewSourceV1() throws IOException {
+    validateSeekToNewSource(accountUsingInputStreamV1.getFileSystem());
+  }
+
+  /**
+   * Validates the implementation of Seekable.seekToNewSource, which should
+   * return false for version 2 of the block blob input stream.
+   * @throws IOException
+   */
+  @Test
+  public void test_0306_SeekToNewSourceV2() throws IOException {
+    validateSeekToNewSource(accountUsingInputStreamV2.getFileSystem());
+  }
+
+  private void validateSeekToNewSource(FileSystem fs) throws IOException {
+    assumeHugeFileExists();
+    try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
+      assertFalse(inputStream.seekToNewSource(0));
+    }
+  }
+
+  /**
+   * Validates the implementation of InputStream.skip and ensures there is no
+   * network I/O for version 1 of the block blob input stream.
+   * @throws Exception
+   */
+  @Test
+  public void test_0307_SkipBoundsV1() throws Exception {
+    validateSkipBounds(accountUsingInputStreamV1.getFileSystem());
+  }
+
+  /**
+   * Validates the implementation of InputStream.skip and ensures there is no
+   * network I/O for version 2 of the block blob input stream.
+   * @throws Exception
+   */
+  @Test
+  public void test_0308_SkipBoundsV2() throws Exception {
+    validateSkipBounds(accountUsingInputStreamV2.getFileSystem());
+  }
+
+  private void validateSkipBounds(FileSystem fs) throws Exception {
+    assumeHugeFileExists();
+    try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
+      NanoTimer timer = new NanoTimer();
+
+      long skipped = inputStream.skip(-1);
+      assertEquals(0, skipped);
+
+      skipped = inputStream.skip(0);
+      assertEquals(0, skipped);
+
+      assertTrue(testFileLength > 0);
+
+      skipped = inputStream.skip(testFileLength);
+      assertEquals(testFileLength, skipped);
+
+      intercept(EOFException.class,
+          new Callable<Long>() {
+            @Override
+            public Long call() throws Exception {
+              return inputStream.skip(1);
+            }
+          }
+      );
+      long elapsedTimeMs = timer.elapsedTimeMs();
+      assertTrue(
+          String.format(
+              "There should not be any network I/O (elapsedTimeMs=%1$d).",
+              elapsedTimeMs),
+          elapsedTimeMs < 20);
+    }
+  }
+
+  /**
+   * Validates the implementation of Seekable.seek and ensures there is no
+   * network I/O for forward seek.
+   * @throws Exception
+   */
+  @Test
+  public void test_0309_SeekBoundsV1() throws Exception {
+    validateSeekBounds(accountUsingInputStreamV1.getFileSystem());
+  }
+
+  /**
+   * Validates the implementation of Seekable.seek and ensures there is no
+   * network I/O for forward seek.
+   * @throws Exception
+   */
+  @Test
+  public void test_0310_SeekBoundsV2() throws Exception {
+    validateSeekBounds(accountUsingInputStreamV2.getFileSystem());
+  }
+
+  private void validateSeekBounds(FileSystem fs) throws Exception {
+    assumeHugeFileExists();
+    try (
+        FSDataInputStream inputStream = fs.open(TEST_FILE_PATH);
+    ) {
+      NanoTimer timer = new NanoTimer();
+
+      inputStream.seek(0);
+      assertEquals(0, inputStream.getPos());
+
+      intercept(EOFException.class,
+          FSExceptionMessages.NEGATIVE_SEEK,
+          new Callable<FSDataInputStream>() {
+            @Override
+            public FSDataInputStream call() throws Exception {
+              inputStream.seek(-1);
+              return inputStream;
+            }
+          }
+      );
+
+      assertTrue("Test file length only " + testFileLength, testFileLength > 0);
+      inputStream.seek(testFileLength);
+      assertEquals(testFileLength, inputStream.getPos());
+
+      intercept(EOFException.class,
+          FSExceptionMessages.CANNOT_SEEK_PAST_EOF,
+          new Callable<FSDataInputStream>() {
+            @Override
+            public FSDataInputStream call() throws Exception {
+              inputStream.seek(testFileLength + 1);
+              return inputStream;
+            }
+          }
+      );
+
+      long elapsedTimeMs = timer.elapsedTimeMs();
+      assertTrue(
+          String.format(
+              "There should not be any network I/O (elapsedTimeMs=%1$d).",
+              elapsedTimeMs),
+          elapsedTimeMs < 20);
+    }
+  }
+
+  /**
+   * Validates the implementation of Seekable.seek, Seekable.getPos,
+   * and InputStream.available.
+   * @throws Exception
+   */
+  @Test
+  public void test_0311_SeekAndAvailableAndPositionV1() throws Exception {
+    validateSeekAndAvailableAndPosition(
+        accountUsingInputStreamV1.getFileSystem());
+  }
+
+  /**
+   * Validates the implementation of Seekable.seek, Seekable.getPos,
+   * and InputStream.available.
+   * @throws Exception
+   */
+  @Test
+  public void test_0312_SeekAndAvailableAndPositionV2() throws Exception {
+    validateSeekAndAvailableAndPosition(
+        accountUsingInputStreamV2.getFileSystem());
+  }
+
+  private void validateSeekAndAvailableAndPosition(FileSystem fs)
+      throws Exception {
+    assumeHugeFileExists();
+    try (FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
+      byte[] expected1 = {(byte) 'a', (byte) 'b', (byte) 'c'};
+      byte[] expected2 = {(byte) 'd', (byte) 'e', (byte) 'f'};
+      byte[] expected3 = {(byte) 'b', (byte) 'c', (byte) 'd'};
+      byte[] expected4 = {(byte) 'g', (byte) 'h', (byte) 'i'};
+      byte[] buffer = new byte[3];
+
+      int bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected1, buffer);
+      assertEquals(buffer.length, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected2, buffer);
+      assertEquals(2 * buffer.length, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+
+      // reverse seek
+      int seekPos = 0;
+      inputStream.seek(seekPos);
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected1, buffer);
+      assertEquals(buffer.length + seekPos, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+
+      // reverse seek
+      seekPos = 1;
+      inputStream.seek(seekPos);
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected3, buffer);
+      assertEquals(buffer.length + seekPos, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+
+      // forward seek
+      seekPos = 6;
+      inputStream.seek(seekPos);
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected4, buffer);
+      assertEquals(buffer.length + seekPos, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+    }
+  }
+
+  /**
+   * Validates the implementation of InputStream.skip, Seekable.getPos,
+   * and InputStream.available.
+   * @throws IOException
+   */
+  @Test
+  public void test_0313_SkipAndAvailableAndPositionV1() throws IOException {
+    validateSkipAndAvailableAndPosition(
+        accountUsingInputStreamV1.getFileSystem());
+  }
+
+  /**
+   * Validates the implementation of InputStream.skip, Seekable.getPos,
+   * and InputStream.available.
+   * @throws IOException
+   */
+  @Test
+  public void test_0314_SkipAndAvailableAndPositionV2() throws IOException {
+    validateSkipAndAvailableAndPosition(
+        accountUsingInputStreamV1.getFileSystem());
+  }
+
+  private void validateSkipAndAvailableAndPosition(FileSystem fs)
+      throws IOException {
+    assumeHugeFileExists();
+    try (
+        FSDataInputStream inputStream = fs.open(TEST_FILE_PATH);
+    ) {
+      byte[] expected1 = {(byte) 'a', (byte) 'b', (byte) 'c'};
+      byte[] expected2 = {(byte) 'd', (byte) 'e', (byte) 'f'};
+      byte[] expected3 = {(byte) 'b', (byte) 'c', (byte) 'd'};
+      byte[] expected4 = {(byte) 'g', (byte) 'h', (byte) 'i'};
+
+      assertEquals(testFileLength, inputStream.available());
+      assertEquals(0, inputStream.getPos());
+
+      int n = 3;
+      long skipped = inputStream.skip(n);
+
+      assertEquals(skipped, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+      assertEquals(skipped, n);
+
+      byte[] buffer = new byte[3];
+      int bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected2, buffer);
+      assertEquals(buffer.length + skipped, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+
+      // does skip still work after seek?
+      int seekPos = 1;
+      inputStream.seek(seekPos);
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected3, buffer);
+      assertEquals(buffer.length + seekPos, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+
+      long currentPosition = inputStream.getPos();
+      n = 2;
+      skipped = inputStream.skip(n);
+
+      assertEquals(currentPosition + skipped, inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+      assertEquals(skipped, n);
+
+      bytesRead = inputStream.read(buffer);
+      assertEquals(buffer.length, bytesRead);
+      assertArrayEquals(expected4, buffer);
+      assertEquals(buffer.length + skipped + currentPosition,
+          inputStream.getPos());
+      assertEquals(testFileLength - inputStream.getPos(),
+          inputStream.available());
+    }
+  }
+
+  /**
+   * Ensures parity in the performance of sequential read for
+   * version 1 and version 2 of the block blob input stream.
+   * @throws IOException
+   */
+  @Test
+  public void test_0315_SequentialReadPerformance() throws IOException {
+    assumeHugeFileExists();
+    final int maxAttempts = 10;
+    final double maxAcceptableRatio = 1.01;
+    double v1ElapsedMs = 0, v2ElapsedMs = 0;
+    double ratio = Double.MAX_VALUE;
+    for (int i = 0; i < maxAttempts && ratio >= maxAcceptableRatio; i++) {
+      v1ElapsedMs = sequentialRead(1,
+          accountUsingInputStreamV1.getFileSystem(), false);
+      v2ElapsedMs = sequentialRead(2,
+          accountUsingInputStreamV2.getFileSystem(), false);
+      ratio = v2ElapsedMs / v1ElapsedMs;
+      LOG.info(String.format(
+          "v1ElapsedMs=%1$d, v2ElapsedMs=%2$d, ratio=%3$.2f",
+          (long) v1ElapsedMs,
+          (long) v2ElapsedMs,
+          ratio));
+    }
+    assertTrue(String.format(
+        "Performance of version 2 is not acceptable: v1ElapsedMs=%1$d,"
+            + " v2ElapsedMs=%2$d, ratio=%3$.2f",
+        (long) v1ElapsedMs,
+        (long) v2ElapsedMs,
+        ratio),
+        ratio < maxAcceptableRatio);
+  }
+
+  /**
+   * Ensures parity in the performance of sequential read after reverse seek for
+   * version 2 of the block blob input stream.
+   * @throws IOException
+   */
+  @Test
+  public void test_0316_SequentialReadAfterReverseSeekPerformanceV2()
+      throws IOException {
+    assumeHugeFileExists();
+    final int maxAttempts = 10;
+    final double maxAcceptableRatio = 1.01;
+    double beforeSeekElapsedMs = 0, afterSeekElapsedMs = 0;
+    double ratio = Double.MAX_VALUE;
+    for (int i = 0; i < maxAttempts && ratio >= maxAcceptableRatio; i++) {
+      beforeSeekElapsedMs = sequentialRead(2,
+          accountUsingInputStreamV2.getFileSystem(), false);
+      afterSeekElapsedMs = sequentialRead(2,
+          accountUsingInputStreamV2.getFileSystem(), true);
+      ratio = afterSeekElapsedMs / beforeSeekElapsedMs;
+      LOG.info(String.format(
+          "beforeSeekElapsedMs=%1$d, afterSeekElapsedMs=%2$d, ratio=%3$.2f",
+          (long) beforeSeekElapsedMs,
+          (long) afterSeekElapsedMs,
+          ratio));
+    }
+    assertTrue(String.format(
+        "Performance of version 2 after reverse seek is not acceptable:"
+            + " beforeSeekElapsedMs=%1$d, afterSeekElapsedMs=%2$d,"
+            + " ratio=%3$.2f",
+        (long) beforeSeekElapsedMs,
+        (long) afterSeekElapsedMs,
+        ratio),
+        ratio < maxAcceptableRatio);
+  }
+
+  private long sequentialRead(int version,
+      FileSystem fs,
+      boolean afterReverseSeek) throws IOException {
+    byte[] buffer = new byte[16 * KILOBYTE];
+    long totalBytesRead = 0;
+    long bytesRead = 0;
+
+    try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
+      if (afterReverseSeek) {
+        while (bytesRead > 0 && totalBytesRead < 4 * MEGABYTE) {
+          bytesRead = inputStream.read(buffer);
+          totalBytesRead += bytesRead;
+        }
+        totalBytesRead = 0;
+        inputStream.seek(0);
+      }
+
+      NanoTimer timer = new NanoTimer();
+      while ((bytesRead = inputStream.read(buffer)) > 0) {
+        totalBytesRead += bytesRead;
+      }
+      long elapsedTimeMs = timer.elapsedTimeMs();
+
+      LOG.info(String.format(
+          "v%1$d: bytesRead=%2$d, elapsedMs=%3$d, Mbps=%4$.2f,"
+              + " afterReverseSeek=%5$s",
+          version,
+          totalBytesRead,
+          elapsedTimeMs,
+          toMbps(totalBytesRead, elapsedTimeMs),
+          afterReverseSeek));
+
+      assertEquals(testFileLength, totalBytesRead);
+      inputStream.close();
+      return elapsedTimeMs;
+    }
+  }
+
+  @Test
+  public void test_0317_RandomReadPerformance() throws IOException {
+    assumeHugeFileExists();
+    final int maxAttempts = 10;
+    final double maxAcceptableRatio = 0.10;
+    double v1ElapsedMs = 0, v2ElapsedMs = 0;
+    double ratio = Double.MAX_VALUE;
+    for (int i = 0; i < maxAttempts && ratio >= maxAcceptableRatio; i++) {
+      v1ElapsedMs = randomRead(1,
+          accountUsingInputStreamV1.getFileSystem());
+      v2ElapsedMs = randomRead(2,
+          accountUsingInputStreamV2.getFileSystem());
+      ratio = v2ElapsedMs / v1ElapsedMs;
+      LOG.info(String.format(
+          "v1ElapsedMs=%1$d, v2ElapsedMs=%2$d, ratio=%3$.2f",
+          (long) v1ElapsedMs,
+          (long) v2ElapsedMs,
+          ratio));
+    }
+    assertTrue(String.format(
+        "Performance of version 2 is not acceptable: v1ElapsedMs=%1$d,"
+            + " v2ElapsedMs=%2$d, ratio=%3$.2f",
+        (long) v1ElapsedMs,
+        (long) v2ElapsedMs,
+        ratio),
+        ratio < maxAcceptableRatio);
+  }
+
+  private long randomRead(int version, FileSystem fs) throws IOException {
+    assumeHugeFileExists();
+    final int minBytesToRead = 2 * MEGABYTE;
+    Random random = new Random();
+    byte[] buffer = new byte[8 * KILOBYTE];
+    long totalBytesRead = 0;
+    long bytesRead = 0;
+    try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
+      NanoTimer timer = new NanoTimer();
+
+      do {
+        bytesRead = inputStream.read(buffer);
+        totalBytesRead += bytesRead;
+        inputStream.seek(random.nextInt(
+            (int) (testFileLength - buffer.length)));
+      } while (bytesRead > 0 && totalBytesRead < minBytesToRead);
+
+      long elapsedTimeMs = timer.elapsedTimeMs();
+
+      inputStream.close();
+
+      LOG.info(String.format(
+          "v%1$d: totalBytesRead=%2$d, elapsedTimeMs=%3$d, Mbps=%4$.2f",
+          version,
+          totalBytesRead,
+          elapsedTimeMs,
+          toMbps(totalBytesRead, elapsedTimeMs)));
+
+      assertTrue(minBytesToRead <= totalBytesRead);
+
+      return elapsedTimeMs;
+    }
+  }
+
+  @Test
+  public void test_999_DeleteHugeFiles() throws IOException {
+    try {
+      NanoTimer timer = new NanoTimer();
+      NativeAzureFileSystem fs = getFileSystem();
+      fs.delete(TEST_FILE_PATH, false);
+      timer.end("time to delete %s", TEST_FILE_PATH);
+    } finally {
+      // clean up the test account
+      AzureTestUtils.cleanupTestAccount(accountUsingInputStreamV1);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestContainerChecks.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestContainerChecks.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestContainerChecks.java
new file mode 100644
index 0000000..cc3baf5
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestContainerChecks.java
@@ -0,0 +1,194 @@
+/**
+ * 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.azure;
+
+import static org.junit.Assume.assumeNotNull;
+
+import java.io.FileNotFoundException;
+import java.util.EnumSet;
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.CreateOptions;
+import org.apache.hadoop.fs.azure.integration.AzureTestUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import org.junit.After;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.microsoft.azure.storage.blob.BlobOutputStream;
+import com.microsoft.azure.storage.blob.CloudBlobContainer;
+import com.microsoft.azure.storage.blob.CloudBlockBlob;
+
+/**
+ * Tests that WASB creates containers only if needed.
+ */
+public class ITestContainerChecks extends AbstractWasbTestWithTimeout {
+  private AzureBlobStorageTestAccount testAccount;
+  private boolean runningInSASMode = false;
+
+  @After
+  public void tearDown() throws Exception {
+    testAccount = AzureTestUtils.cleanup(testAccount);
+  }
+
+  @Before
+  public void setMode() {
+    runningInSASMode = AzureBlobStorageTestAccount.createTestConfiguration().
+        getBoolean(AzureNativeFileSystemStore.KEY_USE_SECURE_MODE, false);
+  }
+
+  @Test
+  public void testContainerExistAfterDoesNotExist() throws Exception {
+    testAccount = blobStorageTestAccount();
+    assumeNotNull(testAccount);
+    CloudBlobContainer container = testAccount.getRealContainer();
+    FileSystem fs = testAccount.getFileSystem();
+
+    // Starting off with the container not there
+    assertFalse(container.exists());
+
+    // A list shouldn't create the container and will set file system store
+    // state to DoesNotExist
+    try {
+      fs.listStatus(new Path("/"));
+      assertTrue("Should've thrown.", false);
+    } catch (FileNotFoundException ex) {
+      assertTrue("Unexpected exception: " + ex,
+          ex.getMessage().contains("does not exist."));
+    }
+    assertFalse(container.exists());
+
+    // Create a container outside of the WASB FileSystem
+    container.create();
+    // Add a file to the container outside of the WASB FileSystem
+    CloudBlockBlob blob = testAccount.getBlobReference("foo");
+    BlobOutputStream outputStream = blob.openOutputStream();
+    outputStream.write(new byte[10]);
+    outputStream.close();
+
+    // Make sure the file is visible
+    assertTrue(fs.exists(new Path("/foo")));
+    assertTrue(container.exists());
+  }
+
+  protected AzureBlobStorageTestAccount blobStorageTestAccount()
+      throws Exception {
+    return AzureBlobStorageTestAccount.create("",
+        EnumSet.noneOf(CreateOptions.class));
+  }
+
+  @Test
+  public void testContainerCreateAfterDoesNotExist() throws Exception {
+    testAccount = blobStorageTestAccount();
+    assumeNotNull(testAccount);
+    CloudBlobContainer container = testAccount.getRealContainer();
+    FileSystem fs = testAccount.getFileSystem();
+
+    // Starting off with the container not there
+    assertFalse(container.exists());
+
+    // A list shouldn't create the container and will set file system store
+    // state to DoesNotExist
+    try {
+      assertNull(fs.listStatus(new Path("/")));
+      assertTrue("Should've thrown.", false);
+    } catch (FileNotFoundException ex) {
+      assertTrue("Unexpected exception: " + ex,
+          ex.getMessage().contains("does not exist."));
+    }
+    assertFalse(container.exists());
+
+    // Create a container outside of the WASB FileSystem
+    container.create();
+
+    // Write should succeed
+    assertTrue(fs.createNewFile(new Path("/foo")));
+    assertTrue(container.exists());
+  }
+
+  @Test
+  public void testContainerCreateOnWrite() throws Exception {
+    testAccount = blobStorageTestAccount();
+    assumeNotNull(testAccount);
+    CloudBlobContainer container = testAccount.getRealContainer();
+    FileSystem fs = testAccount.getFileSystem();
+
+    // Starting off with the container not there
+    assertFalse(container.exists());
+
+    // A list shouldn't create the container.
+    try {
+      fs.listStatus(new Path("/"));
+      assertTrue("Should've thrown.", false);
+    } catch (FileNotFoundException ex) {
+      assertTrue("Unexpected exception: " + ex,
+          ex.getMessage().contains("does not exist."));
+    }
+    assertFalse(container.exists());
+
+    // Neither should a read.
+    Path foo = new Path("/testContainerCreateOnWrite-foo");
+    Path bar = new Path("/testContainerCreateOnWrite-bar");
+    LambdaTestUtils.intercept(FileNotFoundException.class,
+        new Callable<String>() {
+          @Override
+          public String call() throws Exception {
+            fs.open(foo).close();
+            return "Stream to " + foo;
+          }
+        }
+    );
+    assertFalse(container.exists());
+
+    // Neither should a rename
+    assertFalse(fs.rename(foo, bar));
+    assertFalse(container.exists());
+
+    // But a write should.
+    assertTrue(fs.createNewFile(foo));
+    assertTrue(container.exists());
+  }
+
+  @Test
+  public void testContainerChecksWithSas() throws Exception {
+
+    Assume.assumeFalse(runningInSASMode);
+    testAccount = AzureBlobStorageTestAccount.create("",
+        EnumSet.of(CreateOptions.UseSas));
+    assumeNotNull(testAccount);
+    CloudBlobContainer container = testAccount.getRealContainer();
+    FileSystem fs = testAccount.getFileSystem();
+
+    // The container shouldn't be there
+    assertFalse(container.exists());
+
+    // A write should just fail
+    try {
+      fs.createNewFile(new Path("/testContainerChecksWithSas-foo"));
+      assertFalse("Should've thrown.", true);
+    } catch (AzureException ex) {
+    }
+    assertFalse(container.exists());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationExceptionHandling.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationExceptionHandling.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationExceptionHandling.java
new file mode 100644
index 0000000..a45dae4
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationExceptionHandling.java
@@ -0,0 +1,283 @@
+/**
+ * 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.azure;
+
+import java.io.FileNotFoundException;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.junit.After;
+import org.junit.Test;
+
+import static org.apache.hadoop.fs.azure.ExceptionHandlingTestHelper.*;
+
+/**
+ * Single threaded exception handling.
+ */
+public class ITestFileSystemOperationExceptionHandling
+    extends AbstractWasbTestBase {
+
+  private FSDataInputStream inputStream = null;
+
+  private Path testPath;
+  private Path testFolderPath;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    testPath = path("testfile.dat");
+    testFolderPath = path("testfolder");
+  }
+
+  /**
+   * Helper method that creates a InputStream to validate exceptions
+   * for various scenarios.
+   */
+  private void setupInputStreamToTest(AzureBlobStorageTestAccount testAccount)
+      throws Exception {
+
+    FileSystem fs = testAccount.getFileSystem();
+
+    // Step 1: Create a file and write dummy data.
+    Path base = methodPath();
+    Path testFilePath1 = new Path(base, "test1.dat");
+    Path testFilePath2 = new Path(base, "test2.dat");
+    FSDataOutputStream outputStream = fs.create(testFilePath1);
+    String testString = "This is a test string";
+    outputStream.write(testString.getBytes());
+    outputStream.close();
+
+    // Step 2: Open a read stream on the file.
+    inputStream = fs.open(testFilePath1);
+
+    // Step 3: Rename the file
+    fs.rename(testFilePath1, testFilePath2);
+  }
+
+  /**
+   * Tests a basic single threaded read scenario for Page blobs.
+   */
+  @Test(expected=FileNotFoundException.class)
+  public void testSingleThreadedPageBlobReadScenario() throws Throwable {
+    AzureBlobStorageTestAccount testAccount = getPageBlobTestStorageAccount();
+    setupInputStreamToTest(testAccount);
+    byte[] readBuffer = new byte[512];
+    inputStream.read(readBuffer);
+  }
+
+  /**
+   * Tests a basic single threaded seek scenario for Page blobs.
+   */
+  @Test(expected=FileNotFoundException.class)
+  public void testSingleThreadedPageBlobSeekScenario() throws Throwable {
+    AzureBlobStorageTestAccount testAccount = getPageBlobTestStorageAccount();
+    setupInputStreamToTest(testAccount);
+    inputStream.seek(5);
+  }
+
+  /**
+   * Test a basic single thread seek scenario for Block blobs.
+   */
+  @Test(expected=FileNotFoundException.class)
+  public void testSingleThreadBlockBlobSeekScenario() throws Throwable {
+
+    AzureBlobStorageTestAccount testAccount = createTestAccount();
+    setupInputStreamToTest(testAccount);
+    inputStream.seek(5);
+    inputStream.read();
+  }
+
+  /**
+   * Tests a basic single threaded read scenario for Block blobs.
+   */
+  @Test(expected=FileNotFoundException.class)
+  public void testSingledThreadBlockBlobReadScenario() throws Throwable{
+    AzureBlobStorageTestAccount testAccount = createTestAccount();
+    setupInputStreamToTest(testAccount);
+    byte[] readBuffer = new byte[512];
+    inputStream.read(readBuffer);
+  }
+
+  /**
+   * Tests basic single threaded setPermission scenario.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testSingleThreadedBlockBlobSetPermissionScenario() throws Throwable {
+
+    createEmptyFile(createTestAccount(), testPath);
+    fs.delete(testPath, true);
+    fs.setPermission(testPath,
+        new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
+  }
+
+  /**
+   * Tests basic single threaded setPermission scenario.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testSingleThreadedPageBlobSetPermissionScenario()
+      throws Throwable {
+    createEmptyFile(getPageBlobTestStorageAccount(), testPath);
+    fs.delete(testPath, true);
+    fs.setOwner(testPath, "testowner", "testgroup");
+  }
+
+  /**
+   * Tests basic single threaded setPermission scenario.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testSingleThreadedBlockBlobSetOwnerScenario() throws Throwable {
+
+    createEmptyFile(createTestAccount(), testPath);
+    fs.delete(testPath, true);
+    fs.setOwner(testPath, "testowner", "testgroup");
+  }
+
+  /**
+   * Tests basic single threaded setPermission scenario.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testSingleThreadedPageBlobSetOwnerScenario() throws Throwable {
+    createEmptyFile(getPageBlobTestStorageAccount(),
+        testPath);
+    fs.delete(testPath, true);
+    fs.setPermission(testPath,
+        new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
+  }
+
+  /**
+   * Test basic single threaded listStatus scenario.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testSingleThreadedBlockBlobListStatusScenario() throws Throwable {
+    createTestFolder(createTestAccount(),
+        testFolderPath);
+    fs.delete(testFolderPath, true);
+    fs.listStatus(testFolderPath);
+  }
+
+  /**
+   * Test basic single threaded listStatus scenario.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testSingleThreadedPageBlobListStatusScenario() throws Throwable {
+    createTestFolder(getPageBlobTestStorageAccount(),
+        testFolderPath);
+    fs.delete(testFolderPath, true);
+    fs.listStatus(testFolderPath);
+  }
+
+  /**
+   * Test basic single threaded listStatus scenario.
+   */
+  @Test
+  public void testSingleThreadedBlockBlobRenameScenario() throws Throwable {
+
+    createEmptyFile(createTestAccount(),
+        testPath);
+    Path dstPath = new Path("dstFile.dat");
+    fs.delete(testPath, true);
+    boolean renameResult = fs.rename(testPath, dstPath);
+    assertFalse(renameResult);
+  }
+
+  /**
+   * Test basic single threaded listStatus scenario.
+   */
+  @Test
+  public void testSingleThreadedPageBlobRenameScenario() throws Throwable {
+
+    createEmptyFile(getPageBlobTestStorageAccount(),
+        testPath);
+    Path dstPath = new Path("dstFile.dat");
+    fs.delete(testPath, true);
+    boolean renameResult = fs.rename(testPath, dstPath);
+    assertFalse(renameResult);
+  }
+
+  /**
+   * Test basic single threaded listStatus scenario.
+   */
+  @Test
+  public void testSingleThreadedBlockBlobDeleteScenario() throws Throwable {
+
+    createEmptyFile(createTestAccount(),
+        testPath);
+    fs.delete(testPath, true);
+    boolean deleteResult = fs.delete(testPath, true);
+    assertFalse(deleteResult);
+  }
+
+  /**
+   * Test basic single threaded listStatus scenario.
+   */
+  @Test
+  public void testSingleThreadedPageBlobDeleteScenario() throws Throwable {
+
+    createEmptyFile(getPageBlobTestStorageAccount(),
+        testPath);
+    fs.delete(testPath, true);
+    boolean deleteResult = fs.delete(testPath, true);
+    assertFalse(deleteResult);
+  }
+
+  /**
+   * Test basic single threaded listStatus scenario.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testSingleThreadedBlockBlobOpenScenario() throws Throwable {
+
+    createEmptyFile(createTestAccount(),
+        testPath);
+    fs.delete(testPath, true);
+    inputStream = fs.open(testPath);
+  }
+
+  /**
+   * Test delete then open a file.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testSingleThreadedPageBlobOpenScenario() throws Throwable {
+
+    createEmptyFile(getPageBlobTestStorageAccount(),
+        testPath);
+    fs.delete(testPath, true);
+    inputStream = fs.open(testPath);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (inputStream != null) {
+      inputStream.close();
+    }
+
+    ContractTestUtils.rm(fs, testPath, true, true);
+    super.tearDown();
+  }
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount()
+      throws Exception {
+    return AzureBlobStorageTestAccount.create();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationExceptionMessage.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationExceptionMessage.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationExceptionMessage.java
new file mode 100644
index 0000000..6d5e72e
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationExceptionMessage.java
@@ -0,0 +1,79 @@
+/*
+ * 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.azure;
+
+import java.net.URI;
+import java.util.UUID;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azure.integration.AzureTestUtils;
+import org.apache.hadoop.test.GenericTestUtils;
+
+import com.microsoft.azure.storage.CloudStorageAccount;
+import org.junit.Test;
+
+import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.NO_ACCESS_TO_CONTAINER_MSG;
+
+/**
+ * Test for error messages coming from SDK.
+ */
+public class ITestFileSystemOperationExceptionMessage
+    extends AbstractWasbTestWithTimeout {
+
+
+
+  @Test
+  public void testAnonymouseCredentialExceptionMessage() throws Throwable {
+
+    Configuration conf = AzureBlobStorageTestAccount.createTestConfiguration();
+    CloudStorageAccount account =
+        AzureBlobStorageTestAccount.createTestAccount(conf);
+    AzureTestUtils.assume("No test account", account != null);
+
+    String testStorageAccount = conf.get("fs.azure.test.account.name");
+    conf = new Configuration();
+    conf.set("fs.AbstractFileSystem.wasb.impl",
+        "org.apache.hadoop.fs.azure.Wasb");
+    conf.set("fs.azure.skip.metrics", "true");
+
+    String testContainer = UUID.randomUUID().toString();
+    String wasbUri = String.format("wasb://%s@%s",
+        testContainer, testStorageAccount);
+
+    try(NativeAzureFileSystem filesystem = new NativeAzureFileSystem()) {
+      filesystem.initialize(new URI(wasbUri), conf);
+      fail("Expected an exception, got " + filesystem);
+    } catch (Exception ex) {
+
+      Throwable innerException = ex.getCause();
+      while (innerException != null
+          && !(innerException instanceof AzureException)) {
+        innerException = innerException.getCause();
+      }
+
+      if (innerException != null) {
+        GenericTestUtils.assertExceptionContains(String.format(
+            NO_ACCESS_TO_CONTAINER_MSG, testStorageAccount, testContainer),
+            ex);
+      } else {
+        fail("No inner azure exception");
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationsExceptionHandlingMultiThreaded.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationsExceptionHandlingMultiThreaded.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationsExceptionHandlingMultiThreaded.java
new file mode 100644
index 0000000..175a9ec
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestFileSystemOperationsExceptionHandlingMultiThreaded.java
@@ -0,0 +1,366 @@
+/**
+ * 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.azure;
+
+import java.io.FileNotFoundException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.IOUtils;
+
+import static org.apache.hadoop.fs.azure.ExceptionHandlingTestHelper.*;
+
+/**
+ * Multithreaded operations on FS, verify failures are as expected.
+ */
+public class ITestFileSystemOperationsExceptionHandlingMultiThreaded
+    extends AbstractWasbTestBase {
+
+  FSDataInputStream inputStream = null;
+
+  private Path testPath;
+  private Path testFolderPath;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    testPath = path("testfile.dat");
+    testFolderPath = path("testfolder");
+  }
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create();
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+
+    IOUtils.closeStream(inputStream);
+    ContractTestUtils.rm(fs, testPath, true, false);
+    ContractTestUtils.rm(fs, testFolderPath, true, false);
+    super.tearDown();
+  }
+
+  /**
+   * Helper method to creates an input stream to test various scenarios.
+   */
+  private void getInputStreamToTest(FileSystem fs, Path testPath)
+      throws Throwable {
+
+    FSDataOutputStream outputStream = fs.create(testPath);
+    String testString = "This is a test string";
+    outputStream.write(testString.getBytes());
+    outputStream.close();
+
+    inputStream = fs.open(testPath);
+  }
+
+  /**
+   * Test to validate correct exception is thrown for Multithreaded read
+   * scenario for block blobs.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testMultiThreadedBlockBlobReadScenario() throws Throwable {
+
+    AzureBlobStorageTestAccount testAccount = createTestAccount();
+    NativeAzureFileSystem fs = testAccount.getFileSystem();
+    Path base = methodPath();
+    Path testFilePath1 = new Path(base, "test1.dat");
+    Path renamePath = new Path(base, "test2.dat");
+    getInputStreamToTest(fs, testFilePath1);
+    Thread renameThread = new Thread(
+        new RenameThread(fs, testFilePath1, renamePath));
+    renameThread.start();
+
+    renameThread.join();
+
+    byte[] readBuffer = new byte[512];
+    inputStream.read(readBuffer);
+  }
+
+  /**
+   * Test to validate correct exception is thrown for Multithreaded seek
+   * scenario for block blobs.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testMultiThreadBlockBlobSeekScenario() throws Throwable {
+
+/*
+    AzureBlobStorageTestAccount testAccount = createTestAccount();
+    fs = testAccount.getFileSystem();
+*/
+    Path base = methodPath();
+    Path testFilePath1 = new Path(base, "test1.dat");
+    Path renamePath = new Path(base, "test2.dat");
+
+    getInputStreamToTest(fs, testFilePath1);
+    Thread renameThread = new Thread(
+        new RenameThread(fs, testFilePath1, renamePath));
+    renameThread.start();
+
+    renameThread.join();
+
+    inputStream.seek(5);
+    inputStream.read();
+  }
+
+  /**
+   * Tests basic multi threaded setPermission scenario.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testMultiThreadedPageBlobSetPermissionScenario()
+      throws Throwable {
+    createEmptyFile(
+        getPageBlobTestStorageAccount(),
+        testPath);
+    Thread t = new Thread(new DeleteThread(fs, testPath));
+    t.start();
+    while (t.isAlive()) {
+      fs.setPermission(testPath,
+          new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
+    }
+    fs.setPermission(testPath,
+        new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
+  }
+
+  /**
+   * Tests basic multi threaded setPermission scenario.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testMultiThreadedBlockBlobSetPermissionScenario()
+      throws Throwable {
+    createEmptyFile(createTestAccount(),
+        testPath);
+    Thread t = new Thread(new DeleteThread(fs, testPath));
+    t.start();
+    while (t.isAlive()) {
+      fs.setPermission(testPath,
+          new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
+    }
+    fs.setPermission(testPath,
+        new FsPermission(FsAction.EXECUTE, FsAction.READ, FsAction.READ));
+  }
+
+  /**
+   * Tests basic multi threaded setPermission scenario.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testMultiThreadedPageBlobOpenScenario() throws Throwable {
+
+    createEmptyFile(createTestAccount(),
+        testPath);
+    Thread t = new Thread(new DeleteThread(fs, testPath));
+    t.start();
+    while (t.isAlive()) {
+      inputStream = fs.open(testPath);
+      inputStream.close();
+    }
+
+    inputStream = fs.open(testPath);
+    inputStream.close();
+  }
+
+  /**
+   * Tests basic multi threaded setPermission scenario.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testMultiThreadedBlockBlobOpenScenario() throws Throwable {
+
+    createEmptyFile(
+        getPageBlobTestStorageAccount(),
+        testPath);
+    Thread t = new Thread(new DeleteThread(fs, testPath));
+    t.start();
+
+    while (t.isAlive()) {
+      inputStream = fs.open(testPath);
+      inputStream.close();
+    }
+    inputStream = fs.open(testPath);
+    inputStream.close();
+  }
+
+  /**
+   * Tests basic multi threaded setOwner scenario.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testMultiThreadedBlockBlobSetOwnerScenario() throws Throwable {
+
+    createEmptyFile(createTestAccount(), testPath);
+    Thread t = new Thread(new DeleteThread(fs, testPath));
+    t.start();
+    while (t.isAlive()) {
+      fs.setOwner(testPath, "testowner", "testgroup");
+    }
+    fs.setOwner(testPath, "testowner", "testgroup");
+  }
+
+  /**
+   * Tests basic multi threaded setOwner scenario.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testMultiThreadedPageBlobSetOwnerScenario() throws Throwable {
+    createEmptyFile(
+        getPageBlobTestStorageAccount(),
+        testPath);
+    Thread t = new Thread(new DeleteThread(fs, testPath));
+    t.start();
+    while (t.isAlive()) {
+      fs.setOwner(testPath, "testowner", "testgroup");
+    }
+    fs.setOwner(testPath, "testowner", "testgroup");
+  }
+
+  /**
+   * Tests basic multi threaded listStatus scenario.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testMultiThreadedBlockBlobListStatusScenario() throws Throwable {
+
+    createTestFolder(createTestAccount(),
+        testFolderPath);
+    Thread t = new Thread(new DeleteThread(fs, testFolderPath));
+    t.start();
+    while (t.isAlive()) {
+      fs.listStatus(testFolderPath);
+    }
+    fs.listStatus(testFolderPath);
+  }
+
+  /**
+   * Tests basic multi threaded listStatus scenario.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testMultiThreadedPageBlobListStatusScenario() throws Throwable {
+
+    createTestFolder(
+        getPageBlobTestStorageAccount(),
+        testFolderPath);
+    Thread t = new Thread(new DeleteThread(fs, testFolderPath));
+    t.start();
+    while (t.isAlive()) {
+      fs.listStatus(testFolderPath);
+    }
+    fs.listStatus(testFolderPath);
+  }
+
+  /**
+   * Test to validate correct exception is thrown for Multithreaded read
+   * scenario for page blobs.
+   */
+  @Test(expected = FileNotFoundException.class)
+  public void testMultiThreadedPageBlobReadScenario() throws Throwable {
+
+    bindToTestAccount(getPageBlobTestStorageAccount());
+    Path base = methodPath();
+    Path testFilePath1 = new Path(base, "test1.dat");
+    Path renamePath = new Path(base, "test2.dat");
+
+    getInputStreamToTest(fs, testFilePath1);
+    Thread renameThread = new Thread(
+        new RenameThread(fs, testFilePath1, renamePath));
+    renameThread.start();
+
+    renameThread.join();
+    byte[] readBuffer = new byte[512];
+    inputStream.read(readBuffer);
+  }
+
+  /**
+   * Test to validate correct exception is thrown for Multithreaded seek
+   * scenario for page blobs.
+   */
+
+  @Test(expected = FileNotFoundException.class)
+  public void testMultiThreadedPageBlobSeekScenario() throws Throwable {
+
+    bindToTestAccount(getPageBlobTestStorageAccount());
+
+    Path base = methodPath();
+    Path testFilePath1 = new Path(base, "test1.dat");
+    Path renamePath = new Path(base, "test2.dat");
+
+    getInputStreamToTest(fs, testFilePath1);
+    Thread renameThread = new Thread(
+        new RenameThread(fs, testFilePath1, renamePath));
+    renameThread.start();
+
+    renameThread.join();
+    inputStream.seek(5);
+  }
+
+
+  /**
+   * Helper thread that just renames the test file.
+   */
+  private static class RenameThread implements Runnable {
+
+    private final FileSystem fs;
+    private final Path testPath;
+    private final Path renamePath;
+
+    RenameThread(FileSystem fs,
+        Path testPath,
+        Path renamePath) {
+      this.fs = fs;
+      this.testPath = testPath;
+      this.renamePath = renamePath;
+    }
+
+    @Override
+    public void run() {
+      try {
+        fs.rename(testPath, renamePath);
+      } catch (Exception e) {
+        // Swallowing the exception as the
+        // correctness of the test is controlled
+        // by the other thread
+      }
+    }
+  }
+
+  private static class DeleteThread implements Runnable {
+    private final FileSystem fs;
+    private final Path testPath;
+
+    DeleteThread(FileSystem fs, Path testPath) {
+      this.fs = fs;
+      this.testPath = testPath;
+    }
+
+    @Override
+    public void run() {
+      try {
+        fs.delete(testPath, true);
+      } catch (Exception e) {
+        // Swallowing the exception as the
+        // correctness of the test is controlled
+        // by the other thread
+      }
+    }
+  }
+}


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


[18/50] [abbrv] hadoop git commit: HDFS-12424. Datatable sorting on the Datanode Information page in the Namenode UI is broken. Contributed by Shawna Martell.

Posted by ae...@apache.org.
HDFS-12424. Datatable sorting on the Datanode Information page in the Namenode UI is broken. Contributed by Shawna Martell.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/78bdf10a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/78bdf10a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/78bdf10a

Branch: refs/heads/HDFS-7240
Commit: 78bdf10ae40300a5f2f3b95491d280802edc1df2
Parents: 08b9814
Author: Kihwal Lee <ki...@apache.org>
Authored: Fri Sep 15 09:27:51 2017 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Fri Sep 15 09:28:31 2017 -0500

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js      | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/78bdf10a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
index b785274..de62622 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
@@ -331,12 +331,12 @@
             'columns': [
               { 'orderDataType': 'ng-value', 'searchable': true },
               { 'orderDataType': 'ng-value', 'searchable': true },
-              { 'orderDataType': 'ng-value', 'type': 'numeric' },
-              { 'orderDataType': 'ng-value', 'type': 'numeric' },
-              { 'orderDataType': 'ng-value', 'type': 'numeric' },
-              { 'orderData': 3, 'type': 'numeric' },
-              { 'orderDataType': 'ng-value', 'type': 'numeric'},
-              { 'orderData': 5 }
+              { 'orderDataType': 'ng-value', 'type': 'num' },
+              { 'orderDataType': 'ng-value', 'type': 'num' },
+              { 'orderDataType': 'ng-value', 'type': 'num' },
+              { 'type': 'num' },
+              { 'orderDataType': 'ng-value', 'type': 'num'},
+              { 'type': 'string' }
             ]});
           renderHistogram(data);
           $('#ui-tabs a[href="#tab-datanode"]').tab('show');


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


[29/50] [abbrv] hadoop git commit: HADOOP-14553. Add (parallelized) integration tests to hadoop-azure Contributed by Steve Loughran

Posted by ae...@apache.org.
HADOOP-14553. Add (parallelized) integration tests to hadoop-azure
Contributed by Steve Loughran


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2d2d97fa
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2d2d97fa
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2d2d97fa

Branch: refs/heads/HDFS-7240
Commit: 2d2d97fa7d4224369b3c13bc4a45e8cc9e29afb1
Parents: 11390c2
Author: Steve Loughran <st...@apache.org>
Authored: Fri Sep 15 17:03:01 2017 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Sep 15 17:03:01 2017 +0100

----------------------------------------------------------------------
 .../hadoop/fs/FileSystemContractBaseTest.java   |  11 +-
 .../fs/contract/AbstractContractOpenTest.java   |   4 +-
 .../fs/contract/AbstractContractSeekTest.java   |   2 +-
 hadoop-tools/hadoop-azure/pom.xml               | 251 ++++++
 .../fs/azure/AzureNativeFileSystemStore.java    |   2 +-
 .../hadoop-azure/src/site/markdown/index.md     |  94 +-
 .../src/site/markdown/testing_azure.md          | 576 ++++++++++++
 .../hadoop/fs/azure/AbstractWasbTestBase.java   | 136 ++-
 .../fs/azure/AbstractWasbTestWithTimeout.java   |  73 ++
 .../fs/azure/AzureBlobStorageTestAccount.java   |  42 +-
 .../azure/ITestAzureConcurrentOutOfBandIo.java  | 179 ++++
 ...zureConcurrentOutOfBandIoWithSecureMode.java |  33 +
 .../ITestAzureFileSystemErrorConditions.java    | 243 +++++
 .../fs/azure/ITestBlobDataValidation.java       | 244 ++++++
 .../fs/azure/ITestBlobTypeSpeedDifference.java  | 163 ++++
 .../fs/azure/ITestBlockBlobInputStream.java     | 874 ++++++++++++++++++
 .../hadoop/fs/azure/ITestContainerChecks.java   | 194 ++++
 ...estFileSystemOperationExceptionHandling.java | 283 ++++++
 ...TestFileSystemOperationExceptionMessage.java |  79 ++
 ...perationsExceptionHandlingMultiThreaded.java | 366 ++++++++
 .../ITestFileSystemOperationsWithThreads.java   | 821 +++++++++++++++++
 ...stNativeAzureFSAuthWithBlobSpecificKeys.java |  40 +
 .../ITestNativeAzureFSAuthorizationCaching.java |  53 ++
 .../azure/ITestNativeAzureFSPageBlobLive.java   |  43 +
 .../azure/ITestNativeAzureFileSystemAppend.java | 350 ++++++++
 ...ativeAzureFileSystemAtomicRenameDirList.java |  55 ++
 ...veAzureFileSystemAuthorizationWithOwner.java | 122 +++
 ...ITestNativeAzureFileSystemClientLogging.java | 136 +++
 ...estNativeAzureFileSystemConcurrencyLive.java | 185 ++++
 ...stNativeAzureFileSystemContractEmulator.java |  65 ++
 .../ITestNativeAzureFileSystemContractLive.java | 108 +++
 ...tiveAzureFileSystemContractPageBlobLive.java | 114 +++
 .../azure/ITestNativeAzureFileSystemLive.java   | 236 +++++
 .../ITestOutOfBandAzureBlobOperationsLive.java  | 185 ++++
 .../ITestReadAndSeekPageBlobAfterWrite.java     | 341 ++++++++
 .../fs/azure/ITestWasbRemoteCallHelper.java     | 568 ++++++++++++
 .../fs/azure/ITestWasbUriAndConfiguration.java  | 610 +++++++++++++
 .../hadoop/fs/azure/MockWasbAuthorizerImpl.java | 103 ++-
 .../fs/azure/NativeAzureFileSystemBaseTest.java | 115 ++-
 .../hadoop/fs/azure/RunningLiveWasbTests.txt    |  22 -
 .../azure/TestAzureConcurrentOutOfBandIo.java   | 195 -----
 ...zureConcurrentOutOfBandIoWithSecureMode.java |  50 --
 .../TestAzureFileSystemErrorConditions.java     | 244 ------
 .../hadoop/fs/azure/TestBlobDataValidation.java | 237 -----
 .../hadoop/fs/azure/TestBlobMetadata.java       |   7 +-
 .../fs/azure/TestBlobOperationDescriptor.java   |   3 -
 .../fs/azure/TestBlobTypeSpeedDifference.java   | 160 ----
 .../fs/azure/TestBlockBlobInputStream.java      | 875 -------------------
 .../fs/azure/TestClientThrottlingAnalyzer.java  |   5 +-
 .../hadoop/fs/azure/TestContainerChecks.java    | 185 ----
 ...estFileSystemOperationExceptionHandling.java | 269 ------
 ...TestFileSystemOperationExceptionMessage.java |  79 --
 ...perationsExceptionHandlingMultiThreaded.java | 330 -------
 .../TestFileSystemOperationsWithThreads.java    | 821 -----------------
 ...stNativeAzureFSAuthWithBlobSpecificKeys.java |  44 -
 .../TestNativeAzureFSAuthorizationCaching.java  |  60 --
 .../fs/azure/TestNativeAzureFSPageBlobLive.java |  43 -
 .../azure/TestNativeAzureFileSystemAppend.java  | 362 --------
 ...ativeAzureFileSystemAtomicRenameDirList.java |  50 --
 .../TestNativeAzureFileSystemAuthorization.java |  53 +-
 ...veAzureFileSystemAuthorizationWithOwner.java | 122 ---
 ...TestNativeAzureFileSystemBlockLocations.java |   8 +-
 .../TestNativeAzureFileSystemClientLogging.java | 140 ---
 .../TestNativeAzureFileSystemConcurrency.java   |  29 +-
 ...estNativeAzureFileSystemConcurrencyLive.java | 184 ----
 ...stNativeAzureFileSystemContractEmulator.java |  48 -
 .../TestNativeAzureFileSystemContractLive.java  |  80 --
 ...TestNativeAzureFileSystemContractMocked.java |   3 +
 ...tiveAzureFileSystemContractPageBlobLive.java |  93 --
 .../TestNativeAzureFileSystemFileNameCheck.java |  28 +-
 .../fs/azure/TestNativeAzureFileSystemLive.java | 242 -----
 .../azure/TestNativeAzureFileSystemMocked.java  |   4 +
 .../TestNativeAzureFileSystemUploadLogic.java   |  78 +-
 .../azure/TestOutOfBandAzureBlobOperations.java |   8 +-
 .../TestOutOfBandAzureBlobOperationsLive.java   | 203 -----
 .../TestReadAndSeekPageBlobAfterWrite.java      | 355 --------
 .../azure/TestShellDecryptionKeyProvider.java   |  15 +-
 .../apache/hadoop/fs/azure/TestWasbFsck.java    |   9 +-
 .../fs/azure/TestWasbRemoteCallHelper.java      | 569 ------------
 .../fs/azure/TestWasbUriAndConfiguration.java   | 617 -------------
 .../ITestAzureNativeContractAppend.java         |  41 +
 .../ITestAzureNativeContractCreate.java         |  34 +
 .../ITestAzureNativeContractDelete.java         |  33 +
 .../ITestAzureNativeContractDistCp.java         |  47 +
 .../ITestAzureNativeContractGetFileStatus.java  |  35 +
 .../contract/ITestAzureNativeContractMkdir.java |  33 +
 .../contract/ITestAzureNativeContractOpen.java  |  34 +
 .../ITestAzureNativeContractRename.java         |  34 +
 .../contract/ITestAzureNativeContractSeek.java  |  34 +
 .../contract/NativeAzureFileSystemContract.java |  19 +-
 .../contract/TestAzureNativeContractAppend.java |  37 -
 .../contract/TestAzureNativeContractCreate.java |  30 -
 .../contract/TestAzureNativeContractDelete.java |  30 -
 .../contract/TestAzureNativeContractDistCp.java |  33 -
 .../TestAzureNativeContractGetFileStatus.java   |  30 -
 .../contract/TestAzureNativeContractMkdir.java  |  30 -
 .../contract/TestAzureNativeContractOpen.java   |  30 -
 .../contract/TestAzureNativeContractRename.java |  30 -
 .../contract/TestAzureNativeContractSeek.java   |  30 -
 .../integration/AbstractAzureScaleTest.java     |  66 ++
 .../azure/integration/AzureTestConstants.java   | 180 ++++
 .../fs/azure/integration/AzureTestUtils.java    | 479 ++++++++++
 .../integration/CleanupTestContainers.java      |  87 ++
 .../azure/integration/ITestAzureHugeFiles.java  | 456 ++++++++++
 .../hadoop/fs/azure/integration/Sizes.java      |  43 +
 .../ITestAzureFileSystemInstrumentation.java    | 586 +++++++++++++
 .../TestAzureFileSystemInstrumentation.java     | 579 ------------
 107 files changed, 10227 insertions(+), 7901 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
index b49dd53..a4ccee3 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java
@@ -61,7 +61,16 @@ public abstract class FileSystemContractBaseTest {
   protected byte[] data = dataset(getBlockSize() * 2, 0, 255);
 
   @Rule
-  public Timeout globalTimeout = new Timeout(30000);
+  public Timeout globalTimeout = new Timeout(getGlobalTimeout());
+
+  /**
+   * Get the timeout in milliseconds for each test case.
+   * @return a time in milliseconds.
+   */
+  protected int getGlobalTimeout() {
+    return 30 * 1000;
+  }
+
   @Rule
   public ExpectedException thrown = ExpectedException.none();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
index f9b16f4..ccf188f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
@@ -122,7 +122,7 @@ public abstract class AbstractContractOpenTest extends AbstractFSContractTestBas
     Path path = path("testopenfiletwice.txt");
     byte[] block = dataset(TEST_FILE_LEN, 0, 255);
     //this file now has a simple rule: offset => value
-    createFile(getFileSystem(), path, false, block);
+    createFile(getFileSystem(), path, true, block);
     //open first
     FSDataInputStream instream1 = getFileSystem().open(path);
     FSDataInputStream instream2 = null;
@@ -150,7 +150,7 @@ public abstract class AbstractContractOpenTest extends AbstractFSContractTestBas
     int base = 0x40; // 64
     byte[] block = dataset(len, base, base + len);
     //this file now has a simple rule: offset => (value | 0x40)
-    createFile(getFileSystem(), path, false, block);
+    createFile(getFileSystem(), path, true, block);
     //open first
     instream = getFileSystem().open(path);
     assertEquals(base, instream.read());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractSeekTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractSeekTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractSeekTest.java
index 3e71682..7af3cb0 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractSeekTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractSeekTest.java
@@ -341,7 +341,7 @@ public abstract class AbstractContractSeekTest extends AbstractFSContractTestBas
     int filesize = 10 * 1024;
     byte[] buf = dataset(filesize, 0, 255);
     Path randomSeekFile = path("testrandomseeks.bin");
-    createFile(getFileSystem(), randomSeekFile, false, buf);
+    createFile(getFileSystem(), randomSeekFile, true, buf);
     Random r = new Random();
 
     // Record the sequence of seeks and reads which trigger a failure.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml
index 0c5ac63..b479872 100644
--- a/hadoop-tools/hadoop-azure/pom.xml
+++ b/hadoop-tools/hadoop-azure/pom.xml
@@ -34,6 +34,15 @@
   <properties>
     <file.encoding>UTF-8</file.encoding>
     <downloadSources>true</downloadSources>
+    <hadoop.tmp.dir>${project.build.directory}/test</hadoop.tmp.dir>
+    <!-- are scale tests enabled ? -->
+    <fs.azure.scale.test.enabled>unset</fs.azure.scale.test.enabled>
+    <!-- Size in MB of huge files. -->
+    <fs.azure.scale.test.huge.filesize>unset</fs.azure.scale.test.huge.filesize>
+    <!-- Size in MB of the partion size in huge file uploads. -->
+    <fs.azure.scale.test.huge.partitionsize>unset</fs.azure.scale.test.huge.partitionsize>
+    <!-- Timeout in seconds for scale tests.-->
+    <fs.azure.scale.test.timeout>7200</fs.azure.scale.test.timeout>
   </properties>
 
   <build>
@@ -224,4 +233,246 @@
     </dependency>
 
   </dependencies>
+
+  <profiles>
+    <profile>
+      <id>parallel-tests</id>
+      <activation>
+        <property>
+          <name>parallel-tests</name>
+        </property>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>create-parallel-tests-dirs</id>
+                <phase>test-compile</phase>
+                <configuration>
+                  <target>
+                    <script language="javascript"><![CDATA[
+                      var baseDirs = [
+                        project.getProperty("test.build.data"),
+                        project.getProperty("test.build.dir"),
+                        project.getProperty("hadoop.tmp.dir")
+                      ];
+                      for (var i in baseDirs) {
+                        for (var j = 1; j <= ${testsThreadCount}; ++j) {
+                          var mkdir = project.createTask("mkdir");
+                          mkdir.setDir(new java.io.File(baseDirs[i], j));
+                          mkdir.perform();
+                        }
+                      }
+                    ]]></script>
+                  </target>
+                </configuration>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-surefire-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>default-test</id>
+                <goals>
+                  <goal>test</goal>
+                </goals>
+                <configuration>
+                  <forkCount>1</forkCount>
+                  <forkCount>${testsThreadCount}</forkCount>
+                  <reuseForks>false</reuseForks>
+                  <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
+                  <forkedProcessTimeoutInSeconds>${fs.azure.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <systemPropertyVariables>
+                    <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
+                    <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
+                    <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>
+                    <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+                    <fs.azure.scale.test.enabled>${fs.azure.scale.test.enabled}</fs.azure.scale.test.enabled>
+                    <fs.azure.scale.test.huge.filesize>${fs.azure.scale.test.huge.filesize}</fs.azure.scale.test.huge.filesize>
+                    <fs.azure.scale.test.huge.huge.partitionsize>${fs.azure.scale.test.huge.partitionsize}</fs.azure.scale.test.huge.huge.partitionsize>
+                    <fs.azure.scale.test.timeout>${fs.azure.scale.test.timeout}</fs.azure.scale.test.timeout>
+                  </systemPropertyVariables>
+                  <includes>
+                    <include>**/Test*.java</include>
+                  </includes>
+                  <excludes>
+                    <exclude>**/TestRollingWindowAverage*.java</exclude>
+                  </excludes>
+                </configuration>
+              </execution>
+              <execution>
+                <id>serialized-test</id>
+                <goals>
+                  <goal>test</goal>
+                </goals>
+                <configuration>
+                  <forkCount>1</forkCount>
+                  <reuseForks>false</reuseForks>
+                  <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
+                  <forkedProcessTimeoutInSeconds>${fs.azure.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <systemPropertyVariables>
+                    <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
+                    <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
+                    <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>
+                    <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+                    <fs.azure.scale.test.enabled>${fs.azure.scale.test.enabled}</fs.azure.scale.test.enabled>
+                    <fs.azure.scale.test.huge.filesize>${fs.azure.scale.test.huge.filesize}</fs.azure.scale.test.huge.filesize>
+                    <fs.azure.scale.test.huge.huge.partitionsize>${fs.azure.scale.test.huge.partitionsize}</fs.azure.scale.test.huge.huge.partitionsize>
+                    <fs.azure.scale.test.timeout>${fs.azure.scale.test.timeout}</fs.azure.scale.test.timeout>
+                  </systemPropertyVariables>
+                  <includes>
+                    <include>**/TestRollingWindowAverage*.java</include>
+                  </includes>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-failsafe-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>default-integration-test</id>
+                <goals>
+                  <goal>integration-test</goal>
+                  <goal>verify</goal>
+                </goals>
+                <configuration>
+                  <forkCount>${testsThreadCount}</forkCount>
+                  <reuseForks>false</reuseForks>
+                  <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
+                  <forkedProcessTimeoutInSeconds>${fs.azure.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <systemPropertyVariables>
+                    <!-- Tell tests that they are being executed in parallel -->
+                    <test.parallel.execution>true</test.parallel.execution>
+                    <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
+                    <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
+                    <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>
+
+                    <!-- Due to a Maven quirk, setting this to just -->
+                    <!-- surefire.forkNumber won't do the parameter -->
+                    <!-- substitution.  Putting a prefix in front of it like -->
+                    <!-- "fork-" makes it work. -->
+                    <test.unique.fork.id>fork-${surefire.forkNumber}</test.unique.fork.id>
+                    <!-- Propagate scale parameters -->
+                    <fs.azure.scale.test.enabled>${fs.azure.scale.test.enabled}</fs.azure.scale.test.enabled>
+                    <fs.azure.scale.test.huge.filesize>${fs.azure.scale.test.huge.filesize}</fs.azure.scale.test.huge.filesize>
+                    <fs.azure.scale.test.huge.huge.partitionsize>${fs.azure.scale.test.huge.partitionsize}</fs.azure.scale.test.huge.huge.partitionsize>
+                    <fs.azure.scale.test.timeout>${fs.azure.scale.test.timeout}</fs.azure.scale.test.timeout>
+                  </systemPropertyVariables>
+                  <!-- Some tests cannot run in parallel.  Tests that cover -->
+                  <!-- access to the root directory must run in isolation -->
+                  <!-- from anything else that could modify the bucket. -->
+                  <!-- azure tests that cover multi-part upload must run in -->
+                  <!-- isolation, because the file system is configured to -->
+                  <!-- purge existing multi-part upload data on -->
+                  <!-- initialization.  MiniYARNCluster has not yet been -->
+                  <!-- changed to handle parallel test execution gracefully. -->
+                  <!-- Exclude all of these tests from parallel execution, -->
+                  <!-- and instead run them sequentially in a separate -->
+                  <!-- Surefire execution step later. -->
+                  <includes>
+                    <include>**/ITest*.java</include>
+                  </includes>
+                  <excludes>
+                    <exclude>**/ITestFileSystemOperationsExceptionHandlingMultiThreaded.java</exclude>
+                    <exclude>**/ITestFileSystemOperationsWithThreads.java</exclude>
+                    <exclude>**/ITestOutOfBandAzureBlobOperationsLive.java</exclude>
+                    <exclude>**/ITestNativeAzureFileSystemAuthorizationWithOwner.java</exclude>
+                    <exclude>**/ITestNativeAzureFileSystemConcurrencyLive.java</exclude>
+                    <exclude>**/ITestNativeAzureFileSystemLive.java</exclude>
+                    <exclude>**/ITestNativeAzureFSPageBlobLive.java</exclude>
+                    <exclude>**/ITestWasbRemoteCallHelper.java</exclude>
+                    <exclude>**/ITestBlockBlobInputStream.java</exclude>
+                  </excludes>
+                </configuration>
+              </execution>
+              <!-- Do a sequential run for tests that cannot handle -->
+              <!-- parallel execution. -->
+              <execution>
+                <id>sequential-integration-tests</id>
+                <goals>
+                  <goal>integration-test</goal>
+                  <goal>verify</goal>
+                </goals>
+                <configuration>
+                  <forkedProcessTimeoutInSeconds>${fs.azure.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <systemPropertyVariables>
+                    <test.parallel.execution>false</test.parallel.execution>
+                    <fs.azure.scale.test.enabled>${fs.azure.scale.test.enabled}</fs.azure.scale.test.enabled>
+                    <fs.azure.scale.test.huge.filesize>${fs.azure.scale.test.huge.filesize}</fs.azure.scale.test.huge.filesize>
+                    <fs.azure.scale.test.huge.huge.partitionsize>${fs.azure.scale.test.huge.partitionsize}</fs.azure.scale.test.huge.huge.partitionsize>
+                    <fs.azure.scale.test.timeout>${fs.azure.scale.test.timeout}</fs.azure.scale.test.timeout>
+                  </systemPropertyVariables>
+                  <includes>
+                    <include>**/ITestFileSystemOperationsExceptionHandlingMultiThreaded.java</include>
+                    <include>**/ITestFileSystemOperationsWithThreads.java</include>
+                    <include>**/ITestOutOfBandAzureBlobOperationsLive.java</include>
+                    <include>**/ITestNativeAzureFileSystemAuthorizationWithOwner.java</include>
+                    <include>**/ITestNativeAzureFileSystemConcurrencyLive.java</include>
+                    <include>**/ITestNativeAzureFileSystemLive.java</include>
+                    <include>**/ITestWasbRemoteCallHelper.java</include>
+                    <include>**/ITestBlockBlobInputStream.java</include>
+                  </includes>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <profile>
+      <id>sequential-tests</id>
+      <activation>
+        <property>
+          <name>!parallel-tests</name>
+        </property>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-failsafe-plugin</artifactId>
+            <executions>
+              <execution>
+                <goals>
+                  <goal>integration-test</goal>
+                  <goal>verify</goal>
+                </goals>
+                <configuration>
+                  <systemPropertyVariables>
+                    <!-- Propagate scale parameters -->
+                    <fs.azure.scale.test.enabled>${fs.azure.scale.test.enabled}</fs.azure.scale.test.enabled>
+                    <fs.azure.scale.test.huge.filesize>${fs.azure.scale.test.huge.filesize}</fs.azure.scale.test.huge.filesize>
+                    <fs.azure.scale.test.timeout>${fs.azure.scale.test.timeout}</fs.azure.scale.test.timeout>
+                  </systemPropertyVariables>
+                  <forkedProcessTimeoutInSeconds>${fs.azure.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+
+    <!-- Turn on scale tests-->
+    <profile>
+      <id>scale</id>
+      <activation>
+        <property>
+          <name>scale</name>
+        </property>
+      </activation>
+      <properties>
+        <fs.azure.scale.test.enabled>true</fs.azure.scale.test.enabled>
+      </properties>
+    </profile>
+  </profiles>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
index 639862f..f1031b4 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
@@ -346,7 +346,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   private String delegationToken;
 
   /** The error message template when container is not accessible. */
-  static final String NO_ACCESS_TO_CONTAINER_MSG = "No credentials found for "
+  public static final String NO_ACCESS_TO_CONTAINER_MSG = "No credentials found for "
       + "account %s in the configuration, and its container %s is not "
       + "accessible using anonymous credentials. Please check if the container "
       + "exists first. If it is not publicly available, you have to provide "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/index.md b/hadoop-tools/hadoop-azure/src/site/markdown/index.md
index 466bf0b..876d7cc 100644
--- a/hadoop-tools/hadoop-azure/src/site/markdown/index.md
+++ b/hadoop-tools/hadoop-azure/src/site/markdown/index.md
@@ -519,96 +519,8 @@ The maximum number of entries that that cache can hold can be customized using t
       <value>true</value>
     </property>
 ```
-## Testing the hadoop-azure Module
 
-The hadoop-azure module includes a full suite of unit tests.  Most of the tests
-will run without additional configuration by running `mvn test`.  This includes
-tests against mocked storage, which is an in-memory emulation of Azure Storage.
-
-A selection of tests can run against the
-[Azure Storage Emulator](http://msdn.microsoft.com/en-us/library/azure/hh403989.aspx)
-which is a high-fidelity emulation of live Azure Storage.  The emulator is
-sufficient for high-confidence testing.  The emulator is a Windows executable
-that runs on a local machine.
-
-To use the emulator, install Azure SDK 2.3 and start the storage emulator.  Then,
-edit `src/test/resources/azure-test.xml` and add the following property:
-
-```xml
-<property>
-  <name>fs.azure.test.emulator</name>
-  <value>true</value>
-</property>
-```
-
-There is a known issue when running tests with the emulator.  You may see the
-following failure message:
-
-    com.microsoft.windowsazure.storage.StorageException: The value for one of the HTTP headers is not in the correct format.
-
-To resolve this, restart the Azure Emulator.  Ensure it v3.2 or later.
-
-It's also possible to run tests against a live Azure Storage account by saving a
-file to `src/test/resources/azure-auth-keys.xml` and setting
-the name of the storage account and its access key.
-
-For example:
-
-```xml
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<configuration>
-  <property>
-    <name>fs.azure.test.account.name</name>
-    <value>{ACCOUNTNAME}.blob.core.windows.net</value>
-  </property>
-  <property>
-    <name>fs.azure.account.key.{ACCOUNTNAME}.blob.core.windows.net</name>
-    <value>{ACCOUNT ACCESS KEY}</value>
-  </property>
-</configuration>
-```
-
-To run contract tests, set the WASB file system URI in `src/test/resources/azure-auth-keys.xml`
-and the account access key. For example:
-
-```xml
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<configuration>
-  <property>
-    <name>fs.contract.test.fs.wasb</name>
-    <value>wasb://{CONTAINERNAME}@{ACCOUNTNAME}.blob.core.windows.net</value>
-    <description>The name of the azure file system for testing.</description>
-  </property>
-  <property>
-    <name>fs.azure.account.key.{ACCOUNTNAME}.blob.core.windows.net</name>
-    <value>{ACCOUNT ACCESS KEY}</value>
-  </property>
-</configuration>
-```
-
-Overall, to run all the tests using `mvn test`,  a sample `azure-auth-keys.xml` is like following:
-
-```xml
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<configuration>
-  <property>
-    <name>fs.azure.test.account.name</name>
-    <value>{ACCOUNTNAME}.blob.core.windows.net</value>
-  </property>
-  <property>
-    <name>fs.azure.account.key.{ACCOUNTNAME}.blob.core.windows.net</name>
-    <value>{ACCOUNT ACCESS KEY}</value>
-  </property>
-  <property>
-    <name>fs.contract.test.fs.wasb</name>
-    <value>wasb://{CONTAINERNAME}@{ACCOUNTNAME}.blob.core.windows.net</value>
-  </property>
-</configuration>
-```
-
-DO NOT ADD `azure-auth-keys.xml` TO REVISION CONTROL.  The keys to your Azure
-Storage account are a secret and must not be shared.
+## Further Reading
 
+* [Testing the Azure WASB client](testing_azure.html).
+* MSDN article, [Understanding Block Blobs, Append Blobs, and Page Blobs](https://docs.microsoft.com/en-us/rest/api/storageservices/understanding-block-blobs--append-blobs--and-page-blobs)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md b/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md
new file mode 100644
index 0000000..b58e68b
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/site/markdown/testing_azure.md
@@ -0,0 +1,576 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# Testing the Azure WASB client
+
+<!-- MACRO{toc|fromDepth=0|toDepth=5} -->
+
+This module includes both unit tests, which can run in isolation without
+connecting to the Azure Storage service, and integration tests, which require a working
+connection to interact with a container.  Unit test suites follow the naming
+convention `Test*.java`.  Integration tests follow the naming convention
+`ITest*.java`.
+
+## Policy for submitting patches which affect the `hadoop-azure` module.
+
+The Apache Jenkins infrastucture does not run any cloud integration tests,
+due to the need to keep credentials secure.
+
+### The submitter of any patch is required to run all the integration tests and declare which Azure region they used.
+
+This is important: **patches which do not include this declaration will be ignored**
+
+This policy has proven to be the only mechanism to guarantee full regression
+testing of code changes. Why the declaration of region? Two reasons
+
+1. It helps us identify regressions which only surface against specific endpoints.
+1. It forces the submitters to be more honest about their testing. It's easy
+to lie, "yes, I tested this". To say "yes, I tested this against Azure US-west"
+is a more specific lie and harder to make. And, if you get caught out: you
+lose all credibility with the project.
+
+You don't need to test from a VM within the Azure infrastructure, all you need
+are credentials.
+
+It's neither hard nor expensive to run the tests; if you can't,
+there's no guarantee your patch works. The reviewers have enough to do, and
+don't have the time to do these tests, especially as every failure will simply
+make for a slow iterative development.
+
+Please: run the tests. And if you don't, we are sorry for declining your
+patch, but we have to.
+
+
+### What if there's an intermittent failure of a test?
+
+Some of the tests do fail intermittently, especially in parallel runs.
+If this happens, try to run the test on its own to see if the test succeeds.
+
+If it still fails, include this fact in your declaration. We know some tests
+are intermittently unreliable.
+
+### What if the tests are timing out or failing over my network connection?
+
+The tests are designed to be configurable for different
+timeouts. If you are seeing problems and this configuration isn't working,
+that's a sign of the configuration mechanism isn't complete. If it's happening
+in the production code, that could be a sign of a problem which may surface
+over long-haul connections. Please help us identify and fix these problems
+&mdash; especially as you are the one best placed to verify the fixes work.
+
+## Setting up the tests
+
+## Testing the `hadoop-azure` Module
+
+The `hadoop-azure` module includes a full suite of unit tests.  Many of the tests
+will run without additional configuration by running `mvn test`.  This includes
+tests against mocked storage, which is an in-memory emulation of Azure Storage.
+
+The integration tests are designed to test directly against an Azure storage
+service, and require an account and credentials in order to run.
+
+This is done by creating the file to `src/test/resources/azure-auth-keys.xml`
+and setting the name of the storage account and its access key.
+
+For example:
+
+```xml
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<configuration>
+  <property>
+    <name>fs.azure.test.account.name</name>
+    <value>{ACCOUNTNAME}.blob.core.windows.net</value>
+  </property>
+  <property>
+    <name>fs.azure.account.key.{ACCOUNTNAME}.blob.core.windows.net</name>
+    <value>{ACCOUNT ACCESS KEY}</value>
+  </property>
+</configuration>
+```
+
+To run contract tests, set the WASB file system URI in `src/test/resources/azure-auth-keys.xml`
+and the account access key. For example:
+
+```xml
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<configuration>
+  <property>
+    <name>fs.contract.test.fs.wasb</name>
+    <value>wasb://{CONTAINERNAME}@{ACCOUNTNAME}.blob.core.windows.net</value>
+    <description>The name of the azure file system for testing.</description>
+  </property>
+  <property>
+    <name>fs.azure.account.key.{ACCOUNTNAME}.blob.core.windows.net</name>
+    <value>{ACCOUNT ACCESS KEY}</value>
+  </property>
+</configuration>
+```
+
+Overall, to run all the tests using `mvn test`,  a sample `azure-auth-keys.xml` is like following:
+
+```xml
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<configuration>
+  <property>
+    <name>fs.azure.test.account.name</name>
+    <value>{ACCOUNTNAME}.blob.core.windows.net</value>
+  </property>
+  <property>
+    <name>fs.azure.account.key.{ACCOUNTNAME}.blob.core.windows.net</name>
+    <value>{ACCOUNT ACCESS KEY}</value>
+  </property>
+  <property>
+    <name>fs.contract.test.fs.wasb</name>
+    <value>wasb://{CONTAINERNAME}@{ACCOUNTNAME}.blob.core.windows.net</value>
+  </property>
+</configuration>
+```
+
+DO NOT ADD `azure-auth-keys.xml` TO REVISION CONTROL.  The keys to your Azure
+Storage account are a secret and must not be shared.
+
+
+## Running the Tests
+
+After completing the configuration, execute the test run through Maven.
+
+```bash
+mvn -T 1C clean verify
+```
+
+It's also possible to execute multiple test suites in parallel by passing the
+`parallel-tests` property on the command line.  The tests spend most of their
+time blocked on network I/O, so running in parallel tends to
+complete full test runs faster.
+
+```bash
+mvn -T 1C -Dparallel-tests clean verify
+```
+
+Some tests must run with exclusive access to the storage container, so even with the
+`parallel-tests` property, several test suites will run in serial in a separate
+Maven execution step after the parallel tests.
+
+By default, `parallel-tests` runs 4 test suites concurrently.  This can be tuned
+by passing the `testsThreadCount` property.
+
+```bash
+mvn -T 1C -Dparallel-tests -DtestsThreadCount=8 clean verify
+```
+
+<!---
+To run just unit tests, which do not require Azure connectivity or credentials,
+use any of the above invocations, but switch the goal to `test` instead of
+`verify`.
+-->
+
+```bash
+mvn -T 1C clean test
+
+mvn -T 1C -Dparallel-tests clean test
+
+mvn -T 1C -Dparallel-tests -DtestsThreadCount=8 clean test
+```
+
+To run only a specific named subset of tests, pass the `test` property for unit
+tests or the `it.test` property for integration tests.
+
+```bash
+mvn -T 1C clean test -Dtest=TestRollingWindowAverage
+
+mvn -T 1C clean verify -Dscale -Dit.test=ITestFileSystemOperationExceptionMessage -Dtest=none
+
+mvn -T 1C clean verify -Dtest=none -Dit.test=ITest*
+
+```
+
+Note
+
+1. When running a specific subset of tests, the patterns passed in `test`
+and `it.test` override the configuration of which tests need to run in isolation
+in a separate serial phase (mentioned above).  This can cause unpredictable
+results, so the recommendation is to avoid passing `parallel-tests` in
+combination with `test` or `it.test`.  If you know that you are specifying only
+tests that can run safely in parallel, then it will work.  For wide patterns,
+like `ITest*` shown above, it may cause unpredictable test failures.
+
+2. The command line shell may try to expand the "*" and sometimes the "#" symbols
+in test patterns. In such situations, escape the character it with a "\\" prefix.
+Example:
+
+          mvn -T 1C clean verify -Dtest=none -Dit.test=ITest\*
+
+
+## Viewing the results
+
+Integration test results and logs are stored in `target/failsafe-reports/`.
+An HTML report can be generated during site generation, or with the `surefire-report`
+plugin:
+
+```bash
+
+# for the unit tests
+mvn -T 1C surefire-report:report-only
+
+# for the integration tests
+mvn -T 1C surefire-report:failsafe-report-only
+
+# all reports for this module
+mvn -T 1C site:site
+```
+
+## Scale Tests
+
+There are a set of tests designed to measure the scalability and performance
+at scale of the filesystem client, *Scale Tests*. Tests include: creating
+and traversing directory trees, uploading large files, renaming them,
+deleting them, seeking through the files, performing random IO, and others.
+This makes them a foundational part of the benchmarking.
+
+By their very nature they are slow. And, as their execution time is often
+limited by bandwidth between the computer running the tests and the Azure endpoint,
+parallel execution does not speed these tests up.
+
+### Enabling the Scale Tests
+
+The tests are enabled if the `scale` property is set in the maven build
+this can be done regardless of whether or not the parallel test profile
+is used
+
+```bash
+mvn -T 1C verify -Dscale
+
+mvn -T 1C verify -Dparallel-tests -Dscale -DtestsThreadCount=8
+```
+
+The most bandwidth intensive tests (those which upload data) always run
+sequentially; those which are slow due to HTTPS setup costs or server-side
+actions are included in the set of parallelized tests.
+
+
+### Scale test tuning options
+
+
+Some of the tests can be tuned from the maven build or from the
+configuration file used to run the tests.
+
+```bash
+mvn -T 1C verify -Dparallel-tests -Dscale -DtestsThreadCount=8 -Dfs.azure.scale.test.huge.filesize=128M
+```
+
+The algorithm is
+
+1. The value is queried from the configuration file, using a default value if
+it is not set.
+1. The value is queried from the JVM System Properties, where it is passed
+down by maven.
+1. If the system property is null, an empty string, or it has the value `unset`,
+then the configuration value is used. The `unset` option is used to
+[work round a quirk in maven property propagation](http://stackoverflow.com/questions/7773134/null-versus-empty-arguments-in-maven).
+
+Only a few properties can be set this way; more will be added.
+
+| Property | Meaninging |
+|-----------|-------------|
+| `fs.azure.scale.test.huge.filesize`| Size for huge file uploads |
+| `fs.azure.scale.test.huge.huge.partitionsize`| Size for partitions in huge file uploads |
+
+The file and partition sizes are numeric values with a k/m/g/t/p suffix depending
+on the desired size. For example: 128M, 128m, 2G, 2G, 4T or even 1P.
+
+#### Scale test configuration options
+
+Some scale tests perform multiple operations (such as creating many directories).
+
+The exact number of operations to perform is configurable in the option
+`scale.test.operation.count`
+
+```xml
+<property>
+  <name>scale.test.operation.count</name>
+  <value>10</value>
+</property>
+```
+
+Larger values generate more load, and are recommended when testing locally,
+or in batch runs.
+
+Smaller values results in faster test runs, especially when the object
+store is a long way away.
+
+Operations which work on directories have a separate option: this controls
+the width and depth of tests creating recursive directories. Larger
+values create exponentially more directories, with consequent performance
+impact.
+
+```xml
+<property>
+  <name>scale.test.directory.count</name>
+  <value>2</value>
+</property>
+```
+
+DistCp tests targeting Azure support a configurable file size.  The default is
+10 MB, but the configuration value is expressed in KB so that it can be tuned
+smaller to achieve faster test runs.
+
+```xml
+<property>
+  <name>scale.test.distcp.file.size.kb</name>
+  <value>10240</value>
+</property>
+```
+
+Azure-specific scale test properties are
+
+##### `fs.azure.scale.test.huge.filesize`: size in MB for "Huge file tests".
+
+The Huge File tests validate Azure storages's ability to handle large files —the property
+`fs.azure.scale.test.huge.filesize` declares the file size to use.
+
+```xml
+<property>
+  <name>fs.azure.scale.test.huge.filesize</name>
+  <value>200M</value>
+</property>
+```
+
+Tests at this scale are slow: they are best executed from hosts running in
+the cloud infrastructure where the storage endpoint is based.
+
+## Using the emulator
+
+A selection of tests can run against the
+[Azure Storage Emulator](http://msdn.microsoft.com/en-us/library/azure/hh403989.aspx)
+which is a high-fidelity emulation of live Azure Storage.  The emulator is
+sufficient for high-confidence testing.  The emulator is a Windows executable
+that runs on a local machine.
+
+To use the emulator, install Azure SDK 2.3 and start the storage emulator.  Then,
+edit `src/test/resources/azure-test.xml` and add the following property:
+
+```xml
+<property>
+  <name>fs.azure.test.emulator</name>
+  <value>true</value>
+</property>
+```
+
+There is a known issue when running tests with the emulator.  You may see the
+following failure message:
+
+    com.microsoft.windowsazure.storage.StorageException: The value for one of the HTTP headers is not in the correct format.
+
+To resolve this, restart the Azure Emulator.  Ensure it is v3.2 or later.
+
+
+## Debugging Test failures
+
+Logging at debug level is the standard way to provide more diagnostics output;
+after setting this rerun the tests
+
+```properties
+log4j.logger.org.apache.hadoop.fs.azure=DEBUG
+```
+
+## Adding new tests
+
+New tests are always welcome. Bear in mind that we need to keep costs
+and test time down, which is done by
+
+* Not duplicating tests.
+* Being efficient in your use of Hadoop API calls.
+* Isolating large/slow tests into the "scale" test group.
+* Designing all tests to execute in parallel (where possible).
+* Adding new probes and predicates into existing tests, albeit carefully.
+
+*No duplication*: if an operation is tested elsewhere, don't repeat it. This
+applies as much for metadata operations as it does for bulk IO. If a new
+test case is added which completely obsoletes an existing test, it is OK
+to cut the previous one —after showing that coverage is not worsened.
+
+*Efficient*: prefer the `getFileStatus()` and examining the results, rather than
+call to `exists()`, `isFile()`, etc.
+
+*Fail with useful information:* provide as much diagnostics as possible
+on a failure. Using `org.apache.hadoop.fs.contract.ContractTestUtils` to make
+assertions about the state of a filesystem helps here.
+
+*Isolating Scale tests*. Any test doing large amounts of IO MUST extend the
+class `AbstractAzureScaleTest`, so only running if `scale` is defined on a build,
+supporting test timeouts configurable by the user. Scale tests should also
+support configurability as to the actual size of objects/number of operations,
+so that behavior at different scale can be verified.
+
+*Designed for parallel execution*. A key need here is for each test suite to work
+on isolated parts of the filesystem. Subclasses of `AbstractWasbTestBase`
+SHOULD use the `path()`, `methodpath()` and `blobpath()` methods,
+to build isolated paths. Tests MUST NOT assume that they have exclusive access
+to a bucket.
+
+*Extending existing tests where appropriate*. This recommendation goes
+against normal testing best practise of "test one thing per method".
+Because it is so slow to create directory trees or upload large files, we do
+not have that luxury. All the tests against real endpoints are integration
+tests where sharing test setup and teardown saves time and money.
+
+A standard way to do this is to extend existing tests with some extra predicates,
+rather than write new tests. When doing this, make sure that the new predicates
+fail with meaningful diagnostics, so any new problems can be easily debugged
+from test logs.
+
+
+### Requirements of new Tests
+
+
+This is what we expect from new tests; they're an extension of the normal
+Hadoop requirements, based on the need to work with remote servers whose
+use requires the presence of secret credentials, where tests may be slow,
+and where finding out why something failed from nothing but the test output
+is critical.
+
+#### Subclasses Existing Shared Base Blasses
+
+There are a set of base classes which should be extended for Azure tests and
+integration tests.
+
+##### `org.apache.hadoop.fs.azure.AbstractWasbTestWithTimeout`
+
+This extends the junit `Assert` class with thread names and timeouts,
+the default timeout being set in `AzureTestConstants.AZURE_TEST_TIMEOUT` to
+ten minutes. The thread names are set to aid analyzing the stack trace of
+a test: a `jstack` call can be used to
+
+##### `org.apache.hadoop.fs.azure.AbstractWasbTestBase`
+
+The base class for tests which use `AzureBlobStorageTestAccount` to create
+mock or live Azure clients; in test teardown it tries to clean up store state.
+
+1. This class requires subclasses to implement `createTestAccount()` to create
+a mock or real test account.
+
+1. The configuration used to create a test account *should* be that from
+`createConfiguration()`; this can be extended in subclasses to tune the settings.
+
+
+##### `org.apache.hadoop.fs.azure.integration.AbstractAzureScaleTest`
+
+This extends `AbstractWasbTestBase` for scale tests; those test which
+only run when `-Dscale` is used to select the "scale" profile.
+These tests have a timeout of 30 minutes, so as to support slow test runs.
+
+Having shared base classes help reduces future maintenance. Please
+use them.
+
+#### Secure
+
+Don't ever log credentials. The credential tests go out of their way to
+not provide meaningful logs or assertion messages precisely to avoid this.
+
+#### Efficient of Time and Money
+
+This means efficient in test setup/teardown, and, ideally, making use of
+existing public datasets to save setup time and tester cost.
+
+
+The reference example is `ITestAzureHugeFiles`:. This marks the test suite as
+`@FixMethodOrder(MethodSorters.NAME_ASCENDING)` then orders the test cases such
+that each test case expects the previous test to have completed (here: uploaded a file,
+renamed a file, ...). This provides for independent tests in the reports, yet still
+permits an ordered sequence of operations. Do note the use of `Assume.assume()`
+to detect when the preconditions for a single test case are not met, hence,
+the tests become skipped, rather than fail with a trace which is really a false alarm.
+
+
+### Works Over Long-haul Links
+
+As well as making file size and operation counts scaleable, this includes
+making test timeouts adequate. The Scale tests make this configurable; it's
+hard coded to ten minutes in `AbstractAzureIntegrationTest()`; subclasses can
+change this by overriding `getTestTimeoutMillis()`.
+
+Equally importantly: support proxies, as some testers need them.
+
+
+### Provides Diagnostics and timing information
+
+1. Create logs, log things.
+1. you can use `AbstractWasbTestBase.describe(format-string, args)` here; it
+adds some newlines so as to be easier to spot.
+1. Use `ContractTestUtils.NanoTimer` to measure the duration of operations,
+and log the output.
+
+#### Fails Meaningfully
+
+The `ContractTestUtils` class contains a whole set of assertions for making
+statements about the expected state of a filesystem, e.g.
+`assertPathExists(FS, path)`, `assertPathDoesNotExists(FS, path)`, and others.
+These do their best to provide meaningful diagnostics on failures (e.g. directory
+listings, file status, ...), so help make failures easier to understand.
+
+At the very least, *do not use `assertTrue()` or `assertFalse()` without
+including error messages*.
+
+
+### Cleans Up Afterwards
+
+Keeps costs down.
+
+1. Do not only cleanup if a test case completes successfully; test suite
+teardown must do it.
+1. That teardown code must check for the filesystem and other fields being
+null before the cleanup. Why? If test setup fails, the teardown methods still
+get called.
+
+### Works Reliably
+
+We really appreciate this &mdash; you will too.
+
+
+## Tips
+
+### How to keep your credentials really safe
+
+Although the `auth-keys.xml` file is marged as ignored in git and subversion,
+it is still in your source tree, and there's always that risk that it may
+creep out.
+
+You can avoid this by keeping your keys outside the source tree and
+using an absolute XInclude reference to it.
+
+```xml
+<configuration>
+
+  <include xmlns="http://www.w3.org/2001/XInclude"
+    href="file:///users/qe/.auth-keys.xml" />
+
+</configuration>
+```
+
+### Cleaning up Containers
+
+The Azure tests create containers with the prefix `"wasbtests-"` and delete
+them after the test runs. If a test run is interrupted, these containers
+may not get deleted. There is a special test case which can be manually invoked
+to list and delete these, `CleanupTestContainers`
+
+```bash
+mvn test -Dtest=CleanupTestContainers
+```
+
+This will delete the containers; the output log of the test run will
+provide the details and summary of the operation.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AbstractWasbTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AbstractWasbTestBase.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AbstractWasbTestBase.java
index d04a19c..0d3a06c 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AbstractWasbTestBase.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AbstractWasbTestBase.java
@@ -18,15 +18,21 @@
 
 package org.apache.hadoop.fs.azure;
 
-import static org.junit.Assume.assumeNotNull;
+import java.io.IOException;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.junit.After;
 import org.junit.Before;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azure.integration.AzureTestConstants;
+import org.apache.hadoop.io.IOUtils;
+
+import static org.junit.Assume.assumeNotNull;
+import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.*;
+
 /**
  * Abstract test class that provides basic setup and teardown of testing Azure
  * Storage account.  Each subclass defines a different set of test cases to run
@@ -34,41 +40,137 @@ import org.slf4j.LoggerFactory;
  * to run those tests.  The returned account might integrate with Azure Storage
  * directly or it might be a mock implementation.
  */
-public abstract class AbstractWasbTestBase {
+public abstract class AbstractWasbTestBase extends AbstractWasbTestWithTimeout
+    implements AzureTestConstants {
 
   protected static final Logger LOG =
       LoggerFactory.getLogger(AbstractWasbTestBase.class);
 
-  @VisibleForTesting
   protected NativeAzureFileSystem fs;
-  private AzureBlobStorageTestAccount testAccount;
+  protected AzureBlobStorageTestAccount testAccount;
 
   @Before
   public void setUp() throws Exception {
-    testAccount = createTestAccount();
-    if (testAccount != null) {
-      fs = testAccount.getFileSystem();
-    }
-    assumeNotNull(testAccount);
+    AzureBlobStorageTestAccount account = createTestAccount();
+    assumeNotNull(account);
+    bindToTestAccount(account);
   }
 
   @After
   public void tearDown() throws Exception {
-    if (testAccount != null) {
-      testAccount.cleanup();
-      testAccount = null;
-      fs = null;
-    }
+    describe("closing test account and filesystem");
+    testAccount = cleanupTestAccount(testAccount);
+    IOUtils.closeStream(fs);
+    fs = null;
   }
 
-  public Configuration getConfiguration() {
-    return new Configuration();
+  /**
+   * Create the configuration to use when creating a test account.
+   * Subclasses can override this to tune the test account configuration.
+   * @return a configuration.
+   */
+  public Configuration createConfiguration() {
+    return AzureBlobStorageTestAccount.createTestConfiguration();
   }
 
+  /**
+   * Create the test account.
+   * Subclasses must implement this.
+   * @return the test account.
+   * @throws Exception
+   */
   protected abstract AzureBlobStorageTestAccount createTestAccount()
       throws Exception;
 
+  /**
+   * Get the test account.
+   * @return the current test account.
+   */
   protected AzureBlobStorageTestAccount getTestAccount() {
     return testAccount;
   }
+
+  /**
+   * Get the filesystem
+   * @return the current filesystem.
+   */
+  protected NativeAzureFileSystem getFileSystem() {
+    return fs;
+  }
+
+  /**
+   * Get the configuration used to create the filesystem
+   * @return the configuration of the test FS
+   */
+  protected Configuration getConfiguration() {
+    return getFileSystem().getConf();
+  }
+
+  /**
+   * Bind to a new test account; closing any existing one.
+   * This updates the test account returned in {@link #getTestAccount()}
+   * and the filesystem in {@link #getFileSystem()}.
+   * @param account new test account
+   */
+  protected void bindToTestAccount(AzureBlobStorageTestAccount account) {
+    // clean any existing test account
+    cleanupTestAccount(testAccount);
+    IOUtils.closeStream(fs);
+    testAccount = account;
+    if (testAccount != null) {
+      fs = testAccount.getFileSystem();
+    }
+  }
+
+  /**
+   * Return a path to a blob which will be unique for this fork.
+   * @param filepath filepath
+   * @return a path under the default blob directory
+   * @throws IOException
+   */
+  protected Path blobPath(String filepath) throws IOException {
+    return blobPathForTests(getFileSystem(), filepath);
+  }
+
+  /**
+   * Create a path under the test path provided by
+   * the FS contract.
+   * @param filepath path string in
+   * @return a path qualified by the test filesystem
+   * @throws IOException IO problems
+   */
+  protected Path path(String filepath) throws IOException {
+    return pathForTests(getFileSystem(), filepath);
+  }
+
+  /**
+   * Return a path bonded to this method name, unique to this fork during
+   * parallel execution.
+   * @return a method name unique to (fork, method).
+   * @throws IOException IO problems
+   */
+  protected Path methodPath() throws IOException {
+    return path(methodName.getMethodName());
+  }
+
+  /**
+   * Return a blob path bonded to this method name, unique to this fork during
+   * parallel execution.
+   * @return a method name unique to (fork, method).
+   * @throws IOException IO problems
+   */
+  protected Path methodBlobPath() throws IOException {
+    return blobPath(methodName.getMethodName());
+  }
+
+  /**
+   * Describe a test in the logs.
+   * @param text text to print
+   * @param args arguments to format in the printing
+   */
+  protected void describe(String text, Object... args) {
+    LOG.info("\n\n{}: {}\n",
+        methodName.getMethodName(),
+        String.format(text, args));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AbstractWasbTestWithTimeout.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AbstractWasbTestWithTimeout.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AbstractWasbTestWithTimeout.java
new file mode 100644
index 0000000..b7076a4
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AbstractWasbTestWithTimeout.java
@@ -0,0 +1,73 @@
+/*
+ * 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.azure;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+import org.junit.rules.Timeout;
+
+import org.apache.hadoop.fs.azure.integration.AzureTestConstants;
+
+/**
+ * Base class for any Wasb test with timeouts & named threads.
+ * This class does not attempt to bind to Azure.
+ */
+public class AbstractWasbTestWithTimeout extends Assert {
+
+  /**
+   * The name of the current method.
+   */
+  @Rule
+  public TestName methodName = new TestName();
+  /**
+   * Set the timeout for every test.
+   * This is driven by the value returned by {@link #getTestTimeoutMillis()}.
+   */
+  @Rule
+  public Timeout testTimeout = new Timeout(getTestTimeoutMillis());
+
+  /**
+   * Name the junit thread for the class. This will overridden
+   * before the individual test methods are run.
+   */
+  @BeforeClass
+  public static void nameTestThread() {
+    Thread.currentThread().setName("JUnit");
+  }
+
+  /**
+   * Name the thread to the current test method.
+   */
+  @Before
+  public void nameThread() {
+    Thread.currentThread().setName("JUnit-" + methodName.getMethodName());
+  }
+
+  /**
+   * Override point: the test timeout in milliseconds.
+   * @return a timeout in milliseconds
+   */
+  protected int getTestTimeoutMillis() {
+    return AzureTestConstants.AZURE_TEST_TIMEOUT;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AzureBlobStorageTestAccount.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AzureBlobStorageTestAccount.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AzureBlobStorageTestAccount.java
index 7fa59ce..5b36c87 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AzureBlobStorageTestAccount.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AzureBlobStorageTestAccount.java
@@ -21,12 +21,15 @@ package org.apache.hadoop.fs.azure;
 import com.microsoft.azure.storage.*;
 import com.microsoft.azure.storage.blob.*;
 import com.microsoft.azure.storage.core.Base64;
-import org.apache.commons.configuration2.SubsetConfiguration;
+import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.commons.configuration2.SubsetConfiguration;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azure.integration.AzureTestConstants;
 import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
 import org.apache.hadoop.fs.azure.metrics.AzureFileSystemMetricsSystem;
 import org.apache.hadoop.metrics2.AbstractMetric;
@@ -35,6 +38,8 @@ import org.apache.hadoop.metrics2.MetricsSink;
 import org.apache.hadoop.metrics2.MetricsTag;
 import org.apache.hadoop.metrics2.impl.TestMetricsConfig;
 
+import java.io.File;
+import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.*;
@@ -46,10 +51,10 @@ import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.KEY_USE_SECU
 
 /**
  * Helper class to create WASB file systems backed by either a mock in-memory
- * implementation or a real Azure Storage account. See RunningLiveWasbTests.txt
- * for instructions on how to connect to a real Azure Storage account.
+ * implementation or a real Azure Storage account.
  */
-public final class AzureBlobStorageTestAccount {
+public final class AzureBlobStorageTestAccount implements AutoCloseable,
+    AzureTestConstants {
   private static final Logger LOG = LoggerFactory.getLogger(
       AzureBlobStorageTestAccount.class);
 
@@ -166,6 +171,7 @@ public final class AzureBlobStorageTestAccount {
     return new Path("/" + DEFAULT_PAGE_BLOB_DIRECTORY);
   }
 
+  @Deprecated
   public static Path pageBlobPath(String fileName) {
     return new Path(pageBlobPath(), fileName);
   }
@@ -201,6 +207,9 @@ public final class AzureBlobStorageTestAccount {
    * @return
    */
   private boolean wasGeneratedByMe(MetricsRecord currentRecord) {
+    Assert.assertNotNull("null filesystem", fs);
+    Assert.assertNotNull("null filesystemn instance ID",
+        fs.getInstrumentation().getFileSystemInstanceId());
     String myFsId = fs.getInstrumentation().getFileSystemInstanceId().toString();
     for (MetricsTag currentTag : currentRecord.tags()) {
       if (currentTag.name().equalsIgnoreCase("wasbFileSystemId")) {
@@ -247,13 +256,16 @@ public final class AzureBlobStorageTestAccount {
     getBlobReference(blobKey).releaseLease(accessCondition);
   }
 
-  private static void saveMetricsConfigFile() {
+  private static void saveMetricsConfigFile() throws IOException {
     if (!metricsConfigSaved) {
+      String testFilename = TestMetricsConfig.getTestFilename(
+          "hadoop-metrics2-azure-file-system");
+      File dest = new File(testFilename).getCanonicalFile();
+      dest.getParentFile().mkdirs();
       new org.apache.hadoop.metrics2.impl.ConfigBuilder()
       .add("azure-file-system.sink.azuretestcollector.class",
           StandardCollector.class.getName())
-      .save(TestMetricsConfig.getTestFilename(
-          "hadoop-metrics2-azure-file-system.properties"));
+      .save(testFilename);
       metricsConfigSaved = true;
     }
   }
@@ -314,9 +326,8 @@ public final class AzureBlobStorageTestAccount {
     Configuration conf = createTestConfiguration();
     if (!conf.getBoolean(USE_EMULATOR_PROPERTY_NAME, false)) {
       // Not configured to test against the storage emulator.
-      LOG.warn("Skipping emulator Azure test because configuration doesn't "
-          + "indicate that it's running. Please see RunningLiveWasbTests.txt "
-          + "for guidance.");
+      LOG.warn("Skipping emulator Azure test because configuration "
+          + "doesn't indicate that it's running.");
       return null;
     }
     CloudStorageAccount account =
@@ -482,8 +493,7 @@ public final class AzureBlobStorageTestAccount {
         credentials = StorageCredentialsAnonymous.ANONYMOUS;
       } else {
         LOG.warn("Skipping live Azure test because of missing key for"
-            + " account '" + accountName + "'. "
-            + "Please see RunningLiveWasbTests.txt for guidance.");
+            + " account '" + accountName + "'.");
         return null;
       }
     } else {
@@ -517,8 +527,7 @@ public final class AzureBlobStorageTestAccount {
       throws URISyntaxException, KeyProviderException {
     String testAccountName = conf.get(TEST_ACCOUNT_NAME_PROPERTY_NAME);
     if (testAccountName == null) {
-      LOG.warn("Skipping live Azure test because of missing test account. "
-          + "Please see RunningLiveWasbTests.txt for guidance.");
+      LOG.warn("Skipping live Azure test because of missing test account");
       return null;
     }
     return createStorageAccount(testAccountName, conf, false);
@@ -863,6 +872,11 @@ public final class AzureBlobStorageTestAccount {
     }
   }
 
+  @Override
+  public void close() throws Exception {
+    cleanup();
+  }
+
   public NativeAzureFileSystem getFileSystem() {
     return fs;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestAzureConcurrentOutOfBandIo.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestAzureConcurrentOutOfBandIo.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestAzureConcurrentOutOfBandIo.java
new file mode 100644
index 0000000..7e733dc
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestAzureConcurrentOutOfBandIo.java
@@ -0,0 +1,179 @@
+/**
+ * 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.azure;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azure.integration.AzureTestUtils;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+
+/**
+ * Handle OOB IO into a shared container.
+ */
+public class ITestAzureConcurrentOutOfBandIo extends AbstractWasbTestBase {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ITestAzureConcurrentOutOfBandIo.class);
+
+  // Class constants.
+  static final int DOWNLOAD_BLOCK_SIZE = 8 * 1024 * 1024;
+  static final int UPLOAD_BLOCK_SIZE = 4 * 1024 * 1024;
+  static final int BLOB_SIZE = 32 * 1024 * 1024;
+
+  // Number of blocks to be written before flush.
+  static final int NUMBER_OF_BLOCKS = 2;
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.createOutOfBandStore(
+        UPLOAD_BLOCK_SIZE, DOWNLOAD_BLOCK_SIZE);
+  }
+
+  class DataBlockWriter implements Runnable {
+
+    Thread runner;
+    AzureBlobStorageTestAccount writerStorageAccount;
+    String key;
+    boolean done = false;
+
+    /**
+     * Constructor captures the test account.
+     * 
+     * @param testAccount
+     */
+    public DataBlockWriter(AzureBlobStorageTestAccount testAccount, String key) {
+      writerStorageAccount = testAccount;
+      this.key = key;
+    }
+
+    /**
+     * Start writing blocks to Azure storage.
+     */
+    public void startWriting() {
+      runner = new Thread(this); // Create the block writer thread.
+      runner.start(); // Start the block writer thread.
+    }
+
+    /**
+     * Stop writing blocks to Azure storage.
+     */
+    public void stopWriting() {
+      done = true;
+    }
+
+    /**
+     * Implementation of the runnable interface. The run method is a tight loop
+     * which repeatedly updates the blob with a 4 MB block.
+     */
+    public void run() {
+      byte[] dataBlockWrite = new byte[UPLOAD_BLOCK_SIZE];
+
+      OutputStream outputStream = null;
+
+      try {
+        for (int i = 0; !done; i++) {
+          // Write two 4 MB blocks to the blob.
+          //
+          outputStream = writerStorageAccount.getStore().storefile(
+              key,
+              new PermissionStatus("", "", FsPermission.getDefault()),
+              key);
+
+          Arrays.fill(dataBlockWrite, (byte) (i % 256));
+          for (int j = 0; j < NUMBER_OF_BLOCKS; j++) {
+            outputStream.write(dataBlockWrite);
+          }
+
+          outputStream.flush();
+          outputStream.close();
+        }
+      } catch (AzureException e) {
+        LOG.error("DatablockWriter thread encountered a storage exception."
+            + e.getMessage(), e);
+      } catch (IOException e) {
+        LOG.error("DatablockWriter thread encountered an I/O exception."
+            + e.getMessage(), e);
+      }
+    }
+  }
+
+  @Test
+  public void testReadOOBWrites() throws Exception {
+
+    byte[] dataBlockWrite = new byte[UPLOAD_BLOCK_SIZE];
+    byte[] dataBlockRead = new byte[UPLOAD_BLOCK_SIZE];
+
+    // Write to blob to make sure it exists.
+    //
+   // Write five 4 MB blocks to the blob. To ensure there is data in the blob before
+   // reading.  This eliminates the race between the reader and writer threads.
+    String key = "WASB_String" + AzureTestUtils.getForkID() + ".txt";
+    OutputStream outputStream = testAccount.getStore().storefile(
+       key,
+       new PermissionStatus("", "", FsPermission.getDefault()),
+           key);
+   Arrays.fill(dataBlockWrite, (byte) 255);
+   for (int i = 0; i < NUMBER_OF_BLOCKS; i++) {
+     outputStream.write(dataBlockWrite);
+   }
+
+   outputStream.flush();
+   outputStream.close();
+
+   // Start writing blocks to Azure store using the DataBlockWriter thread.
+    DataBlockWriter writeBlockTask = new DataBlockWriter(testAccount, key);
+    writeBlockTask.startWriting();
+   int count = 0;
+
+   for (int i = 0; i < 5; i++) {
+     try(InputStream inputStream = testAccount.getStore().retrieve(key)) {
+        count = 0;
+        int c = 0;
+
+        while (c >= 0) {
+          c = inputStream.read(dataBlockRead, 0, UPLOAD_BLOCK_SIZE);
+          if (c < 0) {
+            break;
+          }
+
+          // Counting the number of bytes.
+          count += c;
+        }
+     } catch (IOException e) {
+       System.out.println(e.getCause().toString());
+       e.printStackTrace();
+       fail();
+     }
+   }
+
+    // Stop writing blocks.
+    writeBlockTask.stopWriting();
+
+    // Validate that a block was read.
+    assertEquals(NUMBER_OF_BLOCKS * UPLOAD_BLOCK_SIZE, count);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestAzureConcurrentOutOfBandIoWithSecureMode.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestAzureConcurrentOutOfBandIoWithSecureMode.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestAzureConcurrentOutOfBandIoWithSecureMode.java
new file mode 100644
index 0000000..2b0ea56
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestAzureConcurrentOutOfBandIoWithSecureMode.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure;
+
+/**
+ * Extends ITestAzureConcurrentOutOfBandIo in order to run testReadOOBWrites with secure mode
+ * (fs.azure.secure.mode) both enabled and disabled.
+ */
+public class ITestAzureConcurrentOutOfBandIoWithSecureMode
+    extends ITestAzureConcurrentOutOfBandIo {
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.createOutOfBandStore(
+        UPLOAD_BLOCK_SIZE, DOWNLOAD_BLOCK_SIZE, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestAzureFileSystemErrorConditions.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestAzureFileSystemErrorConditions.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestAzureFileSystemErrorConditions.java
new file mode 100644
index 0000000..49e6730
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestAzureFileSystemErrorConditions.java
@@ -0,0 +1,243 @@
+/**
+ * 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.azure;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.concurrent.Callable;
+
+import com.microsoft.azure.storage.OperationContext;
+import com.microsoft.azure.storage.SendingRequestEvent;
+import com.microsoft.azure.storage.StorageEvent;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.TestHookOperationContext;
+import org.apache.hadoop.test.GenericTestUtils;
+
+import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.NO_ACCESS_TO_CONTAINER_MSG;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+import static org.junit.Assume.assumeNotNull;
+
+/**
+ * Error handling.
+ */
+public class ITestAzureFileSystemErrorConditions extends
+    AbstractWasbTestWithTimeout {
+  private static final int ALL_THREE_FILE_SIZE = 1024;
+
+  @Test
+  public void testNoInitialize() throws Exception {
+    intercept(AssertionError.class,
+        new Callable<FileMetadata>() {
+          @Override
+          public FileMetadata call() throws Exception {
+            return new AzureNativeFileSystemStore()
+                .retrieveMetadata("foo");
+          }
+        });
+  }
+
+  /**
+   * Try accessing an unauthorized or non-existent (treated the same) container
+   * from WASB.
+   */
+  @Test
+  public void testAccessUnauthorizedPublicContainer() throws Exception {
+    final String container = "nonExistentContainer";
+    final String account = "hopefullyNonExistentAccount";
+    Path noAccessPath = new Path(
+        "wasb://" + container + "@" + account + "/someFile");
+    NativeAzureFileSystem.suppressRetryPolicy();
+    try {
+      FileSystem.get(noAccessPath.toUri(), new Configuration())
+        .open(noAccessPath);
+      assertTrue("Should've thrown.", false);
+    } catch (AzureException ex) {
+      GenericTestUtils.assertExceptionContains(
+          String.format(NO_ACCESS_TO_CONTAINER_MSG, account, container), ex);
+    } finally {
+      NativeAzureFileSystem.resumeRetryPolicy();
+    }
+  }
+
+  @Test
+  public void testAccessContainerWithWrongVersion() throws Exception {
+    AzureNativeFileSystemStore store = new AzureNativeFileSystemStore();
+    MockStorageInterface mockStorage = new MockStorageInterface();
+    store.setAzureStorageInteractionLayer(mockStorage);
+    try (FileSystem fs = new NativeAzureFileSystem(store)) {
+      Configuration conf = new Configuration();
+      AzureBlobStorageTestAccount.setMockAccountKey(conf);
+      HashMap<String, String> metadata = new HashMap<String, String>();
+      metadata.put(AzureNativeFileSystemStore.VERSION_METADATA_KEY,
+          "2090-04-05"); // It's from the future!
+      mockStorage.addPreExistingContainer(
+          AzureBlobStorageTestAccount.getMockContainerUri(), metadata);
+
+      AzureException ex = intercept(AzureException.class,
+          new Callable<FileStatus[]>() {
+            @Override
+            public FileStatus[] call() throws Exception {
+              fs.initialize(new URI(AzureBlobStorageTestAccount.MOCK_WASB_URI),
+                  conf);
+              return fs.listStatus(new Path("/"));
+            }
+          });
+      GenericTestUtils.assertExceptionContains(
+          "unsupported version: 2090-04-05.", ex);
+    }
+  }
+
+  private interface ConnectionRecognizer {
+    boolean isTargetConnection(HttpURLConnection connection);
+  }
+
+  private class TransientErrorInjector extends StorageEvent<SendingRequestEvent> {
+    private final ConnectionRecognizer connectionRecognizer;
+    private boolean injectedErrorOnce = false;
+
+    public TransientErrorInjector(ConnectionRecognizer connectionRecognizer) {
+      this.connectionRecognizer = connectionRecognizer;
+    }
+
+    @Override
+    public void eventOccurred(SendingRequestEvent eventArg) {
+      HttpURLConnection connection
+          = (HttpURLConnection) eventArg.getConnectionObject();
+      if (!connectionRecognizer.isTargetConnection(connection)) {
+        return;
+      }
+      if (!injectedErrorOnce) {
+        connection.setReadTimeout(1);
+        connection.disconnect();
+        injectedErrorOnce = true;
+      }
+    }
+  }
+
+  private void injectTransientError(NativeAzureFileSystem fs,
+      final ConnectionRecognizer connectionRecognizer) {
+    fs.getStore().addTestHookToOperationContext(new TestHookOperationContext() {
+      @Override
+      public OperationContext modifyOperationContext(OperationContext original) {
+        original.getSendingRequestEventHandler().addListener(
+            new TransientErrorInjector(connectionRecognizer));
+        return original;
+      }
+    });
+  }
+
+  @Test
+  public void testTransientErrorOnDelete() throws Exception {
+    // Need to do this test against a live storage account
+    AzureBlobStorageTestAccount testAccount =
+        AzureBlobStorageTestAccount.create();
+    assumeNotNull(testAccount);
+    try {
+      NativeAzureFileSystem fs = testAccount.getFileSystem();
+      injectTransientError(fs, new ConnectionRecognizer() {
+        @Override
+        public boolean isTargetConnection(HttpURLConnection connection) {
+          return connection.getRequestMethod().equals("DELETE");
+        }
+      });
+      Path testFile = new Path("/a/b");
+      assertTrue(fs.createNewFile(testFile));
+      assertTrue(fs.rename(testFile, new Path("/x")));
+    } finally {
+      testAccount.cleanup();
+    }
+  }
+
+  private void writeAllThreeFile(NativeAzureFileSystem fs, Path testFile)
+      throws IOException {
+    byte[] buffer = new byte[ALL_THREE_FILE_SIZE];
+    Arrays.fill(buffer, (byte) 3);
+    try(OutputStream stream = fs.create(testFile)) {
+      stream.write(buffer);
+    }
+  }
+
+  private void readAllThreeFile(NativeAzureFileSystem fs, Path testFile)
+      throws IOException {
+    byte[] buffer = new byte[ALL_THREE_FILE_SIZE];
+    InputStream inStream = fs.open(testFile);
+    assertEquals(buffer.length,
+        inStream.read(buffer, 0, buffer.length));
+    inStream.close();
+    for (int i = 0; i < buffer.length; i++) {
+      assertEquals(3, buffer[i]);
+    }
+  }
+
+  @Test
+  public void testTransientErrorOnCommitBlockList() throws Exception {
+    // Need to do this test against a live storage account
+    AzureBlobStorageTestAccount testAccount =
+        AzureBlobStorageTestAccount.create();
+    assumeNotNull(testAccount);
+    try {
+      NativeAzureFileSystem fs = testAccount.getFileSystem();
+      injectTransientError(fs, new ConnectionRecognizer() {
+        @Override
+        public boolean isTargetConnection(HttpURLConnection connection) {
+          return connection.getRequestMethod().equals("PUT")
+              && connection.getURL().getQuery() != null
+              && connection.getURL().getQuery().contains("blocklist");
+        }
+      });
+      Path testFile = new Path("/a/b");
+      writeAllThreeFile(fs, testFile);
+      readAllThreeFile(fs, testFile);
+    } finally {
+      testAccount.cleanup();
+    }
+  }
+
+  @Test
+  public void testTransientErrorOnRead() throws Exception {
+    // Need to do this test against a live storage account
+    AzureBlobStorageTestAccount testAccount =
+        AzureBlobStorageTestAccount.create();
+    assumeNotNull(testAccount);
+    try {
+      NativeAzureFileSystem fs = testAccount.getFileSystem();
+      Path testFile = new Path("/a/b");
+      writeAllThreeFile(fs, testFile);
+      injectTransientError(fs, new ConnectionRecognizer() {
+        @Override
+        public boolean isTargetConnection(HttpURLConnection connection) {
+          return connection.getRequestMethod().equals("GET");
+        }
+      });
+      readAllThreeFile(fs, testFile);
+    } finally {
+      testAccount.cleanup();
+    }
+  }
+}


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


[45/50] [abbrv] hadoop git commit: Revert "YARN-7162. Remove XML excludes file format (rkanter)" - wrong commit message

Posted by ae...@apache.org.
Revert "YARN-7162. Remove XML excludes file format (rkanter)" - wrong commit message

This reverts commit 3a8d57a0a2e047b34be82f602a2b6cf5593d2125.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5f496683
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5f496683
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5f496683

Branch: refs/heads/HDFS-7240
Commit: 5f496683fb00ba26a6bf5a506ae87d4bc4088727
Parents: a2dcba1
Author: Robert Kanter <rk...@apache.org>
Authored: Mon Sep 18 10:32:08 2017 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Mon Sep 18 10:32:08 2017 -0700

----------------------------------------------------------------------
 .../hadoop-mapreduce-client-core/pom.xml        |  4 --
 .../hadoop/mapreduce/JobResourceUploader.java   | 17 --------
 .../apache/hadoop/mapreduce/MRJobConfig.java    |  5 ---
 .../src/main/resources/mapred-default.xml       |  9 ----
 .../mapreduce/TestJobResourceUploader.java      | 46 --------------------
 5 files changed, 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f496683/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
index ce5fdc8..c34f7bd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
@@ -44,10 +44,6 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs-client</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-hdfs</artifactId>
       <scope>test</scope>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f496683/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
index d9bf988..f1cad57 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
@@ -36,8 +36,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.mapreduce.filecache.ClientDistributedCacheManager;
 import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 
@@ -96,11 +94,6 @@ class JobResourceUploader {
         new FsPermission(JobSubmissionFiles.JOB_DIR_PERMISSION);
     mkdirs(jtFs, submitJobDir, mapredSysPerms);
 
-    if (!conf.getBoolean(MRJobConfig.MR_AM_STAGING_DIR_ERASURECODING_ENABLED,
-        MRJobConfig.DEFAULT_MR_AM_STAGING_ERASURECODING_ENABLED)) {
-      disableErasureCodingForPath(jtFs, submitJobDir);
-    }
-
     Collection<String> files = conf.getStringCollection("tmpfiles");
     Collection<String> libjars = conf.getStringCollection("tmpjars");
     Collection<String> archives = conf.getStringCollection("tmparchives");
@@ -582,14 +575,4 @@ class JobResourceUploader {
     }
     return finalPath;
   }
-
-  private void disableErasureCodingForPath(FileSystem fs, Path path)
-      throws IOException {
-    if (jtFs instanceof DistributedFileSystem) {
-      LOG.info("Disabling Erasure Coding for path: " + path);
-      DistributedFileSystem dfs = (DistributedFileSystem) jtFs;
-      dfs.setErasureCodingPolicy(path,
-          SystemErasureCodingPolicies.getReplicationPolicy().getName());
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f496683/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index 86abb42..2023ba3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -1037,9 +1037,4 @@ public interface MRJobConfig {
   String FINISH_JOB_WHEN_REDUCERS_DONE =
       "mapreduce.job.finish-when-all-reducers-done";
   boolean DEFAULT_FINISH_JOB_WHEN_REDUCERS_DONE = true;
-
-  String MR_AM_STAGING_DIR_ERASURECODING_ENABLED =
-      MR_AM_STAGING_DIR + "erasurecoding.enabled";
-
-  boolean DEFAULT_MR_AM_STAGING_ERASURECODING_ENABLED = false;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f496683/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index 6b6faf2..ee9b906 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -1261,15 +1261,6 @@
 </property>
 
 <property>
-  <name>yarn.app.mapreduce.am.staging-dir.erasurecoding.enabled</name>
-  <value>false</value>
-  <description>Whether Erasure Coding should be enabled for
-  files that are copied to the MR staging area. This is a job-level
-  setting.
-  </description>
-</property>
-
-<property>
   <name>mapreduce.am.max-attempts</name>
   <value>2</value>
   <description>The maximum number of application attempts. It is a

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f496683/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
index d0d7a34..20b7b7d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
@@ -20,11 +20,6 @@ package org.apache.hadoop.mapreduce;
 
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.times;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
 
 import java.io.IOException;
 import java.net.URI;
@@ -41,12 +36,9 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.mapred.JobConf;
 import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.verification.VerificationMode;
 
 /**
  * A class for unit testing JobResourceUploader.
@@ -365,40 +357,6 @@ public class TestJobResourceUploader {
         expectedArchivesWithFrags, expectedJobJar);
   }
 
-  @Test
-  public void testErasureCodingDefault() throws IOException {
-    testErasureCodingSetting(true);
-  }
-
-  @Test
-  public void testErasureCodingDisabled() throws IOException {
-    testErasureCodingSetting(false);
-  }
-
-  private void testErasureCodingSetting(boolean defaultBehavior)
-      throws IOException {
-    JobConf jConf = new JobConf();
-    // don't set to false if EC remains disabled to check default setting
-    if (!defaultBehavior) {
-      jConf.setBoolean(MRJobConfig.MR_AM_STAGING_DIR_ERASURECODING_ENABLED,
-          true);
-    }
-
-    DistributedFileSystem fs = mock(DistributedFileSystem.class);
-    Path path = new Path("/");
-    when(fs.makeQualified(any(Path.class))).thenReturn(path);
-    JobResourceUploader uploader = new StubedUploader(fs, true);
-    Job job = Job.getInstance(jConf);
-
-    uploader.uploadResources(job, new Path("/test"));
-
-    String replicationPolicyName = SystemErasureCodingPolicies
-        .getReplicationPolicy().getName();
-    VerificationMode mode = defaultBehavior ? times(1) : never();
-    verify(fs, mode).setErasureCodingPolicy(eq(path),
-        eq(replicationPolicyName));
-  }
-
   private void runTmpResourcePathTest(JobResourceUploader uploader,
       ResourceConf rConf, JobConf jConf, String[] expectedFiles,
       String[] expectedArchives, String expectedJobJar) throws IOException {
@@ -740,10 +698,6 @@ public class TestJobResourceUploader {
       super(FileSystem.getLocal(conf), useWildcard);
     }
 
-    StubedUploader(FileSystem fs, boolean useWildcard) throws IOException {
-      super(fs, useWildcard);
-    }
-
     @Override
     FileStatus getFileStatus(Map<URI, FileStatus> statCache, Configuration job,
         Path p) throws IOException {


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


[09/50] [abbrv] hadoop git commit: HADOOP-14738 Remove S3N and obsolete bits of S3A; rework docs. Contributed by Steve Loughran.

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
index 1a45db3..5a7129f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -18,312 +18,48 @@
 
 package org.apache.hadoop.fs.s3native;
 
-import java.io.BufferedOutputStream;
-import java.io.EOFException;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
 import java.net.URI;
-import java.security.DigestOutputStream;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
 
-import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.BufferedFSInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FSExceptionMessages;
-import org.apache.hadoop.fs.FSInputStream;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.retry.RetryPolicies;
-import org.apache.hadoop.io.retry.RetryPolicy;
-import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.util.Progressable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_BUFFER_DIR_DEFAULT;
-import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_BUFFER_DIR_KEY;
-import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_MAX_RETRIES_DEFAUL;
-import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_MAX_RETRIES_KEY;
-import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_SLEEP_TIME_DEFAULT;
-import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_SLEEP_TIME_KEY;
-import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.addDeprecatedConfigKeys;
 
 /**
- * A {@link FileSystem} for reading and writing files stored on
- * <a href="http://aws.amazon.com/s3">Amazon S3</a>.
- * This implementation stores files on S3 in their native form so they can be
- * read by other S3 tools.
- * <p>
- * A note about directories. S3 of course has no "native" support for them.
- * The idiom we choose then is: for any directory created by this class,
- * we use an empty object "#{dirpath}_$folder$" as a marker.
- * Further, to interoperate with other S3 tools, we also accept the following:
- * <ul>
- *   <li>an object "#{dirpath}/' denoting a directory marker</li>
- *   <li>
- *     if there exists any objects with the prefix "#{dirpath}/", then the
- *     directory is said to exist
- *   </li>
- *   <li>
- *     if both a file with the name of a directory and a marker for that
- *     directory exists, then the *file masks the directory*, and the directory
- *     is never returned.
- *   </li>
- * </ul>
+ * This is a stub filesystem purely present to fail meaningfully when
+ * someone who explicitly declares
+ * {@code fs.s3n.impl=org.apache.hadoop.fs.s3native.NativeS3FileSystem}
+ * and then tries to create a filesystem off an s3n:// URL.
+ *
+ * The {@link #initialize(URI, Configuration)} method will throw
+ * an IOException informing the user of their need to migrate.
+ * @deprecated Replaced by the S3A client.
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
-public class NativeS3FileSystem extends FileSystem {
+public final class NativeS3FileSystem extends FileSystem {
   
   public static final Logger LOG =
       LoggerFactory.getLogger(NativeS3FileSystem.class);
-  
-  private static final String FOLDER_SUFFIX = "_$folder$";
-  static final String PATH_DELIMITER = Path.SEPARATOR;
-  private static final int S3_MAX_LISTING_LENGTH = 1000;
-
-  static {
-    // Add the deprecated config keys
-    addDeprecatedConfigKeys();
-  }
-
-  static class NativeS3FsInputStream extends FSInputStream {
-    
-    private NativeFileSystemStore store;
-    private Statistics statistics;
-    private InputStream in;
-    private final String key;
-    private long pos = 0;
-    
-    public NativeS3FsInputStream(NativeFileSystemStore store, Statistics statistics, InputStream in, String key) {
-      Preconditions.checkNotNull(in, "Null input stream");
-      this.store = store;
-      this.statistics = statistics;
-      this.in = in;
-      this.key = key;
-    }
-    
-    @Override
-    public synchronized int read() throws IOException {
-      int result;
-      try {
-        result = in.read();
-      } catch (IOException e) {
-        LOG.info("Received IOException while reading '{}', attempting to reopen",
-            key);
-        LOG.debug("{}", e, e);
-        try {
-          reopen(pos);
-          result = in.read();
-        } catch (EOFException eof) {
-          LOG.debug("EOF on input stream read: {}", eof, eof);
-          result = -1;
-        }
-      } 
-      if (result != -1) {
-        pos++;
-      }
-      if (statistics != null && result != -1) {
-        statistics.incrementBytesRead(1);
-      }
-      return result;
-    }
-    @Override
-    public synchronized int read(byte[] b, int off, int len)
-      throws IOException {
-      if (in == null) {
-        throw new EOFException("Cannot read closed stream");
-      }
-      int result = -1;
-      try {
-        result = in.read(b, off, len);
-      } catch (EOFException eof) {
-        throw eof;
-      } catch (IOException e) {
-        LOG.info( "Received IOException while reading '{}'," +
-                  " attempting to reopen.", key);
-        reopen(pos);
-        result = in.read(b, off, len);
-      }
-      if (result > 0) {
-        pos += result;
-      }
-      if (statistics != null && result > 0) {
-        statistics.incrementBytesRead(result);
-      }
-      return result;
-    }
 
-    @Override
-    public synchronized void close() throws IOException {
-      closeInnerStream();
-    }
-
-    /**
-     * Close the inner stream if not null. Even if an exception
-     * is raised during the close, the field is set to null
-     */
-    private void closeInnerStream() {
-      IOUtils.closeStream(in);
-      in = null;
-    }
-
-    /**
-     * Reopen a new input stream with the specified position
-     * @param pos the position to reopen a new stream
-     * @throws IOException
-     */
-    private synchronized void reopen(long pos) throws IOException {
-        LOG.debug("Reopening key '{}' for reading at position '{}", key, pos);
-        InputStream newStream = store.retrieve(key, pos);
-        updateInnerStream(newStream, pos);
-    }
-
-    /**
-     * Update inner stream with a new stream and position
-     * @param newStream new stream -must not be null
-     * @param newpos new position
-     * @throws IOException IO exception on a failure to close the existing
-     * stream.
-     */
-    private synchronized void updateInnerStream(InputStream newStream, long newpos) throws IOException {
-      Preconditions.checkNotNull(newStream, "Null newstream argument");
-      closeInnerStream();
-      in = newStream;
-      this.pos = newpos;
-    }
-
-    @Override
-    public synchronized void seek(long newpos) throws IOException {
-      if (newpos < 0) {
-        throw new EOFException(
-            FSExceptionMessages.NEGATIVE_SEEK);
-      }
-      if (pos != newpos) {
-        // the seek is attempting to move the current position
-        reopen(newpos);
-      }
-    }
-
-    @Override
-    public synchronized long getPos() throws IOException {
-      return pos;
-    }
-    @Override
-    public boolean seekToNewSource(long targetPos) throws IOException {
-      return false;
-    }
-  }
-  
-  private class NativeS3FsOutputStream extends OutputStream {
-    
-    private Configuration conf;
-    private String key;
-    private File backupFile;
-    private OutputStream backupStream;
-    private MessageDigest digest;
-    private boolean closed;
-    private LocalDirAllocator lDirAlloc;
-    
-    public NativeS3FsOutputStream(Configuration conf,
-        NativeFileSystemStore store, String key, Progressable progress,
-        int bufferSize) throws IOException {
-      this.conf = conf;
-      this.key = key;
-      this.backupFile = newBackupFile();
-      LOG.info("OutputStream for key '" + key + "' writing to tempfile '" + this.backupFile + "'");
-      try {
-        this.digest = MessageDigest.getInstance("MD5");
-        this.backupStream = new BufferedOutputStream(new DigestOutputStream(
-            new FileOutputStream(backupFile), this.digest));
-      } catch (NoSuchAlgorithmException e) {
-        LOG.warn("Cannot load MD5 digest algorithm," +
-            "skipping message integrity check.", e);
-        this.backupStream = new BufferedOutputStream(
-            new FileOutputStream(backupFile));
-      }
-    }
-
-    private File newBackupFile() throws IOException {
-      if (conf.get(S3_NATIVE_BUFFER_DIR_KEY, null) != null) {
-        lDirAlloc = new LocalDirAllocator(S3_NATIVE_BUFFER_DIR_KEY);
-      } else {
-        lDirAlloc = new LocalDirAllocator(S3_NATIVE_BUFFER_DIR_DEFAULT);
-      }
-      File result = lDirAlloc.createTmpFileForWrite("output-", LocalDirAllocator.SIZE_UNKNOWN, conf);
-      result.deleteOnExit();
-      return result;
-    }
-    
-    @Override
-    public void flush() throws IOException {
-      backupStream.flush();
-    }
-    
-    @Override
-    public synchronized void close() throws IOException {
-      if (closed) {
-        return;
-      }
-
-      backupStream.close();
-      LOG.info("OutputStream for key '{}' closed. Now beginning upload", key);
-      
-      try {
-        byte[] md5Hash = digest == null ? null : digest.digest();
-        store.storeFile(key, backupFile, md5Hash);
-      } finally {
-        if (!backupFile.delete()) {
-          LOG.warn("Could not delete temporary s3n file: " + backupFile);
-        }
-        super.close();
-        closed = true;
-      } 
-      LOG.info("OutputStream for key '{}' upload complete", key);
-    }
-
-    @Override
-    public void write(int b) throws IOException {
-      backupStream.write(b);
-    }
+  /**
+   * Message in thrown exceptions: {@value}.
+   */
+  private static final String UNSUPPORTED =
+      "The s3n:// client to Amazon S3 is no longer available:"
+          + " please migrate to the s3a:// client";
 
-    @Override
-    public void write(byte[] b, int off, int len) throws IOException {
-      backupStream.write(b, off, len);
-    }
-  }
-  
-  private URI uri;
-  private NativeFileSystemStore store;
-  private Path workingDir;
-  
   public NativeS3FileSystem() {
-    // set store in initialize()
-  }
-  
-  public NativeS3FileSystem(NativeFileSystemStore store) {
-    this.store = store;
   }
 
   /**
@@ -336,504 +72,77 @@ public class NativeS3FileSystem extends FileSystem {
     return "s3n";
   }
 
-  @Override
-  public void initialize(URI uri, Configuration conf) throws IOException {
-    super.initialize(uri, conf);
-    if (store == null) {
-      store = createDefaultStore(conf);
-    }
-    store.initialize(uri, conf);
-    setConf(conf);
-    this.uri = S3xLoginHelper.buildFSURI(uri);
-    this.workingDir =
-      new Path("/user", System.getProperty("user.name")).makeQualified(this.uri, this.getWorkingDirectory());
-  }
-  
-  private static NativeFileSystemStore createDefaultStore(Configuration conf) {
-    NativeFileSystemStore store = new Jets3tNativeFileSystemStore();
-    
-    RetryPolicy basePolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep(
-        conf.getInt(S3_NATIVE_MAX_RETRIES_KEY, S3_NATIVE_MAX_RETRIES_DEFAUL),
-        conf.getLong(S3_NATIVE_SLEEP_TIME_KEY, S3_NATIVE_SLEEP_TIME_DEFAULT),
-        TimeUnit.SECONDS);
-    Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
-      new HashMap<Class<? extends Exception>, RetryPolicy>();
-    exceptionToPolicyMap.put(IOException.class, basePolicy);
-    exceptionToPolicyMap.put(S3Exception.class, basePolicy);
-    
-    RetryPolicy methodPolicy = RetryPolicies.retryByException(
-        RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
-    Map<String, RetryPolicy> methodNameToPolicyMap =
-      new HashMap<String, RetryPolicy>();
-    methodNameToPolicyMap.put("storeFile", methodPolicy);
-    methodNameToPolicyMap.put("rename", methodPolicy);
-    
-    return (NativeFileSystemStore)
-      RetryProxy.create(NativeFileSystemStore.class, store,
-          methodNameToPolicyMap);
-  }
-  
-  private static String pathToKey(Path path) {
-    if (path.toUri().getScheme() != null && path.toUri().getPath().isEmpty()) {
-      // allow uris without trailing slash after bucket to refer to root,
-      // like s3n://mybucket
-      return "";
-    }
-    if (!path.isAbsolute()) {
-      throw new IllegalArgumentException("Path must be absolute: " + path);
-    }
-    String ret = path.toUri().getPath().substring(1); // remove initial slash
-    if (ret.endsWith("/") && (ret.indexOf("/") != ret.length() - 1)) {
-      ret = ret.substring(0, ret.length() -1);
-  }
-    return ret;
-  }
-  
-  private static Path keyToPath(String key) {
-    return new Path("/" + key);
-  }
-  
-  private Path makeAbsolute(Path path) {
-    if (path.isAbsolute()) {
-      return path;
-    }
-    return new Path(workingDir, path);
-  }
-
   /**
-   * Check that a Path belongs to this FileSystem.
-   * Unlike the superclass, this version does not look at authority,
-   * only hostnames.
-   * @param path to check
-   * @throws IllegalArgumentException if there is an FS mismatch
+   * Always fail to initialize.
+   * @throws IOException always.
    */
   @Override
-  protected void checkPath(Path path) {
-    S3xLoginHelper.checkPath(getConf(), getUri(), path, getDefaultPort());
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    super.initialize(uri, conf);
+    throw new IOException(UNSUPPORTED);
   }
 
   @Override
-  protected URI canonicalizeUri(URI rawUri) {
-    return S3xLoginHelper.canonicalizeUri(rawUri, getDefaultPort());
+  public FileStatus getFileStatus(Path f) throws IOException {
+    throw new UnsupportedOperationException(UNSUPPORTED);
   }
 
-  /** This optional operation is not yet supported. */
   @Override
-  public FSDataOutputStream append(Path f, int bufferSize,
-      Progressable progress) throws IOException {
-    throw new UnsupportedOperationException("Append is not supported "
-        + "by NativeS3FileSystem");
+  public URI getUri() {
+    throw new UnsupportedOperationException(UNSUPPORTED);
   }
-  
-  @Override
-  public FSDataOutputStream create(Path f, FsPermission permission,
-      boolean overwrite, int bufferSize, short replication, long blockSize,
-      Progressable progress) throws IOException {
 
-    if (exists(f) && !overwrite) {
-      throw new FileAlreadyExistsException("File already exists: " + f);
-    }
-    
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Creating new file '" + f + "' in S3");
-    }
-    Path absolutePath = makeAbsolute(f);
-    String key = pathToKey(absolutePath);
-    return new FSDataOutputStream(new NativeS3FsOutputStream(getConf(), store,
-        key, progress, bufferSize), statistics);
-  }
-  
   @Override
-  public boolean delete(Path f, boolean recurse) throws IOException {
-    FileStatus status;
-    try {
-      status = getFileStatus(f);
-    } catch (FileNotFoundException e) {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Delete called for '" + f +
-            "' but file does not exist, so returning false");
-      }
-      return false;
-    }
-    Path absolutePath = makeAbsolute(f);
-    String key = pathToKey(absolutePath);
-    if (status.isDirectory()) {
-      if (!recurse && listStatus(f).length > 0) {
-        throw new IOException("Can not delete " + f + " as is a not empty directory and recurse option is false");
-      }
-
-      createParent(f);
-
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Deleting directory '" + f  + "'");
-      }
-      String priorLastKey = null;
-      do {
-        PartialListing listing = store.list(key, S3_MAX_LISTING_LENGTH, priorLastKey, true);
-        for (FileMetadata file : listing.getFiles()) {
-          store.delete(file.getKey());
-        }
-        priorLastKey = listing.getPriorLastKey();
-      } while (priorLastKey != null);
-
-      try {
-        store.delete(key + FOLDER_SUFFIX);
-      } catch (FileNotFoundException e) {
-        //this is fine, we don't require a marker
-      }
-    } else {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Deleting file '" + f + "'");
-      }
-      createParent(f);
-      store.delete(key);
-    }
-    return true;
+  public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+    throw new UnsupportedOperationException(UNSUPPORTED);
   }
 
   @Override
-  public FileStatus getFileStatus(Path f) throws IOException {
-    Path absolutePath = makeAbsolute(f);
-    String key = pathToKey(absolutePath);
-    
-    if (key.length() == 0) { // root always exists
-      return newDirectory(absolutePath);
-    }
-    
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("getFileStatus retrieving metadata for key '" + key + "'");
-    }
-    FileMetadata meta = store.retrieveMetadata(key);
-    if (meta != null) {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("getFileStatus returning 'file' for key '" + key + "'");
-      }
-      return newFile(meta, absolutePath);
-    }
-    if (store.retrieveMetadata(key + FOLDER_SUFFIX) != null) {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("getFileStatus returning 'directory' for key '" + key +
-            "' as '" + key + FOLDER_SUFFIX + "' exists");
-      }
-      return newDirectory(absolutePath);
-    }
-    
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("getFileStatus listing key '" + key + "'");
-    }
-    PartialListing listing = store.list(key, 1);
-    if (listing.getFiles().length > 0 ||
-        listing.getCommonPrefixes().length > 0) {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("getFileStatus returning 'directory' for key '" + key +
-            "' as it has contents");
-      }
-      return newDirectory(absolutePath);
-    }
-    
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("getFileStatus could not find key '" + key + "'");
-    }
-    throw new FileNotFoundException("No such file or directory '" + absolutePath + "'");
+  public FSDataOutputStream create(Path f,
+      FsPermission permission,
+      boolean overwrite,
+      int bufferSize,
+      short replication,
+      long blockSize,
+      Progressable progress) throws IOException {
+    throw new UnsupportedOperationException(UNSUPPORTED);
   }
 
   @Override
-  public URI getUri() {
-    return uri;
+  public FSDataOutputStream append(Path f,
+      int bufferSize,
+      Progressable progress) throws IOException {
+    throw new UnsupportedOperationException(UNSUPPORTED);
   }
 
-  /**
-   * <p>
-   * If <code>f</code> is a file, this method will make a single call to S3.
-   * If <code>f</code> is a directory, this method will make a maximum of
-   * (<i>n</i> / 1000) + 2 calls to S3, where <i>n</i> is the total number of
-   * files and directories contained directly in <code>f</code>.
-   * </p>
-   */
   @Override
-  public FileStatus[] listStatus(Path f) throws IOException {
-
-    Path absolutePath = makeAbsolute(f);
-    String key = pathToKey(absolutePath);
-    
-    if (key.length() > 0) {
-      FileMetadata meta = store.retrieveMetadata(key);
-      if (meta != null) {
-        return new FileStatus[] { newFile(meta, absolutePath) };
-      }
-    }
-    
-    URI pathUri = absolutePath.toUri();
-    Set<FileStatus> status = new TreeSet<FileStatus>();
-    String priorLastKey = null;
-    do {
-      PartialListing listing = store.list(key, S3_MAX_LISTING_LENGTH, priorLastKey, false);
-      for (FileMetadata fileMetadata : listing.getFiles()) {
-        Path subpath = keyToPath(fileMetadata.getKey());
-        String relativePath = pathUri.relativize(subpath.toUri()).getPath();
-
-        if (fileMetadata.getKey().equals(key + "/")) {
-          // this is just the directory we have been asked to list
-        }
-        else if (relativePath.endsWith(FOLDER_SUFFIX)) {
-          status.add(newDirectory(new Path(
-              absolutePath,
-              relativePath.substring(0, relativePath.indexOf(FOLDER_SUFFIX)))));
-        }
-        else {
-          status.add(newFile(fileMetadata, subpath));
-        }
-      }
-      for (String commonPrefix : listing.getCommonPrefixes()) {
-        Path subpath = keyToPath(commonPrefix);
-        String relativePath = pathUri.relativize(subpath.toUri()).getPath();
-        // sometimes the common prefix includes the base dir (HADOOP-13830).
-        // avoid that problem by detecting it and keeping it out
-        // of the list
-        if (!relativePath.isEmpty()) {
-          status.add(newDirectory(new Path(absolutePath, relativePath)));
-        }
-      }
-      priorLastKey = listing.getPriorLastKey();
-    } while (priorLastKey != null);
-    
-    if (status.isEmpty() &&
-        key.length() > 0 &&
-        store.retrieveMetadata(key + FOLDER_SUFFIX) == null) {
-      throw new FileNotFoundException("File " + f + " does not exist.");
-    }
-    
-    return status.toArray(new FileStatus[status.size()]);
-  }
-  
-  private FileStatus newFile(FileMetadata meta, Path path) {
-    return new FileStatus(meta.getLength(), false, 1, getDefaultBlockSize(),
-        meta.getLastModified(), path.makeQualified(this.getUri(), this.getWorkingDirectory()));
-  }
-  
-  private FileStatus newDirectory(Path path) {
-    return new FileStatus(0, true, 1, 0, 0, path.makeQualified(this.getUri(), this.getWorkingDirectory()));
+  public boolean rename(Path src, Path dst) throws IOException {
+    throw new UnsupportedOperationException(UNSUPPORTED);
   }
 
   @Override
-  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
-    Path absolutePath = makeAbsolute(f);
-    List<Path> paths = new ArrayList<Path>();
-    do {
-      paths.add(0, absolutePath);
-      absolutePath = absolutePath.getParent();
-    } while (absolutePath != null);
-    
-    boolean result = true;
-    for (Path path : paths) {
-      result &= mkdir(path);
-    }
-    return result;
-  }
-  
-  private boolean mkdir(Path f) throws IOException {
-    try {
-      FileStatus fileStatus = getFileStatus(f);
-      if (fileStatus.isFile()) {
-        throw new FileAlreadyExistsException(String.format(
-            "Can't make directory for path '%s' since it is a file.", f));
-
-      }
-    } catch (FileNotFoundException e) {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Making dir '" + f + "' in S3");
-      }
-      String key = pathToKey(f) + FOLDER_SUFFIX;
-      store.storeEmptyFile(key);    
-    }
-    return true;
+  public boolean delete(Path f, boolean recursive) throws IOException {
+    throw new UnsupportedOperationException(UNSUPPORTED);
   }
 
   @Override
-  public FSDataInputStream open(Path f, int bufferSize) throws IOException {
-    FileStatus fs = getFileStatus(f); // will throw if the file doesn't exist
-    if (fs.isDirectory()) {
-      throw new FileNotFoundException("'" + f + "' is a directory");
-    }
-    LOG.info("Opening '" + f + "' for reading");
-    Path absolutePath = makeAbsolute(f);
-    String key = pathToKey(absolutePath);
-    return new FSDataInputStream(new BufferedFSInputStream(
-        new NativeS3FsInputStream(store, statistics, store.retrieve(key), key), bufferSize));
-  }
-  
-  // rename() and delete() use this method to ensure that the parent directory
-  // of the source does not vanish.
-  private void createParent(Path path) throws IOException {
-    Path parent = path.getParent();
-    if (parent != null) {
-      String key = pathToKey(makeAbsolute(parent));
-      if (key.length() > 0) {
-          store.storeEmptyFile(key + FOLDER_SUFFIX);
-      }
-    }
+  public FileStatus[] listStatus(Path f)
+      throws IOException {
+    throw new UnsupportedOperationException(UNSUPPORTED);
   }
-  
-    
-  @Override
-  public boolean rename(Path src, Path dst) throws IOException {
-
-    String srcKey = pathToKey(makeAbsolute(src));
-    final String debugPreamble = "Renaming '" + src + "' to '" + dst + "' - ";
-
-    if (srcKey.length() == 0) {
-      // Cannot rename root of file system
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(debugPreamble +
-                  "returning false as cannot rename the root of a filesystem");
-      }
-      return false;
-    }
-
-    //get status of source
-    boolean srcIsFile;
-    try {
-      srcIsFile = getFileStatus(src).isFile();
-    } catch (FileNotFoundException e) {
-      //bail out fast if the source does not exist
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(debugPreamble + "returning false as src does not exist");
-      }
-      return false;
-    }
-    // Figure out the final destination
-    String dstKey = pathToKey(makeAbsolute(dst));
-
-    try {
-      boolean dstIsFile = getFileStatus(dst).isFile();
-      if (dstIsFile) {
-        //destination is a file.
-        //you can't copy a file or a directory onto an existing file
-        //except for the special case of dest==src, which is a no-op
-        if(LOG.isDebugEnabled()) {
-          LOG.debug(debugPreamble +
-              "returning without rename as dst is an already existing file");
-        }
-        //exit, returning true iff the rename is onto self
-        return srcKey.equals(dstKey);
-      } else {
-        //destination exists and is a directory
-        if(LOG.isDebugEnabled()) {
-          LOG.debug(debugPreamble + "using dst as output directory");
-        }
-        //destination goes under the dst path, with the name of the
-        //source entry
-        dstKey = pathToKey(makeAbsolute(new Path(dst, src.getName())));
-      }
-    } catch (FileNotFoundException e) {
-      //destination does not exist => the source file or directory
-      //is copied over with the name of the destination
-      if(LOG.isDebugEnabled()) {
-        LOG.debug(debugPreamble + "using dst as output destination");
-      }
-      try {
-        if (getFileStatus(dst.getParent()).isFile()) {
-          if(LOG.isDebugEnabled()) {
-            LOG.debug(debugPreamble +
-                "returning false as dst parent exists and is a file");
-          }
-          return false;
-        }
-      } catch (FileNotFoundException ex) {
-        if(LOG.isDebugEnabled()) {
-          LOG.debug(debugPreamble +
-              "returning false as dst parent does not exist");
-        }
-        return false;
-      }
-    }
 
-    //rename to self behavior follows Posix rules and is different
-    //for directories and files -the return code is driven by src type
-    if (srcKey.equals(dstKey)) {
-      //fully resolved destination key matches source: fail
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(debugPreamble + "renamingToSelf; returning true");
-      }
-      return true;
-    }
-    if (srcIsFile) {
-      //source is a file; COPY then DELETE
-      if(LOG.isDebugEnabled()) {
-        LOG.debug(debugPreamble +
-            "src is file, so doing copy then delete in S3");
-      }
-      store.copy(srcKey, dstKey);
-      store.delete(srcKey);
-    } else {
-      //src is a directory
-      if(LOG.isDebugEnabled()) {
-        LOG.debug(debugPreamble + "src is directory, so copying contents");
-      }
-      //Verify dest is not a child of the parent
-      if (dstKey.startsWith(srcKey + "/")) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(
-            debugPreamble + "cannot rename a directory to a subdirectory of self");
-        }
-        return false;
-      }
-      //create the subdir under the destination
-      store.storeEmptyFile(dstKey + FOLDER_SUFFIX);
-
-      List<String> keysToDelete = new ArrayList<String>();
-      String priorLastKey = null;
-      do {
-        PartialListing listing = store.list(srcKey, S3_MAX_LISTING_LENGTH, priorLastKey, true);
-        for (FileMetadata file : listing.getFiles()) {
-          keysToDelete.add(file.getKey());
-          store.copy(file.getKey(), dstKey + file.getKey().substring(srcKey.length()));
-        }
-        priorLastKey = listing.getPriorLastKey();
-      } while (priorLastKey != null);
-
-      if(LOG.isDebugEnabled()) {
-        LOG.debug(debugPreamble +
-            "all files in src copied, now removing src files");
-      }
-      for (String key: keysToDelete) {
-        store.delete(key);
-      }
-
-      try {
-        store.delete(srcKey + FOLDER_SUFFIX);
-      } catch (FileNotFoundException e) {
-        //this is fine, we don't require a marker
-      }
-      if(LOG.isDebugEnabled()) {
-        LOG.debug(debugPreamble + "done");
-      }
-    }
-
-    return true;
-  }
-  
   @Override
-  public long getDefaultBlockSize() {
-    return getConf().getLong("fs.s3n.block.size", 64 * 1024 * 1024);
+  public void setWorkingDirectory(Path new_dir) {
+    throw new UnsupportedOperationException(UNSUPPORTED);
   }
 
-  /**
-   * Set the working directory to the given directory.
-   */
-  @Override
-  public void setWorkingDirectory(Path newDir) {
-    workingDir = newDir;
-  }
-  
   @Override
   public Path getWorkingDirectory() {
-    return workingDir;
+    throw new UnsupportedOperationException(UNSUPPORTED);
   }
 
   @Override
-  public String getCanonicalServiceName() {
-    // Does not support Token
-    return null;
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    throw new UnsupportedOperationException(UNSUPPORTED);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/PartialListing.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/PartialListing.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/PartialListing.java
deleted file mode 100644
index 8290092..0000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/PartialListing.java
+++ /dev/null
@@ -1,64 +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.s3native;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * <p>
- * Holds information on a directory listing for a
- * {@link NativeFileSystemStore}.
- * This includes the {@link FileMetadata files} and directories
- * (their names) contained in a directory.
- * </p>
- * <p>
- * This listing may be returned in chunks, so a <code>priorLastKey</code>
- * is provided so that the next chunk may be requested.
- * </p>
- * @see NativeFileSystemStore#list(String, int, String)
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-class PartialListing {
-  
-  private final String priorLastKey;
-  private final FileMetadata[] files;
-  private final String[] commonPrefixes;
-  
-  public PartialListing(String priorLastKey, FileMetadata[] files,
-      String[] commonPrefixes) {
-    this.priorLastKey = priorLastKey;
-    this.files = files;
-    this.commonPrefixes = commonPrefixes;
-  }
-
-  public FileMetadata[] getFiles() {
-    return files;
-  }
-
-  public String[] getCommonPrefixes() {
-    return commonPrefixes;
-  }
-
-  public String getPriorLastKey() {
-    return priorLastKey;
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3Credentials.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3Credentials.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3Credentials.java
deleted file mode 100644
index 713b149..0000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3Credentials.java
+++ /dev/null
@@ -1,100 +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.s3native;
-
-import java.io.IOException;
-import java.net.URI;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-
-import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_AWS_ACCESS_KEY_ID;
-import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_AWS_SECRET_ACCESS_KEY;
-
-/**
- * <p>
- * Extracts AWS credentials from the filesystem URI or configuration.
- * </p>
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class S3Credentials {
-
-  private String accessKey;
-  private String secretAccessKey;
-
-  /**
-   * @param uri bucket URI optionally containing username and password.
-   * @param conf configuration
-   * @throws IllegalArgumentException if credentials for S3 cannot be
-   * determined.
-   * @throws IOException if credential providers are misconfigured and we have
-   *                     to talk to them.
-   */
-  public void initialize(URI uri, Configuration conf) throws IOException {
-    Preconditions.checkArgument(uri.getHost() != null,
-        "Invalid hostname in URI " + uri);
-
-    String userInfo = uri.getUserInfo();
-    if (userInfo != null) {
-      int index = userInfo.indexOf(':');
-      if (index != -1) {
-        accessKey = userInfo.substring(0, index);
-        secretAccessKey = userInfo.substring(index + 1);
-      } else {
-        accessKey = userInfo;
-      }
-    }
-
-    if (accessKey == null) {
-      accessKey = conf.getTrimmed(S3_NATIVE_AWS_ACCESS_KEY_ID);
-    }
-    if (secretAccessKey == null) {
-      final char[] pass = conf.getPassword(S3_NATIVE_AWS_SECRET_ACCESS_KEY);
-      if (pass != null) {
-        secretAccessKey = (new String(pass)).trim();
-      }
-    }
-
-    final String scheme = uri.getScheme();
-    Preconditions.checkArgument(!(accessKey == null && secretAccessKey == null),
-        "AWS Access Key ID and Secret Access Key must be specified as the " +
-            "username or password (respectively) of a " + scheme + " URL, or " +
-            "by setting the " + S3_NATIVE_AWS_ACCESS_KEY_ID + " or " +
-            S3_NATIVE_AWS_SECRET_ACCESS_KEY + " properties (respectively).");
-    Preconditions.checkArgument(accessKey != null,
-        "AWS Access Key ID must be specified as the username of a " + scheme +
-            " URL, or by setting the " + S3_NATIVE_AWS_ACCESS_KEY_ID +
-            " property.");
-    Preconditions.checkArgument(secretAccessKey != null,
-        "AWS Secret Access Key must be specified as the password of a " + scheme
-            + " URL, or by setting the " + S3_NATIVE_AWS_SECRET_ACCESS_KEY +
-            " property.");
-  }
-
-  public String getAccessKey() {
-    return accessKey;
-  }
-
-  public String getSecretAccessKey() {
-    return secretAccessKey;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3Exception.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3Exception.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3Exception.java
deleted file mode 100644
index 9258fd7..0000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3Exception.java
+++ /dev/null
@@ -1,39 +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.s3native;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Thrown if there is a problem communicating with Amazon S3.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class S3Exception extends IOException {
-
-  private static final long serialVersionUID = 1L;
-
-  public S3Exception(Throwable t) {
-    super(t);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3NativeFileSystemConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3NativeFileSystemConfigKeys.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3NativeFileSystemConfigKeys.java
deleted file mode 100644
index 7c8b345..0000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3NativeFileSystemConfigKeys.java
+++ /dev/null
@@ -1,66 +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.s3native;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configuration.DeprecationDelta;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-
-/** 
- * This class contains constants for configuration keys used
- * in the s3 file system. 
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class S3NativeFileSystemConfigKeys extends CommonConfigurationKeys {
-  public static final String  S3_NATIVE_BLOCK_SIZE_KEY = "s3native.blocksize";
-  public static final long    S3_NATIVE_BLOCK_SIZE_DEFAULT = 64*1024*1024;
-  public static final String  S3_NATIVE_REPLICATION_KEY = "s3native.replication";
-  public static final short   S3_NATIVE_REPLICATION_DEFAULT = 1;
-  public static final String  S3_NATIVE_STREAM_BUFFER_SIZE_KEY = 
-                                                    "s3native.stream-buffer-size";
-  public static final int     S3_NATIVE_STREAM_BUFFER_SIZE_DEFAULT = 4096;
-  public static final String  S3_NATIVE_BYTES_PER_CHECKSUM_KEY = 
-                                                    "s3native.bytes-per-checksum";
-  public static final int     S3_NATIVE_BYTES_PER_CHECKSUM_DEFAULT = 512;
-  public static final String  S3_NATIVE_CLIENT_WRITE_PACKET_SIZE_KEY =
-                                                    "s3native.client-write-packet-size";
-  public static final int     S3_NATIVE_CLIENT_WRITE_PACKET_SIZE_DEFAULT = 64*1024;
-  static final String  S3_NATIVE_BUFFER_DIR_KEY = "fs.s3n.buffer.dir";
-  static final String  S3_NATIVE_BUFFER_DIR_DEFAULT = "${hadoop.tmp.dir}/s3n";
-  static final String  S3_NATIVE_MAX_RETRIES_KEY = "fs.s3n.maxRetries";
-  static final int     S3_NATIVE_MAX_RETRIES_DEFAUL = 4;
-  static final String  S3_NATIVE_SLEEP_TIME_KEY = "fs.s3n.sleepTimeSeconds";
-  static final int     S3_NATIVE_SLEEP_TIME_DEFAULT = 10;
-  static final String  S3_NATIVE_AWS_ACCESS_KEY_ID = "fs.s3n.awsAccessKeyId";
-  static final String  S3_NATIVE_AWS_SECRET_ACCESS_KEY =
-      "fs.s3n.awsSecretAccessKey";
-
-  static void addDeprecatedConfigKeys() {
-    Configuration.addDeprecations(new DeprecationDelta[]{
-        new DeprecationDelta("fs.s3.buffer.dir", S3_NATIVE_BUFFER_DIR_KEY),
-        new DeprecationDelta("fs.s3.maxRetries", S3_NATIVE_MAX_RETRIES_KEY),
-        new DeprecationDelta("fs.s3.sleepTimeSeconds", S3_NATIVE_SLEEP_TIME_KEY)
-    });
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/package.html
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/package.html b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/package.html
index 4d3bde9..eb2c471 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/package.html
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/package.html
@@ -23,8 +23,11 @@
 A distributed implementation of {@link
 org.apache.hadoop.fs.FileSystem} for reading and writing files on
 <a href="http://aws.amazon.com/s3">Amazon S3</a>.
-This implementation stores files on S3 in their native form for interoperability
+This implementation stored files on S3 in their native form for interoperability
 with other S3 tools.
+
+It has been replaced by the S3A client.
+
 </p>
 
 </body>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md
new file mode 100644
index 0000000..719c5e5
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/encryption.md
@@ -0,0 +1,427 @@
+
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# Working with Encrypted S3 Data
+
+<!-- MACRO{toc|fromDepth=0|toDepth=2} -->
+
+
+## <a name="introduction"></a> Introduction
+
+The S3A filesystem client supports Amazon S3's Server Side Encryption
+for at-rest data encryption.
+You should to read up on the [AWS documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/serv-side-encryption.html)
+for S3 Server Side Encryption for up to date information on the encryption mechansims.
+
+
+
+When configuring an encryption method in the `core-site.xml`, this will apply cluster wide.
+Any new file written will be encrypted with this encryption configuration.
+When the S3A client reads a file, S3 will attempt to decrypt it using the mechanism
+and keys with which the file was encrypted.
+
+* It is **NOT** advised to mix and match encryption types in a bucket
+* It is much simpler and safer to encrypt with just one type and key per bucket.
+* You can use AWS bucket policies to mandate encryption rules for a bucket.
+* You can use S3A per-bucket configuration to ensure that S3A clients use encryption
+policies consistent with the mandated rules.
+* Changing the encryption options on the client does not change how existing
+files were encrypted, except when the files are renamed.
+* For all mechanisms other than SSE-C, clients do not need any configuration
+options set in order to read encrypted data: it is all automatically handled
+in S3 itself.
+
+## <a name="encryption_types"></a>How data is encrypted
+
+AWS S3 supports server-side encryption inside the storage system itself.
+When an S3 client uploading data requests data to be encrypted, then an encryption key is used
+to encrypt the data as it saved to S3. It remains encrypted on S3 until deleted:
+clients cannot change the encryption attributes of an object once uploaded.
+
+The Amazon AWS SDK also offers client-side encryption, in which all the encoding
+and decoding of data is performed on the client. This is *not* supported by
+the S3A client.
+
+The server-side "SSE" encryption is performed with symmetric AES256 encryption;
+S3 offers different mechanisms for actually defining the key to use.
+
+
+There are thrre key management mechanisms, which in order of simplicity of use,
+are:
+
+* SSE-S3: an AES256 key is generated in S3, and saved alongside the data.
+* SSE-KMS: an AES256 key is generated in S3, and encrypted with a secret key provided
+by Amazon's Key Management Service, a key referenced by name in the uploading client.
+* SSE-C : the client specifies an actual base64 encoded AES-256 key to be used
+to encrypt and decrypt the data.
+
+
+## <a name="sse-s3"></a> SSE-S3 Amazon S3-Managed Encryption Keys
+
+In SSE-S3, all keys and secrets are managed inside S3. This is the simplest encryption mechanism.
+There is no extra cost for storing data with this option.
+
+
+### Enabling SSE-S3
+
+To write S3-SSE encrypted files, the value of
+`fs.s3a.server-side-encryption-algorithm` must be set to that of
+the encryption mechanism used in `core-site`; currently only `AES256` is supported.
+
+```xml
+<property>
+  <name>fs.s3a.server-side-encryption-algorithm</name>
+  <value>AES256</value>
+</property>
+```
+
+Once set, all new data will be stored encrypted. There is no need to set this property when downloading data — the data will be automatically decrypted when read using
+the Amazon S3-managed key.
+
+To learn more, refer to
+[Protecting Data Using Server-Side Encryption with Amazon S3-Managed Encryption Keys (SSE-S3) in AWS documentation](http://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html).
+
+
+### <a name="sse-kms"></a> SSE-KMS: Amazon S3-KMS Managed Encryption Keys
+
+
+Amazon offers a pay-per-use key management service, [AWS KMS](https://aws.amazon.com/documentation/kms/).
+This service can be used to encrypt data on S3 by defining "customer master keys", CMKs,
+which can be centrally managed and assigned to specific roles and IAM accounts.
+
+The AWS KMS [can be used encrypt data on S3uploaded data](http://docs.aws.amazon.com/kms/latest/developerguide/services-s3.html).
+
+> The AWS KMS service is **not** related to the Key Management Service built into Hadoop (*Hadoop KMS*). The *Hadoop KMS* primarily focuses on
+  managing keys for *HDFS Transparent Encryption*. Similarly, HDFS encryption is unrelated to S3 data encryption.
+
+When uploading data encrypted with SSE-KMS, the sequence is as follows.
+
+1. The S3A client must declare a specific CMK in the property `fs.s3a.server-side-encryption.key`, or leave
+it blank to use the default configured for that region.
+
+1. The S3A client uploads all the data as normal, now including encryption information.
+
+1. The S3 service encrypts the data with a symmetric key unique to the new object.
+
+1. The S3 service retrieves the chosen CMK key from the KMS service, and, if the user has
+the right to use it, uses it to encrypt the object-specific key.
+
+
+When downloading SSE-KMS encrypte data, the sequence is as follows
+
+1. The S3A client issues an HTTP GET request to read the data.
+1. S3 sees that the data was encrypted with SSE-KMS, and looks up the specific key in the KMS service
+1. If and only if the requesting user has been granted permission to use the CMS key does
+the KMS service provide S3 with the key.
+1. As a result, S3 will only decode the data if the user has been granted access to the key.
+
+
+KMS keys can be managed by an organization's administrators in AWS, including
+having access permissions assigned and removed from specific users, groups, and IAM roles.
+Only those "principals" with granted rights to a key may access it,
+hence only they may encrypt data with the key, *and decrypt data encrypted with it*.
+This allows KMS to be used to provide a cryptographically secure access control mechanism for data stores on S3.
+
+
+Each KMS server is region specific, and accordingly, so is each CMK configured.
+A CMK defined in one region cannot be used with an S3 bucket in a different region.
+
+
+Notes
+
+* Callers are charged for every use of a key, both for encrypting the data in uploads
+  and for decrypting it when reading it back.
+* Random-access IO on files may result in multiple GET requests of an object during a read
+sequence (especially for columnar data), so may require more than one key retrieval to process a single file,
+* The KMS service is throttled: too many requests may cause requests to fail.
+* As well as incurring charges, heavy I/O *may* reach IO limits for a customer. If those limits are reached,
+they can be increased through the AWS console.
+
+
+### Enabling SSE-KMS
+
+To enable SSE-KMS, the property `fs.s3a.server-side-encryption-algorithm` must be set to `SSE-KMS` in `core-site`:
+
+```xml
+<property>
+  <name>fs.s3a.server-side-encryption-algorithm</name>
+  <value>SSE-KMS</value>
+</property>
+```
+
+The ID of the specific key used to encrypt the data should also be set in the property `fs.s3a.server-side-encryption.key`:
+
+```xml
+<property>
+  <name>fs.s3a.server-side-encryption.key</name>
+  <value>arn:aws:kms:us-west-2:360379543683:key/071a86ff-8881-4ba0-9230-95af6d01ca01</value>
+</property>
+```
+
+Organizations may define a default key in the Amazon KMS; if a default key is set,
+then it will be used whenever SSE-KMS encryption is chosen and the value of `fs.s3a.server-side-encryption.key` is empty.
+
+### the S3A `fs.s3a.encryption.key` key only affects created files
+
+With SSE-KMS, the S3A client option `fs.s3a.server-side-encryption.key` sets the
+key to be used when new files are created. When reading files, this key,
+and indeed the value of `fs.s3a.server-side-encryption-algorithme` is ignored:
+S3 will attempt to retrieve the key and decrypt the file based on the create-time settings.
+
+This means that
+
+* There's no need to configure any client simply reading data.
+* It is possible for a client to read data encrypted with one KMS key, and
+write it with another.
+
+
+## <a name="sse-c"></a> SSE-C: Server side encryption with a client-supplied key.
+
+In SSE-C, the client supplies the secret key needed to read and write data.
+Every client trying to read or write data must be configured with the same
+secret key.
+
+
+SSE-C integration with Hadoop is still stabilizing; issues related to it are still surfacing.
+It is already clear that SSE-C with a common key <b>must</b> be used exclusively within
+a bucket if it is to be used at all. This is the only way to ensure that path and
+directory listings do not fail with "Bad Request" errors.
+
+### Enabling SSE-C
+
+To use SSE-C, the configuration option `fs.s3a.server-side-encryption-algorithm`
+must be set to `SSE-C`, and a base-64 encoding of the key placed in
+`fs.s3a.server-side-encryption.key`.
+
+```xml
+<property>
+  <name>fs.s3a.server-side-encryption-algorithm</name>
+  <value>SSE-C</value>
+</property>
+
+<property>
+  <name>fs.s3a.server-side-encryption.key</name>
+  <value>SGVscCwgSSdtIHRyYXBwZWQgaW5zaWRlIGEgYmFzZS02NC1jb2RlYyE=</value>
+</property>
+```
+
+All clients must share this same key.
+
+### The `fs.s3a.encryption.key` value is used to read and write data
+
+With SSE-C, the S3A client option `fs.s3a.server-side-encryption.key` sets the
+key to be used for both reading *and* writing data.
+
+When reading any file written with SSE-C, the same key must be set
+in the property `fs.s3a.server-side-encryption.key`.
+
+This is unlike SSE-S3 and SSE-KMS, where the information needed to
+decode data is kept in AWS infrastructure.
+
+
+### SSE-C Warning
+
+You need to fully understand how SSE-C works in the S3
+environment before using this encryption type.  Please refer to the Server Side
+Encryption documentation available from AWS.  SSE-C is only recommended for
+advanced users with advanced encryption use cases.  Failure to properly manage
+encryption keys can cause data loss.  Currently, the AWS S3 API(and thus S3A)
+only supports one encryption key and cannot support decrypting objects during
+moves under a previous key to a new destination.  It is **NOT** advised to use
+multiple encryption keys in a bucket, and is recommended to use one key per
+bucket and to not change this key.  This is due to when a request is made to S3,
+the actual encryption key must be provided to decrypt the object and access the
+metadata.  Since only one encryption key can be provided at a time, S3A will not
+pass the correct encryption key to decrypt the data.
+
+
+## <a name="best_practises"></a> Encryption best practises
+
+
+### <a name="bucket_policy"></a> Mandate encryption through policies
+
+Because it is up to the clients to enable encryption on new objects, all clients
+must be correctly configured in order to guarantee that data is encrypted.
+
+
+To mandate that all data uploaded to a bucket is encrypted,
+you can set a [bucket policy](https://aws.amazon.com/blogs/security/how-to-prevent-uploads-of-unencrypted-objects-to-amazon-s3/)
+declaring that clients must provide encryption information with all data uploaded.
+
+
+* Mandating an encryption mechanism on newly uploaded data does not encrypt existing data; existing data will retain whatever encryption (if any) applied at the time of creation*
+
+Here is a policy to mandate `SSE-S3/AES265` encryption on all data uploaded to a bucket. This covers uploads as well as the copy operations which take place when file/directory rename operations are mimicked.
+
+
+```json
+{
+  "Version": "2012-10-17",
+  "Id": "EncryptionPolicy",
+  "Statement": [
+    {
+      "Sid": "RequireEncryptionHeaderOnPut",
+      "Effect": "Deny",
+      "Principal": "*",
+      "Action": [
+        "s3:PutObject"
+      ],
+      "Resource": "arn:aws:s3:::BUCKET/*",
+      "Condition": {
+        "Null": {
+          "s3:x-amz-server-side-encryption": true
+        }
+      }
+    },
+    {
+      "Sid": "RequireAESEncryptionOnPut",
+      "Effect": "Deny",
+      "Principal": "*",
+      "Action": [
+        "s3:PutObject"
+      ],
+      "Resource": "arn:aws:s3:::BUCKET/*",
+      "Condition": {
+        "StringNotEquals": {
+          "s3:x-amz-server-side-encryption": "AES256"
+        }
+      }
+    }
+  ]
+}
+```
+
+To use SSE-KMS, a different restriction must be defined:
+
+
+```json
+{
+  "Version": "2012-10-17",
+  "Id": "EncryptionPolicy",
+  "Statement": [
+    {
+      "Sid": "RequireEncryptionHeaderOnPut",
+      "Effect": "Deny",
+      "Principal": "*",
+      "Action": [
+        "s3:PutObject"
+      ],
+      "Resource": "arn:aws:s3:::BUCKET/*",
+      "Condition": {
+        "Null": {
+          "s3:x-amz-server-side-encryption": true
+        }
+      }
+    },
+    {
+      "Sid": "RequireKMSEncryptionOnPut",
+      "Effect": "Deny",
+      "Principal": "*",
+      "Action": [
+        "s3:PutObject"
+      ],
+      "Resource": "arn:aws:s3:::BUCKET/*",
+      "Condition": {
+        "StringNotEquals": {
+          "s3:x-amz-server-side-encryption": "SSE-KMS"
+        }
+      }
+    }
+  ]
+}
+```
+
+To use one of these policies:
+
+1. Replace `BUCKET` with the specific name of the bucket being secured.
+1. Locate the bucket in the AWS console [S3 section](https://console.aws.amazon.com/s3/home).
+1. Select the "Permissions" tab.
+1. Select the "Bucket Policy" tab in the permissions section.
+1. Paste the edited policy into the form.
+1. Save the policy.
+
+### <a name="per_bucket_config"></a> Use S3a per-bucket configuration to control encryption settings
+
+In an organisation which has embraced S3 encryption, different buckets inevitably have
+different encryption policies, such as different keys for SSE-KMS encryption.
+In particular, as different keys need to be named for different regions, unless
+you rely on the administrator-managed "default" key for each S3 region, you
+will need unique keys.
+
+S3A's per-bucket configuration enables this.
+
+
+Here, for example, are settings for a bucket in London, `london-stats`:
+
+
+```xml
+<property>
+  <name>fs.s3a.bucket.london-stats.server-side-encryption-algorithm</name>
+  <value>AES256</value>
+</property>
+```
+
+This requests SSE-S; if matched with a bucket policy then all data will
+be encrypted as it is uploaded.
+
+
+A different bucket can use a different policy
+(here SSE-KMS) and, when necessary, declare a key.
+
+Here is an example bucket in S3 Ireland, which uses SSE-KMS and
+a KMS key hosted in the AWS-KMS service in the same region.
+
+
+```xml
+<property>
+  <name>fs.s3a.bucket.ireland-dev.server-side-encryption-algorithm</name>
+  <value>SSE-KMS</value>
+</property>
+
+<property>
+  <name>fs.s3a.bucket.ireland-dev.server-side-encryption.key</name>
+  <value>arn:aws:kms:eu-west-1:98067faff834c:key/071a86ff-8881-4ba0-9230-95af6d01ca01</value>
+</property>
+
+```
+
+Again the approprate bucket policy can be used to guarantee that all callers
+will use SSE-KMS; they can even mandata the name of the key used to encrypt
+the data, so guaranteeing that access to thee data can be read by everyone
+granted access to that key, and nobody without access to it.
+
+
+###<a name="changing-encryption"></a> Use rename() to encrypt files with new keys
+
+The encryption of an object is set when it is uploaded. If you want to encrypt
+an unencrypted file, or change the SEE-KMS key of a file, the only way to do
+so is by copying the object.
+
+How can you do that from Hadoop? With `rename()`.
+
+The S3A client mimics a real filesystem's' rename operation by copying all the
+source files to the destination paths, then deleting the old ones.
+If you do a rename()
+
+Note: this does not work for SSE-C, because you cannot set a different key
+for reading as for writing, and you must supply that key for reading. There
+you need to copy one bucket to a different bucket, one with a different key.
+Use `distCp`for this, with per-bucket encryption policies.
+
+
+## <a name="Troubleshooting"></a> Troubleshooting Encryption
+
+The [troubleshooting](./troubleshooting_s3a.html) document covers
+stack traces which may surface when working with encrypted data.


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


[03/50] [abbrv] hadoop git commit: YARN-6612. Update fair scheduler policies to be aware of resource types. (Contributed by Daniel Templeton via Yufei Gu)

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/09b476e6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/TestDominantResourceFairnessPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/TestDominantResourceFairnessPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/TestDominantResourceFairnessPolicy.java
index 3719e2a..097558f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/TestDominantResourceFairnessPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/TestDominantResourceFairnessPolicy.java
@@ -23,15 +23,22 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 import java.util.Comparator;
+import java.util.Map;
+import org.apache.curator.shaded.com.google.common.base.Joiner;
+import org.apache.hadoop.conf.Configuration;
 
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceType;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FakeSchedulable;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.Schedulable;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy.DominantResourceFairnessComparator;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
@@ -39,10 +46,15 @@ import org.junit.Test;
  * container before sched2
  */
 public class TestDominantResourceFairnessPolicy {
+  @BeforeClass
+  public static void setup() {
+    addResources("test");
+  }
 
   private Comparator<Schedulable> createComparator(int clusterMem,
       int clusterCpu) {
-    DominantResourceFairnessPolicy policy = new DominantResourceFairnessPolicy();
+    DominantResourceFairnessPolicy policy =
+        new DominantResourceFairnessPolicy();
     FSContext fsContext = mock(FSContext.class);
     when(fsContext.getClusterResource()).
         thenReturn(Resources.createResource(clusterMem, clusterCpu));
@@ -51,23 +63,23 @@ public class TestDominantResourceFairnessPolicy {
   }
   
   private Schedulable createSchedulable(int memUsage, int cpuUsage) {
-    return createSchedulable(memUsage, cpuUsage, ResourceWeights.NEUTRAL, 0, 0);
+    return createSchedulable(memUsage, cpuUsage, 1.0f, 0, 0);
   }
   
   private Schedulable createSchedulable(int memUsage, int cpuUsage,
       int minMemShare, int minCpuShare) {
-    return createSchedulable(memUsage, cpuUsage, ResourceWeights.NEUTRAL,
+    return createSchedulable(memUsage, cpuUsage, 1.0f,
         minMemShare, minCpuShare);
   }
   
   private Schedulable createSchedulable(int memUsage, int cpuUsage,
-      ResourceWeights weights) {
+      float weights) {
     return createSchedulable(memUsage, cpuUsage, weights, 0, 0);
   }
 
   
   private Schedulable createSchedulable(int memUsage, int cpuUsage,
-      ResourceWeights weights, int minMemShare, int minCpuShare) {
+      float weights, int minMemShare, int minCpuShare) {
     Resource usage = BuilderUtils.newResource(memUsage, cpuUsage);
     Resource minShare = BuilderUtils.newResource(minMemShare, minCpuShare);
     return new FakeSchedulable(minShare,
@@ -77,94 +89,260 @@ public class TestDominantResourceFairnessPolicy {
   
   @Test
   public void testSameDominantResource() {
-    assertTrue(createComparator(8000, 4).compare(
-        createSchedulable(1000, 1),
-        createSchedulable(2000, 1)) < 0);
+    Comparator c = createComparator(8000, 4);
+    Schedulable s1 = createSchedulable(1000, 1);
+    Schedulable s2 = createSchedulable(2000, 1);
+
+    assertTrue("Comparison didn't return a value less than 0",
+        c.compare(s1, s2) < 0);
   }
   
   @Test
   public void testDifferentDominantResource() {
-    assertTrue(createComparator(8000, 8).compare(
-        createSchedulable(4000, 3),
-        createSchedulable(2000, 5)) < 0);
+    Comparator c = createComparator(8000, 8);
+    Schedulable s1 = createSchedulable(4000, 3);
+    Schedulable s2 = createSchedulable(2000, 5);
+
+    assertTrue("Comparison didn't return a value less than 0",
+        c.compare(s1, s2) < 0);
   }
   
   @Test
   public void testOneIsNeedy() {
-    assertTrue(createComparator(8000, 8).compare(
-        createSchedulable(2000, 5, 0, 6),
-        createSchedulable(4000, 3, 0, 0)) < 0);
+    Comparator c = createComparator(8000, 8);
+    Schedulable s1 = createSchedulable(2000, 5, 0, 6);
+    Schedulable s2 = createSchedulable(4000, 3, 0, 0);
+
+    assertTrue("Comparison didn't return a value less than 0",
+        c.compare(s1, s2) < 0);
   }
   
   @Test
   public void testBothAreNeedy() {
-    assertTrue(createComparator(8000, 100).compare(
-        // dominant share is 2000/8000
-        createSchedulable(2000, 5),
-        // dominant share is 4000/8000
-        createSchedulable(4000, 3)) < 0);
-    assertTrue(createComparator(8000, 100).compare(
-        // dominant min share is 2/3
-        createSchedulable(2000, 5, 3000, 6),
-        // dominant min share is 4/5
-        createSchedulable(4000, 3, 5000, 4)) < 0);
+    Comparator c = createComparator(8000, 100);
+    // dominant share is 2000/8000
+    Schedulable s1 = createSchedulable(2000, 5);
+    // dominant share is 4000/8000
+    Schedulable s2 = createSchedulable(4000, 3);
+
+    assertTrue("Comparison didn't return a value less than 0",
+        c.compare(s1, s2) < 0);
+
+    // dominant min share is 2/3
+    s1 = createSchedulable(2000, 5, 3000, 6);
+    // dominant min share is 4/5
+    s2 = createSchedulable(4000, 3, 5000, 4);
+
+    assertTrue("Comparison didn't return a value less than 0",
+        c.compare(s1, s2) < 0);
   }
   
   @Test
   public void testEvenWeightsSameDominantResource() {
     assertTrue(createComparator(8000, 8).compare(
-        createSchedulable(3000, 1, new ResourceWeights(2.0f)),
+        createSchedulable(3000, 1, 2.0f),
         createSchedulable(2000, 1)) < 0);
     assertTrue(createComparator(8000, 8).compare(
-        createSchedulable(1000, 3, new ResourceWeights(2.0f)),
+        createSchedulable(1000, 3, 2.0f),
         createSchedulable(1000, 2)) < 0);
   }
   
   @Test
   public void testEvenWeightsDifferentDominantResource() {
     assertTrue(createComparator(8000, 8).compare(
-        createSchedulable(1000, 3, new ResourceWeights(2.0f)),
+        createSchedulable(1000, 3, 2.0f),
         createSchedulable(2000, 1)) < 0);
     assertTrue(createComparator(8000, 8).compare(
-        createSchedulable(3000, 1, new ResourceWeights(2.0f)),
+        createSchedulable(3000, 1, 2.0f),
         createSchedulable(1000, 2)) < 0);
   }
   
   @Test
-  public void testUnevenWeightsSameDominantResource() {
-    assertTrue(createComparator(8000, 8).compare(
-        createSchedulable(3000, 1, new ResourceWeights(2.0f, 1.0f)),
-        createSchedulable(2000, 1)) < 0);
-    assertTrue(createComparator(8000, 8).compare(
-        createSchedulable(1000, 3, new ResourceWeights(1.0f, 2.0f)),
-        createSchedulable(1000, 2)) < 0);
+  public void testSortShares() {
+    float[][] ratios1 = {{0.3f, 2.0f}, {0.2f, 1.0f}, {0.4f, 0.1f}};
+    float[][] ratios2 = {{0.2f, 9.0f}, {0.3f, 2.0f}, {0.25f, 0.1f}};
+    float[][] expected1 = {{0.4f, 0.1f}, {0.3f, 2.0f}, {0.2f, 1.0f}};
+    float[][] expected2 = {{0.3f, 2.0f}, {0.25f, 0.1f}, {0.2f, 9.0f}};
+    DominantResourceFairnessComparator comparator =
+        new DominantResourceFairnessComparator();
+
+    comparator.sortRatios(ratios1, ratios2);
+
+    for (int i = 0; i < ratios1.length; i++) {
+      Assert.assertArrayEquals("The shares array was not sorted into the "
+          + "expected order: incorrect inner array encountered",
+          expected1[i], ratios1[i], 0.00001f);
+      Assert.assertArrayEquals("The shares array was not sorted into the "
+          + "expected order: incorrect inner array encountered",
+          expected2[i], ratios2[i], 0.00001f);
+    }
   }
-  
+
   @Test
-  public void testUnevenWeightsDifferentDominantResource() {
-    assertTrue(createComparator(8000, 8).compare(
-        createSchedulable(1000, 3, new ResourceWeights(1.0f, 2.0f)),
-        createSchedulable(2000, 1)) < 0);
-    assertTrue(createComparator(8000, 8).compare(
-        createSchedulable(3000, 1, new ResourceWeights(2.0f, 1.0f)),
-        createSchedulable(1000, 2)) < 0);
+  public void testCalculateClusterAndFairRatios() {
+    Map<String, Integer> index = ResourceUtils.getResourceTypeIndex();
+    Resource used = Resources.createResource(10, 5);
+    Resource capacity = Resources.createResource(100, 10);
+    float[][] shares = new float[3][2];
+    DominantResourceFairnessComparator comparator =
+        new DominantResourceFairnessComparator();
+
+    used.setResourceValue("test", 2L);
+    capacity.setResourceValue("test", 5L);
+
+    int dominant = comparator.calculateClusterAndFairRatios(used, capacity,
+        shares, 1.0f);
+
+    assertEquals("Calculated usage ratio for memory (10MB out of 100MB) is "
+        + "incorrect", 0.1,
+        shares[index.get(ResourceInformation.MEMORY_MB.getName())][0], .00001);
+    assertEquals("Calculated usage ratio for vcores (5 out of 10) is "
+        + "incorrect", 0.5,
+        shares[index.get(ResourceInformation.VCORES.getName())][0], .00001);
+    assertEquals("Calculated usage ratio for test resource (2 out of 5) is "
+        + "incorrect", 0.4, shares[index.get("test")][0], .00001);
+    assertEquals("The wrong dominant resource index was returned",
+        index.get(ResourceInformation.VCORES.getName()).intValue(),
+        dominant);
   }
-  
+
   @Test
-  public void testCalculateShares() {
+  public void testCalculateMinShareRatios() {
+    Map<String, Integer> index = ResourceUtils.getResourceTypeIndex();
     Resource used = Resources.createResource(10, 5);
-    Resource capacity = Resources.createResource(100, 10);
-    ResourceType[] resourceOrder = new ResourceType[2];
-    ResourceWeights shares = new ResourceWeights();
-    DominantResourceFairnessPolicy.DominantResourceFairnessComparator comparator =
-        new DominantResourceFairnessPolicy.DominantResourceFairnessComparator();
-    comparator.calculateShares(used, capacity, shares, resourceOrder,
-        ResourceWeights.NEUTRAL);
-    
-    assertEquals(.1, shares.getWeight(ResourceType.MEMORY), .00001);
-    assertEquals(.5, shares.getWeight(ResourceType.CPU), .00001);
-    assertEquals(ResourceType.CPU, resourceOrder[0]);
-    assertEquals(ResourceType.MEMORY, resourceOrder[1]);
+    Resource minShares = Resources.createResource(5, 10);
+    float[][] ratios = new float[3][3];
+    DominantResourceFairnessComparator comparator =
+        new DominantResourceFairnessComparator();
+
+    used.setResourceValue("test", 2L);
+    minShares.setResourceValue("test", 0L);
+
+    comparator.calculateMinShareRatios(used, minShares, ratios);
+
+    assertEquals("Calculated min share ratio for memory (10MB out of 5MB) is "
+        + "incorrect", 2.0,
+        ratios[index.get(ResourceInformation.MEMORY_MB.getName())][2], .00001f);
+    assertEquals("Calculated min share ratio for vcores (5 out of 10) is "
+        + "incorrect", 0.5,
+        ratios[index.get(ResourceInformation.VCORES.getName())][2], .00001f);
+    assertEquals("Calculated min share ratio for test resource (0 out of 5) is "
+        + "incorrect", Float.POSITIVE_INFINITY, ratios[index.get("test")][2],
+        0.00001f);
+  }
+
+  @Test
+  public void testCompareShares() {
+    float[][] ratios1 = {
+        {0.4f, 0.1f, 2.0f},
+        {0.3f, 2.0f, 0.1f},
+        {0.2f, 1.0f, 9.0f}
+    };
+    float[][] ratios2 = {
+        {0.3f, 2.0f, 1.0f},
+        {0.2f, 0.1f, 0.5f},
+        {0.2f, 1.0f, 2.0f}
+    };
+    float[][] ratios3 = {
+        {0.3f, 2.0f, 1.0f},
+        {0.2f, 0.1f, 2.0f},
+        {0.1f, 2.0f, 1.0f}
+    };
+    DominantResourceFairnessComparator comparator =
+        new DominantResourceFairnessComparator();
+
+    int ret = comparator.compareRatios(ratios1, ratios2, 0);
+
+    assertEquals("Expected the first array to be larger because the first "
+        + "usage ratio element is larger", 1, ret);
+
+    ret = comparator.compareRatios(ratios2, ratios1, 0);
+
+    assertEquals("Expected the first array to be smaller because the first "
+        + "usage ratio element is smaller", -1, ret);
+
+    ret = comparator.compareRatios(ratios1, ratios1, 0);
+
+    assertEquals("Expected the arrays to be equal, since they're the same "
+        + "array", 0, ret);
+
+    ret = comparator.compareRatios(ratios2, ratios2, 0);
+
+    assertEquals("Expected the arrays to be equal, since they're the same "
+        + "array", 0, ret);
+
+    ret = comparator.compareRatios(ratios3, ratios3, 0);
+
+    assertEquals("Expected the arrays to be equal, since they're the same "
+        + "array", 0, ret);
+
+    ret = comparator.compareRatios(ratios2, ratios3, 0);
+
+    assertEquals("Expected the first array to be larger because the last "
+        + "usage ratio element is larger, and all other elements are equal",
+        1, ret);
+
+    ret = comparator.compareRatios(ratios1, ratios2, 1);
+
+    assertEquals("Expected the first array to be smaller because the first "
+        + "fair share ratio element is smaller", -1, ret);
+
+    ret = comparator.compareRatios(ratios2, ratios1, 1);
+
+    assertEquals("Expected the first array to be larger because the first "
+        + "fair share ratio element is larger", 1, ret);
+
+    ret = comparator.compareRatios(ratios1, ratios1, 1);
+
+    assertEquals("Expected the arrays to be equal, since they're the same "
+        + "array", 0, ret);
+
+    ret = comparator.compareRatios(ratios2, ratios2, 1);
+
+    assertEquals("Expected the arrays to be equal, since they're the same "
+        + "array", 0, ret);
+
+    ret = comparator.compareRatios(ratios3, ratios3, 1);
+
+    assertEquals("Expected the arrays to be equal, since they're the same "
+        + "array", 0, ret);
+
+    ret = comparator.compareRatios(ratios2, ratios3, 1);
+
+    assertEquals("Expected the first array to be smaller because the last "
+        + "usage ratio element is smaller, and all other elements are equal",
+        -1, ret);
+
+    ret = comparator.compareRatios(ratios1, ratios2, 2);
+
+    assertEquals("Expected the first array to be larger because the first "
+        + "min share ratio element is larger", 1, ret);
+
+    ret = comparator.compareRatios(ratios2, ratios1, 2);
+
+    assertEquals("Expected the first array to be smaller because the first "
+        + "min share ratio element is smaller", -1, ret);
+
+    ret = comparator.compareRatios(ratios1, ratios1, 2);
+
+    assertEquals("Expected the arrays to be equal, since they're the same "
+        + "array", 0, ret);
+
+    ret = comparator.compareRatios(ratios2, ratios2, 2);
+
+    assertEquals("Expected the arrays to be equal, since they're the same "
+        + "array", 0, ret);
+
+    ret = comparator.compareRatios(ratios3, ratios3, 2);
+
+    assertEquals("Expected the arrays to be equal, since they're the same "
+        + "array", 0, ret);
+
+    ret = comparator.compareRatios(ratios2, ratios3, 2);
+
+    assertEquals("Expected the first array to be smaller because the second "
+        + "min share ratio element is smaller, and all the first elements are "
+        + "equal", -1, ret);
   }
 
   @Test
@@ -183,4 +361,12 @@ public class TestDominantResourceFairnessPolicy {
     assertTrue(createComparator(8000, 6)
         .compare(schedulable1, schedulable2) < 0);
   }
+
+  private static void addResources(String... resources) {
+    Configuration conf = new Configuration();
+
+    // Add a third resource to the allowed set
+    conf.set(YarnConfiguration.RESOURCE_TYPES, Joiner.on(',').join(resources));
+    ResourceUtils.resetResourceTypes(conf);
+  }
 }


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


[08/50] [abbrv] hadoop git commit: HADOOP-14738 Remove S3N and obsolete bits of S3A; rework docs. Contributed by Steve Loughran.

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index ffae1e9..75c638f 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -15,40 +15,128 @@
 
 # Hadoop-AWS module: Integration with Amazon Web Services
 
-<!-- MACRO{toc|fromDepth=0|toDepth=5} -->
+<!-- MACRO{toc|fromDepth=0|toDepth=2} -->
 
-## Overview
+**NOTE:  Hadoop's `s3:` and `s3n:` connectors have been removed.
+Please use `s3a:` as the connector to data hosted in S3 with Apache Hadoop.**
 
-The `hadoop-aws` module provides support for AWS integration. The generated
-JAR file, `hadoop-aws.jar` also declares a transitive dependency on all
-external artifacts which are needed for this support —enabling downstream
+**Consult the [s3n documentation](./s3n.html) for migration instructions.**
+
+
+See also:
+
+* [Encryption](./encryption.html)
+* [S3Guard](./s3guard.html)
+* [Troubleshooting](./troubleshooting_s3a.html)
+* [Testing](./testing.html)
+
+##<a name="overview"></a> Overview
+
+Apache Hadoop's `hadoop-aws` module provides support for AWS integration.
 applications to easily use this support.
 
-To make it part of Apache Hadoop's default classpath, simply make sure that
-HADOOP_OPTIONAL_TOOLS in hadoop-env.sh has 'hadoop-aws' in the list.
+To include the S3A client in Apache Hadoop's default classpath:
 
-### Features
+1. Make sure that`HADOOP_OPTIONAL_TOOLS` in `hadoop-env.sh` includes `hadoop-aws`
+in its list of optional modules to add in the classpath.
 
-**NOTE: `s3:` has been phased out; `s3n:`, while
-distributed should now be considered deprecated.
-Please use `s3a:` as the connector to data hosted in S3.**
+1. For client side interaction, you can declare that relevant JARs must be loaded
+in your `~/.hadooprc` file:
 
-1. The second-generation, `s3n:` filesystem, making it easy to share
-data between hadoop and other applications via the S3 object store.
-1. The third generation, `s3a:` filesystem. Designed to be a switch in
-replacement for `s3n:`, this filesystem binding supports larger files and promises
-higher performance.
+        hadoop_add_to_classpath_tools hadoop-aws
 
-The specifics of using these filesystems are documented in this section.
+The settings in this file does not propagate to deployed applications, but it will
+work for local clients such as the `hadoop fs` command.
 
 
-See also:
+## <a name="introduction"></a> Introducing the Hadoop S3A client.
+
+Hadoop's "S3A" client offers high-performance IO against Amazon S3 object store
+and compatible implementations.
+
+* Directly reads and writes S3 objects.
+* Compatible with standard S3 clients.
+* Compatible with files created by the older `s3n://` client and Amazon EMR's `s3://` client.
+* Supports partitioned uploads for many-GB objects.
+* Offers a high-performance random IO mode for working with columnar data such
+as Apache ORC and Apache Parquet files.
+* Uses Amazon's Java S3 SDK with support for latest S3 features and authentication
+schemes.
+* Supports authentication via: environment variables, Hadoop configuration
+properties, the Hadoop key management store and IAM roles.
+* Supports per-bucket configuration.
+* With [S3Guard](./s3guard.html), adds high performance and consistent metadata/
+directory read operations. This delivers consistency as well as speed.
+* Supports S3 "Server Side Encryption" for both reading and writing:
+ SSE-S3, SSE-KMS and SSE-C
+* Instrumented with Hadoop metrics.
+* Actively maintained by the open source community.
+
+
+### Other S3 Connectors
+
+There other Hadoop connectors to S3. Only S3A is actively maintained by
+the Hadoop project itself.
+
+1. Apache's Hadoop's original `s3://` client. This is no longer included in Hadoop.
+1. Amazon EMR's `s3://` client. This is from the Amazon EMR team, who actively
+maintain it.
+1. Apache's Hadoop's [`s3n:` filesystem client](./s3n.html).
+   This connectore is no longer available: users must migrate to the newer `s3a:` client.
+
+
+##<a name="getting_started"></a> Getting Started
+
+S3A depends upon two JARs, alongside `hadoop-common` and its dependencies.
+
+* `hadoop-aws` JAR.
+* `aws-java-sdk-bundle` JAR.
+
+The versions of `hadoop-common` and `hadoop-aws` must be identical.
+
+To import the libraries into a Maven build, add `hadoop-aws` JAR to the
+build dependencies; it will pull in a compatible aws-sdk JAR.
+
+The `hadoop-aws` JAR *does not* declare any dependencies other than that
+dependencies unique to it, the AWS SDK JAR. This is simplify excluding/tuning
+Hadoop dependency JARs in downstream applications. The `hadoop-client` or
+`hadoop-common` dependency must be declared
+
+
+```xml
+<properties>
+ <!-- Your exact Hadoop version here-->
+  <hadoop.version>3.0.0</hadoop.version>
+</properties>
+
+<dependencies>
+  <dependency>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-client</artifactId>
+    <version>${hadoop.version}</version>
+  </dependency>
+  <dependency>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-aws</artifactId>
+    <version>${hadoop.version}</version>
+  </dependency>
+</dependencies>
+```
+
+## <a name="warning"></a> Warnings
+
+Amazon S3 is an example of "an object store". In order to achieve scalability
+and especially high availability, S3 has —as many other cloud object stores have
+done— relaxed some of the constraints which classic "POSIX" filesystems promise.
 
-* [Testing](testing.html)
-* [Troubleshooting S3a](troubleshooting_s3a.html)
-* [S3Guard](s3guard.html)
+The [S3Guard](./s3guard.html) feature attempts to address some of these, but
+it cannot do so completely. Do read these warnings and consider how
+they apply.
 
-### Warning #1: Object Stores are not filesystems
+For further discussion on these topics, please consult
+[The Hadoop FileSystem API Definition](../../../hadoop-project-dist/hadoop-common/filesystem/index.html).
+
+### Warning #1: S3 Consistency model
 
 Amazon S3 is an example of "an object store". In order to achieve scalability
 and especially high availability, S3 has —as many other cloud object stores have
@@ -65,23 +153,49 @@ recursive file-by-file operations. They take time at least proportional to
 the number of files, during which time partial updates may be visible. If
 the operations are interrupted, the filesystem is left in an intermediate state.
 
-### Warning #2: Object stores don't track modification times of directories
 
-Features of Hadoop relying on this can have unexpected behaviour. E.g. the
-AggregatedLogDeletionService of YARN will not remove the appropriate logfiles.
+### Warning #2: Directories are mimiced
+
+The S3A clients mimics directories by:
+
+1. Creating a stub entry after a `mkdirs` call, deleting it when a file
+is added anywhere underneath
+1. When listing a directory, searching for all objects whose path starts with
+the directory path, and returning them as the listing.
+1. When renaming a directory, taking such a listing and asking S3 to copying the
+individual objects to new objects with the destination filenames.
+1. When deleting a directory, taking such a listing and deleting the entries in
+batches.
+1. When renaming or deleting directories, taking such a listing and working
+on the individual files.
+
+
+Here are some of the consequences:
+
+* Directories may lack modification times.
+Parts of Hadoop relying on this can have unexpected behaviour. E.g. the
+`AggregatedLogDeletionService` of YARN will not remove the appropriate logfiles.
+* Directory listing can be slow. Use `listFiles(path, recursive)` for high
+performance recursive listings whenever possible.
+* The time to rename a directory is proportional to the number of files
+underneath it (directory or indirectly) and the size of the files. (The copyis
+executed inside the S3 storage, so the time is independent of the bandwidth
+from client to S3).
+* Directory renames are not atomic: they can fail partway through, and callers
+cannot safely rely on atomic renames as part of a commit algorithm.
+* Directory deletion is not atomic and can fail partway through.
+* It is possible to create files under files if the caller tries hard.
+
 
-For further discussion on these topics, please consult
-[The Hadoop FileSystem API Definition](../../../hadoop-project-dist/hadoop-common/filesystem/index.html).
 
 ### Warning #3: Object stores have differerent authorization models
 
 The object authorization model of S3 is much different from the file
-authorization model of HDFS and traditional file systems.  It is not feasible to
-persist file ownership and permissions in S3, so S3A reports stub information
-from APIs that would query this metadata:
+authorization model of HDFS and traditional file systems.
+The S3A client simply reports stub information from APIs that would query this metadata:
 
 * File owner is reported as the current user.
-* File group also is reported as the current user.  Prior to Apache Hadoop
+* File group also is reported as the current user. Prior to Apache Hadoop
 2.8.0, file group was reported as empty (no group associated), which is a
 potential incompatibility problem for scripts that perform positional parsing of
 shell output and other clients that expect to find a well-defined group.
@@ -93,10 +207,7 @@ Users authenticate to an S3 bucket using AWS credentials.  It's possible that
 object ACLs have been defined to enforce authorization at the S3 side, but this
 happens entirely within the S3 service, not within the S3A implementation.
 
-For further discussion on these topics, please consult
-[The Hadoop FileSystem API Definition](../../../hadoop-project-dist/hadoop-common/filesystem/index.html).
-
-### Warning #4: Your AWS credentials are valuable
+### Warning #4: Your AWS credentials are very, very valuable
 
 Your AWS credentials not only pay for services, they offer read and write
 access to the data. Anyone with the credentials can not only read your datasets
@@ -107,250 +218,100 @@ Do not inadvertently share these credentials through means such as
 1. Checking in to SCM any configuration files containing the secrets.
 1. Logging them to a console, as they invariably end up being seen.
 1. Defining filesystem URIs with the credentials in the URL, such as
-`s3a://AK0010:secret@landsat/`. They will end up in logs and error messages.
+`s3a://AK0010:secret@landsat-pds/`. They will end up in logs and error messages.
 1. Including the secrets in bug reports.
 
 If you do any of these: change your credentials immediately!
 
-### Warning #5: The S3 client provided by Amazon EMR are not from the Apache Software foundation, and are only supported by Amazon.
-
-Specifically: on Amazon EMR, s3a is not supported, and amazon recommend
-a different filesystem implementation. If you are using Amazon EMR, follow
-these instructions —and be aware that all issues related to S3 integration
-in EMR can only be addressed by Amazon themselves: please raise your issues
-with them.
-
-## S3N
-
-S3N was the first S3 Filesystem client which used "native" S3 objects, hence
-the schema `s3n://`.
-
-### Features
-
-* Directly reads and writes S3 objects.
-* Compatible with standard S3 clients.
-* Supports partitioned uploads for many-GB objects.
-* Available across all Hadoop 2.x releases.
-
-The S3N filesystem client, while widely used, is no longer undergoing
-active maintenance except for emergency security issues. There are
-known bugs, especially: it reads to end of a stream when closing a read;
-this can make `seek()` slow on large files. The reason there has been no
-attempt to fix this is that every upgrade of the Jets3t library, while
-fixing some problems, has unintentionally introduced new ones in either the changed
-Hadoop code, or somewhere in the Jets3t/Httpclient code base.
-The number of defects remained constant, they merely moved around.
-
-By freezing the Jets3t jar version and avoiding changes to the code,
-we reduce the risk of making things worse.
+### Warning #5: The S3A client cannot be used on Amazon EMR
 
-The S3A filesystem client can read all files created by S3N. Accordingly
-it should be used wherever possible.
+On Amazon EMR `s3a://` URLs are not supported; Amazon provide
+their own filesystem client, `s3://`.
+If you are using Amazon EMR, follow their instructions for use —and be aware
+that all issues related to S3 integration in EMR can only be addressed by Amazon
+themselves: please raise your issues with them.
 
+Equally importantly: much of this document does not apply to the EMR `s3://` client.
+Pleae consult
+[the EMR storage documentation](http://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-plan-file-systems.html)
+instead.
 
-### Dependencies
+## <a name="authenticating"></a> Authenticating with S3
 
-* `jets3t` jar
-* `commons-codec` jar
-* `commons-logging` jar
-* `httpclient` jar
-* `httpcore` jar
-* `java-xmlbuilder` jar
+Except when interacting with public S3 buckets, the S3A client
+needs the credentials needed to interact with buckets.
 
-
-### Authentication properties
-
-    <property>
-      <name>fs.s3n.awsAccessKeyId</name>
-      <description>AWS access key ID</description>
-    </property>
-
-    <property>
-      <name>fs.s3n.awsSecretAccessKey</name>
-      <description>AWS secret key</description>
-    </property>
-
-### Other properties
-
-    <property>
-      <name>fs.s3n.buffer.dir</name>
-      <value>${hadoop.tmp.dir}/s3</value>
-      <description>Determines where on the local filesystem the s3n: filesystem
-      should store files before sending them to S3
-      (or after retrieving them from S3).
-      </description>
-    </property>
-
-    <property>
-      <name>fs.s3n.maxRetries</name>
-      <value>4</value>
-      <description>The maximum number of retries for reading or writing files to
-        S3, before we signal failure to the application.
-      </description>
-    </property>
-
-    <property>
-      <name>fs.s3n.sleepTimeSeconds</name>
-      <value>10</value>
-      <description>The number of seconds to sleep between each S3 retry.
-      </description>
-    </property>
-
-    <property>
-      <name>fs.s3n.block.size</name>
-      <value>67108864</value>
-      <description>Block size to use when reading files using the native S3
-      filesystem (s3n: URIs).</description>
-    </property>
-
-    <property>
-      <name>fs.s3n.multipart.uploads.enabled</name>
-      <value>false</value>
-      <description>Setting this property to true enables multiple uploads to
-      native S3 filesystem. When uploading a file, it is split into blocks
-      if the size is larger than fs.s3n.multipart.uploads.block.size.
-      </description>
-    </property>
-
-    <property>
-      <name>fs.s3n.multipart.uploads.block.size</name>
-      <value>67108864</value>
-      <description>The block size for multipart uploads to native S3 filesystem.
-      Default size is 64MB.
-      </description>
-    </property>
-
-    <property>
-      <name>fs.s3n.multipart.copy.block.size</name>
-      <value>5368709120</value>
-      <description>The block size for multipart copy in native S3 filesystem.
-      Default size is 5GB.
-      </description>
-    </property>
-
-    <property>
-      <name>fs.s3n.server-side-encryption-algorithm</name>
-      <value></value>
-      <description>Specify a server-side encryption algorithm for S3.
-      Unset by default, and the only other currently allowable value is AES256.
-      </description>
-    </property>
-
-## S3A
-
-
-The S3A filesystem client, prefix `s3a://`, is the S3 client undergoing
-active development and maintenance.
-While this means that there is a bit of instability
-of configuration options and behavior, it also means
-that the code is getting better in terms of reliability, performance,
-monitoring and other features.
-
-### Features
-
-* Directly reads and writes S3 objects.
-* Compatible with standard S3 clients.
-* Can read data created with S3N.
-* Can write data back that is readable by S3N. (Note: excluding encryption).
-* Supports partitioned uploads for many-GB objects.
-* Instrumented with Hadoop metrics.
-* Performance optimized operations, including `seek()` and `readFully()`.
-* Uses Amazon's Java S3 SDK with support for latest S3 features and authentication
-schemes.
-* Supports authentication via: environment variables, Hadoop configuration
-properties, the Hadoop key management store and IAM roles.
-* Supports S3 "Server Side Encryption" for both reading and writing.
-* Supports proxies
-* Test suites includes distcp and suites in downstream projects.
-* Available since Hadoop 2.6; considered production ready in Hadoop 2.7.
-* Actively maintained.
-* Supports per-bucket configuration.
-
-S3A is now the recommended client for working with S3 objects. It is also the
-one where patches for functionality and performance are very welcome.
-
-### Dependencies
-
-* `hadoop-aws` jar.
-* `aws-java-sdk-s3` jar.
-* `aws-java-sdk-core` jar.
-* `aws-java-sdk-kms` jar.
-* `joda-time` jar; use version 2.8.1 or later.
-* `httpclient` jar.
-* Jackson `jackson-core`, `jackson-annotations`, `jackson-databind` jars.
-
-### S3A Authentication methods
-
-S3A supports multiple authentication mechanisms, and can be configured as to
-which mechanisms to use, and the order to use them. Custom implementations
+The client supports multiple authentication mechanisms and can be configured as to
+which mechanisms to use, and their order of use. Custom implementations
 of `com.amazonaws.auth.AWSCredentialsProvider` may also be used.
 
 ### Authentication properties
 
-    <property>
-      <name>fs.s3a.access.key</name>
-      <description>AWS access key ID.
-       Omit for IAM role-based or provider-based authentication.</description>
-    </property>
-
-    <property>
-      <name>fs.s3a.secret.key</name>
-      <description>AWS secret key.
-       Omit for IAM role-based or provider-based authentication.</description>
-    </property>
+```xml
+<property>
+  <name>fs.s3a.access.key</name>
+  <description>AWS access key ID.
+   Omit for IAM role-based or provider-based authentication.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.aws.credentials.provider</name>
-      <description>
-        Comma-separated class names of credential provider classes which implement
-        com.amazonaws.auth.AWSCredentialsProvider.
-
-        These are loaded and queried in sequence for a valid set of credentials.
-        Each listed class must implement one of the following means of
-        construction, which are attempted in order:
-        1. a public constructor accepting java.net.URI and
-            org.apache.hadoop.conf.Configuration,
-        2. a public static method named getInstance that accepts no
-           arguments and returns an instance of
-           com.amazonaws.auth.AWSCredentialsProvider, or
-        3. a public default constructor.
-
-        Specifying org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider allows
-        anonymous access to a publicly accessible S3 bucket without any credentials.
-        Please note that allowing anonymous access to an S3 bucket compromises
-        security and therefore is unsuitable for most use cases. It can be useful
-        for accessing public data sets without requiring AWS credentials.
-
-        If unspecified, then the default list of credential provider classes,
-        queried in sequence, is:
-        1. org.apache.hadoop.fs.s3a.BasicAWSCredentialsProvider: supports
-            static configuration of AWS access key ID and secret access key.
-            See also fs.s3a.access.key and fs.s3a.secret.key.
-        2. com.amazonaws.auth.EnvironmentVariableCredentialsProvider: supports
-            configuration of AWS access key ID and secret access key in
-            environment variables named AWS_ACCESS_KEY_ID and
-            AWS_SECRET_ACCESS_KEY, as documented in the AWS SDK.
-        3. com.amazonaws.auth.InstanceProfileCredentialsProvider: supports use
-            of instance profile credentials if running in an EC2 VM.
-      </description>
-    </property>
+<property>
+  <name>fs.s3a.secret.key</name>
+  <description>AWS secret key.
+   Omit for IAM role-based or provider-based authentication.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.session.token</name>
-      <description>
-        Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider
-        as one of the providers.
-      </description>
-    </property>
+<property>
+  <name>fs.s3a.aws.credentials.provider</name>
+  <description>
+    Comma-separated class names of credential provider classes which implement
+    com.amazonaws.auth.AWSCredentialsProvider.
+
+    These are loaded and queried in sequence for a valid set of credentials.
+    Each listed class must implement one of the following means of
+    construction, which are attempted in order:
+    1. a public constructor accepting java.net.URI and
+        org.apache.hadoop.conf.Configuration,
+    2. a public static method named getInstance that accepts no
+       arguments and returns an instance of
+       com.amazonaws.auth.AWSCredentialsProvider, or
+    3. a public default constructor.
+
+    Specifying org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider allows
+    anonymous access to a publicly accessible S3 bucket without any credentials.
+    Please note that allowing anonymous access to an S3 bucket compromises
+    security and therefore is unsuitable for most use cases. It can be useful
+    for accessing public data sets without requiring AWS credentials.
+
+    If unspecified, then the default list of credential provider classes,
+    queried in sequence, is:
+    1. org.apache.hadoop.fs.s3a.BasicAWSCredentialsProvider: supports
+        static configuration of AWS access key ID and secret access key.
+        See also fs.s3a.access.key and fs.s3a.secret.key.
+    2. com.amazonaws.auth.EnvironmentVariableCredentialsProvider: supports
+        configuration of AWS access key ID and secret access key in
+        environment variables named AWS_ACCESS_KEY_ID and
+        AWS_SECRET_ACCESS_KEY, as documented in the AWS SDK.
+    3. com.amazonaws.auth.InstanceProfileCredentialsProvider: supports use
+        of instance profile credentials if running in an EC2 VM.
+  </description>
+</property>
 
+<property>
+  <name>fs.s3a.session.token</name>
+  <description>
+    Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider
+    as one of the providers.
+  </description>
+</property>
+```
 
-#### Authenticating via environment variables
+### <a name="auth_env_vars"></a> Authenticating via the AWS Environment Variables
 
 S3A supports configuration via [the standard AWS environment variables](http://docs.aws.amazon.com/cli/latest/userguide/cli-chap-getting-started.html#cli-environment).
 
 The core environment variables are for the access key and associated secret:
 
-```
+```bash
 export AWS_ACCESS_KEY_ID=my.aws.key
 export AWS_SECRET_ACCESS_KEY=my.secret.key
 ```
@@ -359,7 +320,7 @@ If the environment variable `AWS_SESSION_TOKEN` is set, session authentication
 using "Temporary Security Credentials" is enabled; the Key ID and secret key
 must be set to the credentials for that specific sesssion.
 
-```
+```bash
 export AWS_SESSION_TOKEN=SECRET-SESSION-TOKEN
 export AWS_ACCESS_KEY_ID=SESSION-ACCESS-KEY
 export AWS_SECRET_ACCESS_KEY=SESSION-SECRET-KEY
@@ -369,14 +330,13 @@ These environment variables can be used to set the authentication credentials
 instead of properties in the Hadoop configuration.
 
 *Important:*
-These environment variables are not propagated from client to server when
+These environment variables are generally not propagated from client to server when
 YARN applications are launched. That is: having the AWS environment variables
 set when an application is launched will not permit the launched application
 to access S3 resources. The environment variables must (somehow) be set
 on the hosts/processes where the work is executed.
 
-
-#### Changing Authentication Providers
+### <a name="auth_providers"></a> Changing Authentication Providers
 
 The standard way to authenticate is with an access key and secret key using the
 properties in the configuration file.
@@ -421,7 +381,7 @@ set up in the authentication chain:
 | `com.amazonaws.auth.EnvironmentVariableCredentialsProvider`| AWS Environment Variables |
 
 
-*EC2 Metadata Credentials with `InstanceProfileCredentialsProvider`*
+### <a name="auth_iam"></a> EC2 IAM Metadata Authentication with `InstanceProfileCredentialsProvider`
 
 Applications running in EC2 may associate an IAM role with the VM and query the
 [EC2 Instance Metadata Service](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/ec2-instance-metadata.html)
@@ -429,7 +389,7 @@ for credentials to access S3.  Within the AWS SDK, this functionality is
 provided by `InstanceProfileCredentialsProvider`, which internally enforces a
 singleton instance in order to prevent throttling problem.
 
-*Session Credentials with `TemporaryAWSCredentialsProvider`*
+### <a name="auth_session"></a> Using Session Credentials with `TemporaryAWSCredentialsProvider`
 
 [Temporary Security Credentials](http://docs.aws.amazon.com/IAM/latest/UserGuide/id_credentials_temp.html)
 can be obtained from the Amazon Security Token Service; these
@@ -470,7 +430,7 @@ The lifetime of session credentials are fixed when the credentials
 are issued; once they expire the application will no longer be able to
 authenticate to AWS.
 
-*Anonymous Login with `AnonymousAWSCredentialsProvider`*
+### <a name="auth_anon"></a> Anonymous Login with `AnonymousAWSCredentialsProvider`
 
 Specifying `org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider` allows
 anonymous access to a publicly accessible S3 bucket without any credentials.
@@ -511,10 +471,12 @@ supports the secret key in `fs.s3a.access.key` and token in `fs.s3a.secret.key`
 values. It does not support authentication with logins credentials declared
 in the URLs.
 
-    <property>
-      <name>fs.s3a.aws.credentials.provider</name>
-      <value>org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider</value>
-    </property>
+```xml
+<property>
+  <name>fs.s3a.aws.credentials.provider</name>
+  <value>org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider</value>
+</property>
+```
 
 Apart from its lack of support of user:password details being included in filesystem
 URLs (a dangerous practise that is strongly discouraged), this provider acts
@@ -522,17 +484,18 @@ exactly at the basic authenticator used in the default authentication chain.
 
 This means that the default S3A authentication chain can be defined as
 
-    <property>
-      <name>fs.s3a.aws.credentials.provider</name>
-      <value>
-      org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider,
-      com.amazonaws.auth.EnvironmentVariableCredentialsProvider,
-      com.amazonaws.auth.InstanceProfileCredentialsProvider
-      </value>
-    </property>
-
+```xml
+<property>
+  <name>fs.s3a.aws.credentials.provider</name>
+  <value>
+  org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider,
+  com.amazonaws.auth.EnvironmentVariableCredentialsProvider,
+  com.amazonaws.auth.InstanceProfileCredentialsProvider
+  </value>
+</property>
+```
 
-#### Protecting the AWS Credentials
+### <a name="auth_security"></a> Protecting the AWS Credentials
 
 To protect the access/secret keys from prying eyes, it is recommended that you
 use either IAM role-based authentication (such as EC2 instance profile) or
@@ -541,7 +504,7 @@ through configuration. The following describes using the latter for AWS
 credentials in the S3A FileSystem.
 
 
-##### Storing secrets with Hadoop Credential Providers
+## <a name="credential_providers"></a>Storing secrets with Hadoop Credential Providers
 
 The Hadoop Credential Provider Framework allows secure "Credential Providers"
 to keep secrets outside Hadoop configuration files, storing them in encrypted
@@ -557,7 +520,7 @@ For additional reading on the Hadoop Credential Provider API see:
 [Credential Provider API](../../../hadoop-project-dist/hadoop-common/CredentialProviderAPI.html).
 
 
-###### Create a credential file
+### Step 1: Create a credential file
 
 A credential file can be created on any Hadoop filesystem; when creating one on HDFS or
 a Unix filesystem the permissions are automatically set to keep the file
@@ -585,7 +548,7 @@ fs.s3a.access.key
 ```
 At this point, the credentials are ready for use.
 
-###### Configure the `hadoop.security.credential.provider.path` property
+### Step 2: Configure the `hadoop.security.credential.provider.path` property
 
 The URL to the provider must be set in the configuration property
 `hadoop.security.credential.provider.path`, either on the command line or
@@ -625,18 +588,16 @@ Supporting a separate list in an `fs.s3a.` prefix permits per-bucket configurati
 of credential files.
 
 
-###### Using the credentials
+### Using secrets from credential providers
 
 Once the provider is set in the Hadoop configuration, hadoop commands
 work exactly as if the secrets were in an XML file.
 
 ```bash
-
 hadoop distcp \
     hdfs://nn1.example.com:9001/user/backup/007020615 s3a://glacier1/
 
 hadoop fs -ls s3a://glacier1/
-
 ```
 
 The path to the provider can also be set on the command line:
@@ -649,262 +610,272 @@ hadoop distcp \
 hadoop fs \
   -D fs.s3a.security.credential.provider.path=jceks://hdfs@nn1.example.com:9001/user/backup/s3.jceks \
   -ls s3a://glacier1/
-
 ```
 
 Because the provider path is not itself a sensitive secret, there is no risk
 from placing its declaration on the command line.
 
 
-### Other properties
+## <a name="general_configuration"></a>Genaral S3A Client configuration
 
-    <property>
-      <name>fs.s3a.connection.maximum</name>
-      <value>15</value>
-      <description>Controls the maximum number of simultaneous connections to S3.</description>
-    </property>
+All S3A client options are configured with options with the prefix `fs.s3a.`.
 
-    <property>
-      <name>fs.s3a.connection.ssl.enabled</name>
-      <value>true</value>
-      <description>Enables or disables SSL connections to S3.</description>
-    </property>
+The client supports <a href="per_bucket_configuration">Per-bucket configuration</a>
+to allow different buckets to override the shared settings. This is commonly
+used to change the endpoint, encryption and authentication mechanisms of buckets.
+S3Guard options, various minor options.
 
-    <property>
-      <name>fs.s3a.endpoint</name>
-      <description>AWS S3 endpoint to connect to. An up-to-date list is
-        provided in the AWS Documentation: regions and endpoints. Without this
-        property, the standard region (s3.amazonaws.com) is assumed.
-      </description>
-    </property>
+Here are the S3A properties for use in production. The S3Guard options are
+documented in the [S3Guard documenents](./s3guard.html); some testing-related
+options are covered in [Testing](./testing.md).
 
-    <property>
-      <name>fs.s3a.path.style.access</name>
-      <value>false</value>
-      <description>Enable S3 path style access ie disabling the default virtual hosting behaviour.
-        Useful for S3A-compliant storage providers as it removes the need to set up DNS for virtual hosting.
-      </description>
-    </property>
+```xml
+<property>
+  <name>fs.s3a.connection.maximum</name>
+  <value>15</value>
+  <description>Controls the maximum number of simultaneous connections to S3.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.proxy.host</name>
-      <description>Hostname of the (optional) proxy server for S3 connections.</description>
-    </property>
+<property>
+  <name>fs.s3a.connection.ssl.enabled</name>
+  <value>true</value>
+  <description>Enables or disables SSL connections to S3.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.proxy.port</name>
-      <description>Proxy server port. If this property is not set
-        but fs.s3a.proxy.host is, port 80 or 443 is assumed (consistent with
-        the value of fs.s3a.connection.ssl.enabled).</description>
-    </property>
+<property>
+  <name>fs.s3a.endpoint</name>
+  <description>AWS S3 endpoint to connect to. An up-to-date list is
+    provided in the AWS Documentation: regions and endpoints. Without this
+    property, the standard region (s3.amazonaws.com) is assumed.
+  </description>
+</property>
 
-    <property>
-      <name>fs.s3a.proxy.username</name>
-      <description>Username for authenticating with proxy server.</description>
-    </property>
+<property>
+  <name>fs.s3a.path.style.access</name>
+  <value>false</value>
+  <description>Enable S3 path style access ie disabling the default virtual hosting behaviour.
+    Useful for S3A-compliant storage providers as it removes the need to set up DNS for virtual hosting.
+  </description>
+</property>
 
-    <property>
-      <name>fs.s3a.proxy.password</name>
-      <description>Password for authenticating with proxy server.</description>
-    </property>
+<property>
+  <name>fs.s3a.proxy.host</name>
+  <description>Hostname of the (optional) proxy server for S3 connections.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.proxy.domain</name>
-      <description>Domain for authenticating with proxy server.</description>
-    </property>
+<property>
+  <name>fs.s3a.proxy.port</name>
+  <description>Proxy server port. If this property is not set
+    but fs.s3a.proxy.host is, port 80 or 443 is assumed (consistent with
+    the value of fs.s3a.connection.ssl.enabled).</description>
+</property>
 
-    <property>
-      <name>fs.s3a.proxy.workstation</name>
-      <description>Workstation for authenticating with proxy server.</description>
-    </property>
+<property>
+  <name>fs.s3a.proxy.username</name>
+  <description>Username for authenticating with proxy server.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.attempts.maximum</name>
-      <value>20</value>
-      <description>How many times we should retry commands on transient errors.</description>
-    </property>
+<property>
+  <name>fs.s3a.proxy.password</name>
+  <description>Password for authenticating with proxy server.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.connection.establish.timeout</name>
-      <value>5000</value>
-      <description>Socket connection setup timeout in milliseconds.</description>
-    </property>
+<property>
+  <name>fs.s3a.proxy.domain</name>
+  <description>Domain for authenticating with proxy server.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.connection.timeout</name>
-      <value>200000</value>
-      <description>Socket connection timeout in milliseconds.</description>
-    </property>
+<property>
+  <name>fs.s3a.proxy.workstation</name>
+  <description>Workstation for authenticating with proxy server.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.paging.maximum</name>
-      <value>5000</value>
-      <description>How many keys to request from S3 when doing
-         directory listings at a time.</description>
-    </property>
+<property>
+  <name>fs.s3a.attempts.maximum</name>
+  <value>20</value>
+  <description>How many times we should retry commands on transient errors.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.threads.max</name>
-      <value>10</value>
-      <description> Maximum number of concurrent active (part)uploads,
-      which each use a thread from the threadpool.</description>
-    </property>
+<property>
+  <name>fs.s3a.connection.establish.timeout</name>
+  <value>5000</value>
+  <description>Socket connection setup timeout in milliseconds.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.socket.send.buffer</name>
-      <value>8192</value>
-      <description>Socket send buffer hint to amazon connector. Represented in bytes.</description>
-    </property>
+<property>
+  <name>fs.s3a.connection.timeout</name>
+  <value>200000</value>
+  <description>Socket connection timeout in milliseconds.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.socket.recv.buffer</name>
-      <value>8192</value>
-      <description>Socket receive buffer hint to amazon connector. Represented in bytes.</description>
-    </property>
+<property>
+  <name>fs.s3a.paging.maximum</name>
+  <value>5000</value>
+  <description>How many keys to request from S3 when doing
+     directory listings at a time.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.threads.keepalivetime</name>
-      <value>60</value>
-      <description>Number of seconds a thread can be idle before being
-        terminated.</description>
-    </property>
+<property>
+  <name>fs.s3a.threads.max</name>
+  <value>10</value>
+  <description> Maximum number of concurrent active (part)uploads,
+  which each use a thread from the threadpool.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.max.total.tasks</name>
-      <value>5</value>
-      <description>Number of (part)uploads allowed to the queue before
-      blocking additional uploads.</description>
-    </property>
+<property>
+  <name>fs.s3a.socket.send.buffer</name>
+  <value>8192</value>
+  <description>Socket send buffer hint to amazon connector. Represented in bytes.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.multipart.size</name>
-      <value>100M</value>
-      <description>How big (in bytes) to split upload or copy operations up into.
-        A suffix from the set {K,M,G,T,P} may be used to scale the numeric value.
-      </description>
-    </property>
+<property>
+  <name>fs.s3a.socket.recv.buffer</name>
+  <value>8192</value>
+  <description>Socket receive buffer hint to amazon connector. Represented in bytes.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.multipart.threshold</name>
-      <value>2147483647</value>
-      <description>How big (in bytes) to split upload or copy operations up into.
-        This also controls the partition size in renamed files, as rename() involves
-        copying the source file(s).
-        A suffix from the set {K,M,G,T,P} may be used to scale the numeric value.
-      </description>
-    </property>
+<property>
+  <name>fs.s3a.threads.keepalivetime</name>
+  <value>60</value>
+  <description>Number of seconds a thread can be idle before being
+    terminated.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.multiobjectdelete.enable</name>
-      <value>true</value>
-      <description>When enabled, multiple single-object delete requests are replaced by
-        a single 'delete multiple objects'-request, reducing the number of requests.
-        Beware: legacy S3-compatible object stores might not support this request.
-      </description>
-    </property>
+<property>
+  <name>fs.s3a.max.total.tasks</name>
+  <value>5</value>
+  <description>Number of (part)uploads allowed to the queue before
+  blocking additional uploads.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.acl.default</name>
-      <description>Set a canned ACL for newly created and copied objects. Value may be Private,
-        PublicRead, PublicReadWrite, AuthenticatedRead, LogDeliveryWrite, BucketOwnerRead,
-        or BucketOwnerFullControl.</description>
-    </property>
+<property>
+  <name>fs.s3a.multipart.size</name>
+  <value>100M</value>
+  <description>How big (in bytes) to split upload or copy operations up into.
+    A suffix from the set {K,M,G,T,P} may be used to scale the numeric value.
+  </description>
+</property>
 
-    <property>
-      <name>fs.s3a.multipart.purge</name>
-      <value>false</value>
-      <description>True if you want to purge existing multipart uploads that may not have been
-         completed/aborted correctly</description>
-    </property>
+<property>
+  <name>fs.s3a.multipart.threshold</name>
+  <value>2147483647</value>
+  <description>How big (in bytes) to split upload or copy operations up into.
+    This also controls the partition size in renamed files, as rename() involves
+    copying the source file(s).
+    A suffix from the set {K,M,G,T,P} may be used to scale the numeric value.
+  </description>
+</property>
 
-    <property>
-      <name>fs.s3a.multipart.purge.age</name>
-      <value>86400</value>
-      <description>Minimum age in seconds of multipart uploads to purge</description>
-    </property>
+<property>
+  <name>fs.s3a.multiobjectdelete.enable</name>
+  <value>true</value>
+  <description>When enabled, multiple single-object delete requests are replaced by
+    a single 'delete multiple objects'-request, reducing the number of requests.
+    Beware: legacy S3-compatible object stores might not support this request.
+  </description>
+</property>
 
-    <property>
-      <name>fs.s3a.signing-algorithm</name>
-      <description>Override the default signing algorithm so legacy
-        implementations can still be used</description>
-    </property>
+<property>
+  <name>fs.s3a.acl.default</name>
+  <description>Set a canned ACL for newly created and copied objects. Value may be Private,
+    PublicRead, PublicReadWrite, AuthenticatedRead, LogDeliveryWrite, BucketOwnerRead,
+    or BucketOwnerFullControl.</description>
+</property>
+
+<property>
+  <name>fs.s3a.multipart.purge</name>
+  <value>false</value>
+  <description>True if you want to purge existing multipart uploads that may not have been
+     completed/aborted correctly</description>
+</property>
+
+<property>
+  <name>fs.s3a.multipart.purge.age</name>
+  <value>86400</value>
+  <description>Minimum age in seconds of multipart uploads to purge</description>
+</property>
+
+<property>
+  <name>fs.s3a.signing-algorithm</name>
+  <description>Override the default signing algorithm so legacy
+    implementations can still be used</description>
+</property>
 
-    <property>
-      <name>fs.s3a.server-side-encryption-algorithm</name>
-      <description>Specify a server-side encryption algorithm for s3a: file system.
-        Unset by default. It supports the following values: 'AES256' (for SSE-S3), 'SSE-KMS'
-         and 'SSE-C'
-      </description>
-    </property>
+<property>
+  <name>fs.s3a.server-side-encryption-algorithm</name>
+  <description>Specify a server-side encryption algorithm for s3a: file system.
+    Unset by default. It supports the following values: 'AES256' (for SSE-S3), 'SSE-KMS'
+     and 'SSE-C'
+  </description>
+</property>
 
-    <property>
-        <name>fs.s3a.server-side-encryption.key</name>
-        <description>Specific encryption key to use if fs.s3a.server-side-encryption-algorithm
-        has been set to 'SSE-KMS' or 'SSE-C'. In the case of SSE-C, the value of this property
-        should be the Base64 encoded key. If you are using SSE-KMS and leave this property empty,
-        you'll be using your default's S3 KMS key, otherwise you should set this property to
-        the specific KMS key id.</description>
-    </property>
+<property>
+    <name>fs.s3a.server-side-encryption.key</name>
+    <description>Specific encryption key to use if fs.s3a.server-side-encryption-algorithm
+    has been set to 'SSE-KMS' or 'SSE-C'. In the case of SSE-C, the value of this property
+    should be the Base64 encoded key. If you are using SSE-KMS and leave this property empty,
+    you'll be using your default's S3 KMS key, otherwise you should set this property to
+    the specific KMS key id.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.buffer.dir</name>
-      <value>${hadoop.tmp.dir}/s3a</value>
-      <description>Comma separated list of directories that will be used to buffer file
-        uploads to. No effect if fs.s3a.fast.upload is true.</description>
-    </property>
+<property>
+  <name>fs.s3a.buffer.dir</name>
+  <value>${hadoop.tmp.dir}/s3a</value>
+  <description>Comma separated list of directories that will be used to buffer file
+    uploads to.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.block.size</name>
-      <value>32M</value>
-      <description>Block size to use when reading files using s3a: file system.
-      </description>
-    </property>
+<property>
+  <name>fs.s3a.block.size</name>
+  <value>32M</value>
+  <description>Block size to use when reading files using s3a: file system.
+  </description>
+</property>
 
-    <property>
-      <name>fs.s3a.user.agent.prefix</name>
-      <value></value>
-      <description>
-        Sets a custom value that will be prepended to the User-Agent header sent in
-        HTTP requests to the S3 back-end by S3AFileSystem.  The User-Agent header
-        always includes the Hadoop version number followed by a string generated by
-        the AWS SDK.  An example is "User-Agent: Hadoop 2.8.0, aws-sdk-java/1.10.6".
-        If this optional property is set, then its value is prepended to create a
-        customized User-Agent.  For example, if this configuration property was set
-        to "MyApp", then an example of the resulting User-Agent would be
-        "User-Agent: MyApp, Hadoop 2.8.0, aws-sdk-java/1.10.6".
-      </description>
-    </property>
+<property>
+  <name>fs.s3a.user.agent.prefix</name>
+  <value></value>
+  <description>
+    Sets a custom value that will be prepended to the User-Agent header sent in
+    HTTP requests to the S3 back-end by S3AFileSystem.  The User-Agent header
+    always includes the Hadoop version number followed by a string generated by
+    the AWS SDK.  An example is "User-Agent: Hadoop 2.8.0, aws-sdk-java/1.10.6".
+    If this optional property is set, then its value is prepended to create a
+    customized User-Agent.  For example, if this configuration property was set
+    to "MyApp", then an example of the resulting User-Agent would be
+    "User-Agent: MyApp, Hadoop 2.8.0, aws-sdk-java/1.10.6".
+  </description>
+</property>
 
-    <property>
-      <name>fs.s3a.impl</name>
-      <value>org.apache.hadoop.fs.s3a.S3AFileSystem</value>
-      <description>The implementation class of the S3A Filesystem</description>
-    </property>
+<property>
+  <name>fs.s3a.impl</name>
+  <value>org.apache.hadoop.fs.s3a.S3AFileSystem</value>
+  <description>The implementation class of the S3A Filesystem</description>
+</property>
 
-    <property>
-      <name>fs.AbstractFileSystem.s3a.impl</name>
-      <value>org.apache.hadoop.fs.s3a.S3A</value>
-      <description>The implementation class of the S3A AbstractFileSystem.</description>
-    </property>
+<property>
+  <name>fs.AbstractFileSystem.s3a.impl</name>
+  <value>org.apache.hadoop.fs.s3a.S3A</value>
+  <description>The implementation class of the S3A AbstractFileSystem.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.readahead.range</name>
-      <value>64K</value>
-      <description>Bytes to read ahead during a seek() before closing and
-      re-opening the S3 HTTP connection. This option will be overridden if
-      any call to setReadahead() is made to an open stream.</description>
-    </property>
+<property>
+  <name>fs.s3a.readahead.range</name>
+  <value>64K</value>
+  <description>Bytes to read ahead during a seek() before closing and
+  re-opening the S3 HTTP connection. This option will be overridden if
+  any call to setReadahead() is made to an open stream.</description>
+</property>
 
-    <property>
-      <name>fs.s3a.list.version</name>
-      <value>2</value>
-      <description>
-        Select which version of the S3 SDK's List Objects API to use.  Currently
-        support 2 (default) and 1 (older API).
-      </description>
-    </property>
+<property>
+  <name>fs.s3a.list.version</name>
+  <value>2</value>
+  <description>Select which version of the S3 SDK's List Objects API to use.
+  Currently support 2 (default) and 1 (older API).</description>
+</property>
+```
 
-### Configuring different S3 buckets
+## <a name="per_bucket_configuration"></a>Configuring different S3 buckets with Per-Bucket Configuration
 
 Different S3 buckets can be accessed with different S3A client configurations.
 This allows for different endpoints, data read and write strategies, as well
@@ -927,9 +898,10 @@ role information available when deployed in Amazon EC2.
 </property>
 ```
 
-This will be the default authentication mechanism for S3A buckets.
+This will become the default authentication mechanism for S3A buckets.
 
-A bucket `s3a://nightly/` used for nightly data uses a session key:
+A bucket `s3a://nightly/` used for nightly data can then be given
+a session key:
 
 ```xml
 <property>
@@ -953,7 +925,7 @@ A bucket `s3a://nightly/` used for nightly data uses a session key:
 </property>
 ```
 
-Finally, the public `s3a://landsat-pds/` bucket is accessed anonymously:
+Finally, the public `s3a://landsat-pds/` bucket can be accessed anonymously:
 
 ```xml
 <property>
@@ -962,7 +934,7 @@ Finally, the public `s3a://landsat-pds/` bucket is accessed anonymously:
 </property>
 ```
 
-**Customizing S3A secrets held in credential files**
+### Customizing S3A secrets held in credential files
 
 Although most properties are automatically propagated from their
 `fs.s3a.bucket.`-prefixed custom entry to that of the base `fs.s3a.` option
@@ -976,7 +948,7 @@ then declare the path to the appropriate credential file in
 a bucket-specific version of the property `fs.s3a.security.credential.provider.path`.
 
 
-### Using Per-Bucket Configuration to access data round the world
+###  <a name="per_bucket_endpoints"></a>Using Per-Bucket Configuration to access data round the world
 
 S3 Buckets are hosted in different "regions", the default being "US-East".
 The S3A client talks to this region by default, issing HTTP requests
@@ -1082,7 +1054,6 @@ Here is a list of properties defining all AWS S3 regions, current as of June 201
 </property>
 ```
 
-
 This list can be used to specify the endpoint of individual buckets, for example
 for buckets in the central and EU/Ireland endpoints.
 
@@ -1098,57 +1069,28 @@ for buckets in the central and EU/Ireland endpoints.
   <value>${ireland.endpoint}</value>
   <description>The endpoint for s3a://eu-dataset URLs</description>
 </property>
-
 ```
 
 Why explicitly declare a bucket bound to the central endpoint? It ensures
 that if the default endpoint is changed to a new region, data store in
 US-east is still reachable.
 
+## <a name="upload"></a>How S3A writes data to S3
 
-### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload
-
-
-**New in Hadoop 2.7; significantly enhanced in Hadoop 2.8**
+The original S3A client implemented file writes by
+buffering all data to disk as it was written to the `OutputStream`.
+Only when the stream's `close()` method was called would the upload start.
 
+This can made output slow, especially on large uploads, and could even
+fill up the disk space of small (virtual) disks.
 
-Because of the nature of the S3 object store, data written to an S3A `OutputStream`
-is not written incrementally —instead, by default, it is buffered to disk
-until the stream is closed in its `close()` method.
+Hadoop 2.7 added the `S3AFastOutputStream` alternative, which Hadoop 2.8 expanded.
+It is now considered stable and has replaced the original `S3AOutputStream`,
+which is no longer shipped in hadoop.
 
-This can make output slow:
+The "fast" output stream
 
-* The execution time for `OutputStream.close()` is proportional to the amount of data
-buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`.
-* The bandwidth is that available from the host to S3: other work in the same
-process, server or network at the time of upload may increase the upload time,
-hence the duration of the `close()` call.
-* If a process uploading data fails before `OutputStream.close()` is called,
-all data is lost.
-* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must
-have the capacity to store the entire buffered file.
-
-Put succinctly: the further the process is from the S3 endpoint, or the smaller
-the EC-hosted VM is, the longer it will take work to complete.
-
-This can create problems in application code:
-
-* Code often assumes that the `close()` call is fast;
- the delays can create bottlenecks in operations.
-* Very slow uploads sometimes cause applications to time out. (generally,
-threads blocking during the upload stop reporting progress, so trigger timeouts)
-* Streaming very large amounts of data may consume all disk space before the upload begins.
-
-
-Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream`
-[HADOOP-11183](https://issues.apache.org/jira/browse/HADOOP-11183), and
-has continued with ` S3ABlockOutputStream`
-[HADOOP-13560](https://issues.apache.org/jira/browse/HADOOP-13560).
-
-
-This adds an alternative output stream, "S3a Fast Upload" which:
-
-1.  Always uploads large files as blocks with the size set by
+1.  Uploads large files as blocks with the size set by
     `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads
     begin and the size of each upload are identical.
 1.  Buffers blocks to disk (default) or in on-heap or off-heap memory.
@@ -1163,34 +1105,19 @@ This adds an alternative output stream, "S3a Fast Upload" which:
 1.  Has the time to `close()` set by the amount of remaning data to upload, rather
     than the total size of the file.
 
-With incremental writes of blocks, "S3A fast upload" offers an upload
-time at least as fast as the "classic" mechanism, with significant benefits
-on long-lived output streams, and when very large amounts of data are generated.
-The in memory buffering mechanims may also  offer speedup when running adjacent to
+Because it starts uploading while data is still being written, it offers
+significant benefits when very large amounts of data are generated.
+The in memory buffering mechanims may also offer speedup when running adjacent to
 S3 endpoints, as disks are not used for intermediate data storage.
 
 
 ```xml
 <property>
-  <name>fs.s3a.fast.upload</name>
-  <value>true</value>
-  <description>
-    Use the incremental block upload mechanism with
-    the buffering mechanism set in fs.s3a.fast.upload.buffer.
-    The number of threads performing uploads in the filesystem is defined
-    by fs.s3a.threads.max; the queue of waiting uploads limited by
-    fs.s3a.max.total.tasks.
-    The size of each buffer is set by fs.s3a.multipart.size.
-  </description>
-</property>
-
-<property>
   <name>fs.s3a.fast.upload.buffer</name>
   <value>disk</value>
   <description>
-    The buffering mechanism to use when using S3A fast upload
-    (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
-    This configuration option has no effect if fs.s3a.fast.upload is false.
+    The buffering mechanism to use.
+    Values: disk, array, bytebuffer.
 
     "disk" will use the directories listed in fs.s3a.buffer.dir as
     the location(s) to save data prior to being uploaded.
@@ -1244,45 +1171,36 @@ upload operation counts, so identifying when there is a backlog of work/
 a mismatch between data generation rates and network bandwidth. Per-stream
 statistics can also be logged by calling `toString()` on the current stream.
 
-* Incremental writes are not visible; the object can only be listed
-or read when the multipart operation completes in the `close()` call, which
-will block until the upload is completed.
+* Files being written are still invisible untl the write
+completes in the `close()` call, which will block until the upload is completed.
 
 
-#### <a name="s3a_fast_upload_disk"></a>Fast Upload with Disk Buffers `fs.s3a.fast.upload.buffer=disk`
+### <a name="upload_disk"></a>Buffering upload data on disk `fs.s3a.fast.upload.buffer=disk`
 
 When `fs.s3a.fast.upload.buffer` is set to `disk`, all data is buffered
 to local hard disks prior to upload. This minimizes the amount of memory
 consumed, and so eliminates heap size as the limiting factor in queued uploads
-—exactly as the original "direct to disk" buffering used when
-`fs.s3a.fast.upload=false`.
+—exactly as the original "direct to disk" buffering.
 
 
 ```xml
 <property>
-  <name>fs.s3a.fast.upload</name>
-  <value>true</value>
-</property>
-
-<property>
   <name>fs.s3a.fast.upload.buffer</name>
   <value>disk</value>
 </property>
 
 <property>
   <name>fs.s3a.buffer.dir</name>
-  <value></value>
-  <description>Comma separated list of temporary directories use for
-  storing blocks of data prior to their being uploaded to S3.
-  When unset, the Hadoop temporary directory hadoop.tmp.dir is used</description>
+  <value>${hadoop.tmp.dir}/s3a</value>
+  <description>Comma separated list of directories that will be used to buffer file
+    uploads to.</description>
 </property>
-
 ```
 
 This is the default buffer mechanism. The amount of data which can
 be buffered is limited by the amount of available disk space.
 
-#### <a name="s3a_fast_upload_bytebuffer"></a>Fast Upload with ByteBuffers: `fs.s3a.fast.upload.buffer=bytebuffer`
+### <a name="upload_bytebuffer"></a>Buffering upload data in ByteBuffers: `fs.s3a.fast.upload.buffer=bytebuffer`
 
 When `fs.s3a.fast.upload.buffer` is set to `bytebuffer`, all data is buffered
 in "Direct" ByteBuffers prior to upload. This *may* be faster than buffering to disk,
@@ -1296,52 +1214,39 @@ the amount of memory requested for each container.
 
 The slower the upload bandwidth to S3, the greater the risk of running out
 of memory —and so the more care is needed in
-[tuning the upload settings](#s3a_fast_upload_thread_tuning).
+[tuning the upload settings](#upload_thread_tuning).
 
 
 ```xml
 <property>
-  <name>fs.s3a.fast.upload</name>
-  <value>true</value>
-</property>
-
-<property>
   <name>fs.s3a.fast.upload.buffer</name>
   <value>bytebuffer</value>
 </property>
 ```
 
-#### <a name="s3a_fast_upload_array"></a>Fast Upload with Arrays: `fs.s3a.fast.upload.buffer=array`
+### <a name="upload_array"></a>Buffering upload data in byte arrays: `fs.s3a.fast.upload.buffer=array`
 
 When `fs.s3a.fast.upload.buffer` is set to `array`, all data is buffered
 in byte arrays in the JVM's heap prior to upload.
 This *may* be faster than buffering to disk.
 
-This `array` option is similar to the in-memory-only stream offered in
-Hadoop 2.7 with `fs.s3a.fast.upload=true`
-
 The amount of data which can be buffered is limited by the available
 size of the JVM heap heap. The slower the write bandwidth to S3, the greater
 the risk of heap overflows. This risk can be mitigated by
-[tuning the upload settings](#s3a_fast_upload_thread_tuning).
+[tuning the upload settings](#upload_thread_tuning).
 
 ```xml
 <property>
-  <name>fs.s3a.fast.upload</name>
-  <value>true</value>
-</property>
-
-<property>
   <name>fs.s3a.fast.upload.buffer</name>
   <value>array</value>
 </property>
-
 ```
-#### <a name="s3a_fast_upload_thread_tuning"></a>S3A Fast Upload Thread Tuning
 
-Both the [Array](#s3a_fast_upload_array) and [Byte buffer](#s3a_fast_upload_bytebuffer)
+### <a name="upload_thread_tuning"></a>Upload Thread Tuning
+
+Both the [Array](#upload_array) and [Byte buffer](#upload_bytebuffer)
 buffer mechanisms can consume very large amounts of memory, on-heap or
-off-heap respectively. The [disk buffer](#s3a_fast_upload_disk) mechanism
+off-heap respectively. The [disk buffer](#upload_disk) mechanism
 does not use much memory up, but will consume hard disk capacity.
 
 If there are many output streams being written to in a single process, the
@@ -1428,14 +1333,12 @@ from VMs running on EC2.
   <description>Number of seconds a thread can be idle before being
     terminated.</description>
 </property>
-
 ```
 
+### <a name="multipart_purge"></a>Cleaning up after partial Upload Failures: `fs.s3a.multipart.purge`
 
-#### <a name="s3a_multipart_purge"></a>Cleaning up After Incremental Upload Failures: `fs.s3a.multipart.purge`
 
-
-If an incremental streaming operation is interrupted, there may be
+If an large stream writeoperation is interrupted, there may be
 intermediate partitions uploaded to S3 —data which will be billed for.
 
 These charges can be reduced by enabling `fs.s3a.multipart.purge`,
@@ -1459,7 +1362,7 @@ older than this time.
 </property>
 ```
 
-If an S3A client is instantited with `fs.s3a.multipart.purge=true`,
+If an S3A client is instantiated with `fs.s3a.multipart.purge=true`,
 it will delete all out of date uploads *in the entire bucket*. That is: it will affect all
 multipart uploads to that bucket, from all applications.
 
@@ -1470,15 +1373,13 @@ rate.
 The best practise for using this option is to disable multipart purges in
 normal use of S3A, enabling only in manual/scheduled housekeeping operations.
 
-### S3A Experimental "fadvise" input policy support
-
-**Warning: EXPERIMENTAL: behavior may change in future**
+### S3A "fadvise" input policy support
 
 The S3A Filesystem client supports the notion of input policies, similar
 to that of the Posix `fadvise()` API call. This tunes the behavior of the S3A
 client to optimise HTTP GET requests for the different use cases.
 
-#### "sequential" (default)
+*"sequential"*
 
 Read through the file, possibly with some short forward seeks.
 
@@ -1490,11 +1391,11 @@ This is leads to maximum read throughput —but with very expensive
 backward seeks.
 
 
-#### "normal"
+*"normal" (default)*
 
-This is currently the same as "sequential".
+This is currently the same as "sequential", though it may evolve in future.
 
-#### "random"
+*"random"*
 
 Optimised for random IO, specifically the Hadoop `PositionedReadable`
 operations —though `seek(offset); read(byte_buffer)` also benefits.
@@ -1543,627 +1444,13 @@ to set fadvise policies on input streams. Once implemented,
 this will become the supported mechanism used for configuring the input IO policy.
 
 
-### <a name="s3a_encryption"></a> Encrypting objects with S3A
-
-Currently, S3A only supports S3's Server Side Encryption for at rest data encryption.
-It is *encouraged* to read up on the [AWS documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/serv-side-encryption.html)
-for S3 Server Side Encryption before using these options as each behave differently
-and the documentation will be more up to date on its behavior.  When configuring
-an encryption method in the `core-site.xml`, this will apply cluster wide.  Any
-new files written will be encrypted with this encryption configuration.  Any
-existing files when read, will decrypt using the existing method (if possible)
-and will not be re-encrypted with the new method. It is also possible if mixing
-multiple keys that the user does not have access to decrypt the object. It is
-**NOT** advised to mix and match encryption types in a bucket, and is *strongly*
-recommended to just one type and key per bucket.
-
-SSE-S3 is where S3 will manage the encryption keys for each object. The parameter
-for `fs.s3a.server-side-encryption-algorithm` is `AES256`.
-
-SSE-KMS is where the user specifies a Customer Master Key(CMK) that is used to
-encrypt the objects. The user may specify a specific CMK or leave the
-`fs.s3a.server-side-encryption.key` empty to use the default auto-generated key
-in AWS IAM.  Each CMK configured in AWS IAM is region specific, and cannot be
-used in a in a S3 bucket in a different region.  There is can also be policies
-assigned to the CMK that prohibit or restrict its use for users causing S3A
-requests to fail.
-
-SSE-C is where the user specifies an actual base64 encoded AES-256 key supplied
-and managed by the user.
-
-#### SSE-C Warning
-
-It is strongly recommended to fully understand how SSE-C works in the S3
-environment before using this encryption type.  Please refer to the Server Side
-Encryption documentation available from AWS.  SSE-C is only recommended for
-advanced users with advanced encryption use cases.  Failure to properly manage
-encryption keys can cause data loss.  Currently, the AWS S3 API(and thus S3A)
-only supports one encryption key and cannot support decrypting objects during
-moves under a previous key to a new destination.  It is **NOT** advised to use
-multiple encryption keys in a bucket, and is recommended to use one key per
-bucket and to not change this key.  This is due to when a request is made to S3,
-the actual encryption key must be provided to decrypt the object and access the
-metadata.  Since only one encryption key can be provided at a time, S3A will not
-pass the correct encryption key to decrypt the data. Please see the
-troubleshooting section for more information.
-
-
-## Troubleshooting S3A
-
-Common problems working with S3A are
-
-1. Classpath
-1. Authentication
-1. S3 Inconsistency side-effects
-
-Classpath is usually the first problem. For the S3x filesystem clients,
-you need the Hadoop-specific filesystem clients, third party S3 client libraries
-compatible with the Hadoop code, and any dependent libraries compatible with
-Hadoop and the specific JVM.
-
-The classpath must be set up for the process talking to S3: if this is code
-running in the Hadoop cluster, the JARs must be on that classpath. That
-includes `distcp`.
-
-
-### `ClassNotFoundException: org.apache.hadoop.fs.s3a.S3AFileSystem`
-
-(or `org.apache.hadoop.fs.s3native.NativeS3FileSystem`).
-
-These are the Hadoop classes, found in the `hadoop-aws` JAR. An exception
-reporting one of these classes is missing means that this JAR is not on
-the classpath.
-
-### `ClassNotFoundException: com.amazonaws.services.s3.AmazonS3Client`
-
-(or other `com.amazonaws` class.)
-
-This means that one or more of the `aws-*-sdk` JARs are missing. Add them.
-
-### Missing method in `com.amazonaws` class
-
-This can be triggered by incompatibilities between the AWS SDK on the classpath
-and the version which Hadoop was compiled with.
-
-The AWS SDK JARs change their signature enough between releases that the only
-way to safely update the AWS SDK version is to recompile Hadoop against the later
-version.
-
-There's nothing the Hadoop team can do here: if you get this problem, then sorry,
-but you are on your own. The Hadoop developer team did look at using reflection
-to bind to the SDK, but there were too many changes between versions for this
-to work reliably. All it did was postpone version compatibility problems until
-the specific codepaths were executed at runtime —this was actually a backward
-step in terms of fast detection of compatibility problems.
-
-### Missing method in a Jackson class
-
-This is usually caused by version mismatches between Jackson JARs on the
-classpath. All Jackson JARs on the classpath *must* be of the same version.
-
-
-### Authentication failure
-
-If Hadoop cannot authenticate with the S3 service endpoint,
-the client retries a number of times before eventually failing.
-When it finally gives up, it will report a message about signature mismatch:
-
-```
-com.amazonaws.services.s3.model.AmazonS3Exception:
- The request signature we calculated does not match the signature you provided.
- Check your key and signing method.
-  (Service: Amazon S3; Status Code: 403; Error Code: SignatureDoesNotMatch,
-```
-
-The likely cause is that you either have the wrong credentials or somehow
-the credentials were not readable on the host attempting to read or write
-the S3 Bucket.
-
-Enabling debug logging for the package `org.apache.hadoop.fs.s3a`
-can help provide more information.
-
-The most common cause is that you have the wrong credentials for any of the current
-authentication mechanism(s) —or somehow
-the credentials were not readable on the host attempting to read or write
-the S3 Bucket. However, there are a couple of system configuration problems
-(JVM version, system clock) which also need to be checked.
-
-Most common: there's an error in the configuration properties.
-
-
-1. Make sure that the name of the bucket is the correct one.
-That is: check the URL.
-
-1. Make sure the property names are correct. For S3A, they are
-`fs.s3a.access.key` and `fs.s3a.secret.key` —you cannot just copy the S3N
-properties and replace `s3n` with `s3a`.
-
-1. Make sure the properties are visible to the process attempting to
-talk to the object store. Placing them in `core-site.xml` is the standard
-mechanism.
-
-1. If using session authentication, the session may have expired.
-Generate a new session token and secret.
-
-1. If using environement variable-based authentication, make sure that the
-relevant variables are set in the environment in which the process is running.
-
-The standard first step is: try to use the AWS command line tools with the same
-credentials, through a command such as:
-
-    hdfs fs -ls s3a://my-bucket/
-
-Note the trailing "/" here; without that the shell thinks you are trying to list
-your home directory under the bucket, which will only exist if explicitly created.
-
-
-Attempting to list a bucket using inline credentials is a
-means of verifying that the key and secret can access a bucket;
+##<a name="further_reading"></a> Other Topics
 
-    hdfs fs -ls s3a://key:secret@my-bucket/
+### Copying Data with distcp
 
-Do escape any `+` or `/` symbols in the secret, as discussed below, and never
-share the URL, logs generated using it, or use such an inline authentication
-mechanism in production.
+Hadoop's `distcp` application can be used to copy data between a Hadoop
+cluster and Amazon S3.
+See [Copying Data Between a Cluster and Amazon S3](https://hortonworks.github.io/hdp-aws/s3-copy-data/index.html)
+for details on S3 copying specifically.
 
-Finally, if you set the environment variables, you can take advantage of S3A's
-support of environment-variable authentication by attempting the same ls operation.
-That is: unset the `fs.s3a` secrets and rely on the environment variables.
-
-#### Authentication failure due to clock skew
-
-The timestamp is used in signing to S3, so as to
-defend against replay attacks. If the system clock is too far behind *or ahead*
-of Amazon's, requests will be rejected.
-
-This can surface as the situation where
-read requests are allowed, but operations which write to the bucket are denied.
-
-Check the system clock.
-
-#### Authentication failure when using URLs with embedded secrets
-
-If using the (strongly discouraged) mechanism of including the
-AWS Key and secret in a URL, then both "+" and "/" symbols need
-to encoded in the URL. As many AWS secrets include these characters,
-encoding problems are not uncommon.
-
-| symbol | encoded  value|
-|-----------|-------------|
-| `+` | `%2B` |
-| `/` | `%2F` |
-
-
-As an example, a URL for `bucket` with AWS ID `user1` and secret `a+b/c` would
-be represented as
-
-```
-s3a://user1:a%2Bb%2Fc@bucket/
-```
-
-This technique is only needed when placing secrets in the URL. Again,
-this is something users are strongly advised against using.
-
-#### Authentication Failures When Running on Java 8u60+
-
-A change in the Java 8 JVM broke some of the `toString()` string generation
-of Joda Time 2.8.0, which stopped the Amazon S3 client from being able to
-generate authentication headers suitable for validation by S3.
-
-**Fix**: Make sure that the version of Joda Time is 2.8.1 or later, or
-use a new version of Java 8.
-
-
-### "Bad Request" exception when working with AWS S3 Frankfurt, Seoul, or other "V4" endpoint
-
-
-S3 Frankfurt and Seoul *only* support
-[the V4 authentication API](http://docs.aws.amazon.com/AmazonS3/latest/API/sig-v4-authenticating-requests.html).
-
-Requests using the V2 API will be rejected with 400 `Bad Request`
-
-```
-$ bin/hadoop fs -ls s3a://frankfurt/
-WARN s3a.S3AFileSystem: Client: Amazon S3 error 400: 400 Bad Request; Bad Request (retryable)
-
-com.amazonaws.services.s3.model.AmazonS3Exception: Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: 923C5D9E75E44C06), S3 Extended Request ID: HDwje6k+ANEeDsM6aJ8+D5gUmNAMguOk2BvZ8PH3g9z0gpH+IuwT7N19oQOnIr5CIx7Vqb/uThE=
-    at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182)
-    at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770)
-    at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
-    at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
-    at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
-    at com.amazonaws.services.s3.AmazonS3Client.headBucket(AmazonS3Client.java:1107)
-    at com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1070)
-    at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:307)
-    at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:284)
-    at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2793)
-    at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:101)
-    at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2830)
-    at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2812)
-    at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:389)
-    at org.apache.hadoop.fs.Path.getFileSystem(Path.java:356)
-    at org.apache.hadoop.fs.shell.PathData.expandAsGlob(PathData.java:325)
-    at org.apache.hadoop.fs.shell.Command.expandArgument(Command.java:235)
-    at org.apache.hadoop.fs.shell.Command.expandArguments(Command.java:218)
-    at org.apache.hadoop.fs.shell.FsCommand.processRawArguments(FsCommand.java:103)
-    at org.apache.hadoop.fs.shell.Command.run(Command.java:165)
-    at org.apache.hadoop.fs.FsShell.run(FsShell.java:315)
-    at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
-    at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90)
-    at org.apache.hadoop.fs.FsShell.main(FsShell.java:373)
-ls: doesBucketExist on frankfurt-new: com.amazonaws.services.s3.model.AmazonS3Exception:
-  Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request;
-```
-
-This happens when trying to work with any S3 service which only supports the
-"V4" signing API —but the client is configured to use the default S3A service
-endpoint.
-
-The S3A client needs to be given the endpoint to use via the `fs.s3a.endpoint`
-property.
-
-As an example, the endpoint for S3 Frankfurt is `s3.eu-central-1.amazonaws.com`:
-
-```xml
-<property>
-  <name>fs.s3a.endpoint</name>
-  <value>s3.eu-central-1.amazonaws.com</value>
-</property>
-```
-
-### Error message "The bucket you are attempting to access must be addressed using the specified endpoint"
-
-This surfaces when `fs.s3a.endpoint` is configured to use an S3 service endpoint
-which is neither the original AWS one, `s3.amazonaws.com` , nor the one where
-the bucket is hosted.  The error message contains the redirect target returned
-by S3, which can be used to determine the correct value for `fs.s3a.endpoint`.
-
-```
-org.apache.hadoop.fs.s3a.AWSS3IOException: Received permanent redirect response
-  to bucket.s3-us-west-2.amazonaws.com.  This likely indicates that the S3
-  endpoint configured in fs.s3a.endpoint does not match the AWS region
-  containing the bucket.: The bucket you are attempting to access must be
-  addressed using the specified endpoint. Please send all future requests to
-  this endpoint. (Service: Amazon S3; Status Code: 301;
-  Error Code: PermanentRedirect; Request ID: 7D39EC1021C61B11)
-        at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:132)
-        at org.apache.hadoop.fs.s3a.S3AFileSystem.initMultipartUploads(S3AFileSystem.java:287)
-        at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:203)
-        at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2895)
-        at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:102)
-        at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2932)
-        at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2914)
-        at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:390)
-```
-
-1. Use the [Specific endpoint of the bucket's S3 service](http://docs.aws.amazon.com/general/latest/gr/rande.html#s3_region)
-1. If not using "V4" authentication (see above), the original S3 endpoint
-can be used:
-
-```xml
-<property>
-  <name>fs.s3a.endpoint</name>
-  <value>s3.amazonaws.com</value>
-</property>
-```
-
-Using the explicit endpoint for the region is recommended for speed and
-to use the V4 signing API.
-
-
-### "Timeout waiting for connection from pool" when writing to S3A
-
-This happens when using the Block output stream, `fs.s3a.fast.upload=true` and
-the thread pool runs out of capacity.
-
-```
-[s3a-transfer-shared-pool1-t20] INFO  http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) - Unable to execute HTTP request: Timeout waiting for connection from poolorg.apache.http.conn.ConnectionPoolTimeoutException: Timeout waiting for connection from pool
-  at org.apache.http.impl.conn.PoolingClientConnectionManager.leaseConnection(PoolingClientConnectionManager.java:230)
-  at org.apache.http.impl.conn.PoolingClientConnectionManager$1.getConnection(PoolingClientConnectionManager.java:199)
-  at sun.reflect.GeneratedMethodAccessor13.invoke(Unknown Source)
-  at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
-  at java.lang.reflect.Method.invoke(Method.java:498)
-  at com.amazonaws.http.conn.ClientConnectionRequestFactory$Handler.invoke(ClientConnectionRequestFactory.java:70)
-  at com.amazonaws.http.conn.$Proxy10.getConnection(Unknown Source)
-  at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:424)
-  at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884)
-  at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
-  at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
-  at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728)
-  at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
-  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
-  at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
-  at com.amazonaws.services.s3.AmazonS3Client.doUploadPart(AmazonS3Client.java:2921)
-  at com.amazonaws.services.s3.AmazonS3Client.uploadPart(AmazonS3Client.java:2906)
-  at org.apache.hadoop.fs.s3a.S3AFileSystem.uploadPart(S3AFileSystem.java:1025)
-  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:360)
-  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:355)
-  at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239)
-  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
-  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
-  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
-  at java.lang.Thread.run(Thread.java:745)
-```
-
-Make sure that `fs.s3a.connection.maximum` is at least larger
-than `fs.s3a.threads.max`.
-
-```xml
-<property>
-  <name>fs.s3a.threads.max</name>
-  <value>20</value>
-</property>
-
-<property>
-  <name>fs.s3a.connection.maximum</name>
-  <value>30</value>
-</property>
-```
-
-### "Timeout waiting for connection from pool" when reading from S3A
-
-This happens when more threads are trying to read from an S3A system than
-the maximum number of allocated HTTP connections.
-
-Set `fs.s3a.connection.maximum` to a larger value (and at least as large as
-`fs.s3a.threads.max`)
-
-### Out of heap memory when writing to S3A via Fast Upload
-
-This can happen when using the fast upload mechanism (`fs.s3a.fast.upload=true`)
-and in-memory buffering (either `fs.s3a.fast.upload.buffer=array` or
-`fs.s3a.fast.upload.buffer=bytebuffer`).
-
-More data is being generated than in the JVM than it can upload to S3 —and
-so much data has been buffered that the JVM has run out of memory.
-
-Consult [S3A Fast Upload Thread Tuning](#s3a_fast_upload_thread_tuning) for
-detail on this issue and options to address it. Consider also buffering to
-disk, rather than memory.
-
-
-### When writing to S3A: "java.io.FileNotFoundException: Completing multi-part upload"
-
-
-```
-java.io.FileNotFoundException: Completing multi-part upload on fork-5/test/multipart/1c397ca6-9dfb-4ac1-9cf7-db666673246b: com.amazonaws.services.s3.model.AmazonS3Exception: The specified upload does not exist. The upload ID may be invalid, or the upload may have been aborted or completed. (Service: Amazon S3; Status Code: 404; Error Code: NoSuchUpload; Request ID: 84FF8057174D9369), S3 Extended Request ID: Ij5Yn6Eq/qIERH4Z6Io3YL2t9/qNZ7z9gjPb1FrTtTovZ8k1MXqh+zCYYjqmfJ/fCY6E1+JR9jA=
-  at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182)
-  at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770)
-  at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
-  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
-  at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
-  at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:2705)
-  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.complete(S3ABlockOutputStream.java:473)
-  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$200(S3ABlockOutputStream.java:382)
-  at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:272)
-  at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
-  at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
-```
-
-This surfaces if, while a multipart upload was taking place, all outstanding multipart
-uploads were garbage collected. The upload operation cannot complete because
-the data uploaded has been deleted.
-
-Consult [Cleaning up After Incremental Upload Failures](#s3a_multipart_purge) for
-details on how the multipart purge timeout can be set. If multipart uploads
-are failing with the message above, it may be a sign that this value is too low.
-
-### `MultiObjectDeleteException` during delete or rename of files
-
-```
-Exception in thread "main" com.amazonaws.services.s3.model.MultiObjectDeleteException:
-    Status Code: 0, AWS Service: null, AWS Request ID: null, AWS Error Code: null,
-    AWS Error Message: One or more objects could not be deleted, S3 Extended Request ID: null
-  at com.amazonaws.services.s3.AmazonS3Client.deleteObjects(AmazonS3Client.java:1745)
-```
-This happens when trying to delete multiple objects, and one of the objects
-could not be deleted. It *should not occur* just because the object is missing.
-More specifically: at the time this document was written, we could not create
-such a failure.
-
-It will occur if the caller lacks the permission to delete any of the objects.
-
-Consult the log to see the specifics of which objects could not be deleted.
-Do you have permission to do so?
-
-If this operation is failing for reasons other than the caller lacking
-permissions:
-
-1. Try setting `fs.s3a.multiobjectdelete.enable` to `false`.
-1. Consult [HADOOP-11572](https://issues.apache.org/jira/browse/HADOOP-11572)
-for up to date advice.
-
-### When writing to S3A, HTTP Exceptions logged at info from `AmazonHttpClient`
-
-```
-[s3a-transfer-shared-pool4-t6] INFO  http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) - Unable to execute HTTP request: hwdev-steve-ireland-new.s3.amazonaws.com:443 failed to respond
-org.apache.http.NoHttpResponseException: bucket.s3.amazonaws.com:443 failed to respond
-  at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:143)
-  at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:57)
-  at org.apache.http.impl.io.AbstractMessageParser.parse(AbstractMessageParser.java:261)
-  at org.apache.http.impl.AbstractHttpClientConnection.receiveResponseHeader(AbstractHttpClientConnection.java:283)
-  at org.apache.http.impl.conn.DefaultClientConnection.receiveResponseHeader(DefaultClientConnection.java:259)
-  at org.apache.http.impl.conn.ManagedClientConnectionImpl.receiveResponseHeader(ManagedClientConnectionImpl.java:209)
-  at org.apache.http.protocol.HttpRequestExecutor.doReceiveResponse(HttpRequestExecutor.java:272)
-  at com.amazonaws.http.protocol.SdkHttpRequestExecutor.doReceiveResponse(SdkHttpRequestExecutor.java:66)
-  at org.apache.http.protocol.HttpRequestExecutor.execute(HttpRequestExecutor.java:124)
-  at org.apache.http.impl.client.DefaultRequestDirector.tryExecute(DefaultRequestDirector.java:686)
-  at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:488)
-  at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884)
-  at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
-  at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
-  at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728)
-  at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
-  at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
-  at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
-  at com.amazonaws.services.s3.AmazonS3Client.copyPart(AmazonS3Client.java:1731)
-  at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:41)
-  at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:28)
-  at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239)
-  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
-  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
-  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
-  at java.lang.Thread.run(Thread.java:745)
-```
-
-These are HTTP I/O exceptions caught and logged inside the AWS SDK. The client
-will attempt to retry the operation; it may just be a transient event. If there
-are many such exceptions in logs, it may be a symptom of connectivity or network
-problems.
-
-### Visible S3 Inconsistency
-
-Amazon S3 is *an eventually consistent object store*. That is: not a filesystem.
-
-It offers read-after-create consistency: a newly created file is immediately
-visible. Except, there is a small quirk: a negative GET may be cached, such
-that even if an object is immediately created, the fact that there "wasn't"
-an object is still remembered.
-
-That means the following sequence on its own will be consistent
-```
-touch(path) -> getFileStatus(path)
-```
-
-But this sequence *may* be inconsistent.
-
-```
-getFileStatus(path) -> touch(path) -> getFileStatus(path)
-```
-
-A common source of visible inconsistencies is that the S3 metadata
-database —the part of S3 which serves list requests— is updated asynchronously.
-Newly added or deleted files may not be visible in the index, even though direct
-operations on the object (`HEAD` and `GET`) succeed.
-
-In S3A, that means the `getFileStatus()` and `open()` operations are more likely
-to be consistent with the state of the object store than any directory list
-operations (`listStatus()`, `listFiles()`, `listLocatedStatus()`,
-`listStatusIterator()`).
-
-
-### `FileNotFoundException` even though the file was just written.
-
-This can be a sign of consistency problems. It may also surface if there is some
-asynchronous file write operation still in progress in the client: the operation
-has returned, but the write has not yet completed. While the S3A client code
-does block during the `close()` operation, we suspect that asynchronous writes
-may be taking place somewhere in the stack —this could explain why parallel tests
-fail more often than serialized tests.
-
-### File not found in a directory listing, even though `getFileStatus()` finds it
-
-(Similarly: deleted file found in listing, though `getFileStatus()` reports
-that it is not there)
-
-This is a visible sign of updates to the metadata server lagging
-behind the state of the underlying filesystem.
-
-
-### File not visible/saved
-
-The files in an object store are not visible until the write has been completed.
-In-progress writes are simply saved to a local file/cached in RAM and only uploaded.
-at the end of a write operation. If a process terminated unexpectedly, or failed
-to call the `close()` method on an output stream, the pending data will have
-been lost.
-
-### File `flush()` and `hflush()` calls do not save data to S3A
-
-Again, this is due to the 

<TRUNCATED>

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


[30/50] [abbrv] hadoop git commit: HDFS-12441. Supress UnresolvedPathException in namenode log. Contributed by Kihwal Lee.

Posted by ae...@apache.org.
HDFS-12441. Supress UnresolvedPathException in namenode log. Contributed by Kihwal Lee.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/de197fc1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/de197fc1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/de197fc1

Branch: refs/heads/HDFS-7240
Commit: de197fc1562c4d1c39d24738c7cf8a8869dcec9c
Parents: 2d2d97f
Author: Nathan Roberts <nr...@apache.org>
Authored: Fri Sep 15 13:08:41 2017 -0500
Committer: Nathan Roberts <nr...@apache.org>
Committed: Fri Sep 15 13:08:41 2017 -0500

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java  | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/de197fc1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 09c9d83..d62c0f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -517,10 +517,10 @@ public class NameNodeRpcServer implements NamenodeProtocols {
         QuotaByStorageTypeExceededException.class,
         AclException.class,
         FSLimitException.PathComponentTooLongException.class,
-        FSLimitException.MaxDirectoryItemsExceededException.class,
-        UnresolvedPathException.class);
+        FSLimitException.MaxDirectoryItemsExceededException.class);
 
-    clientRpcServer.addSuppressedLoggingExceptions(StandbyException.class);
+    clientRpcServer.addSuppressedLoggingExceptions(StandbyException.class,
+        UnresolvedPathException.class);
 
     clientRpcServer.setTracer(nn.tracer);
     if (serviceRpcServer != null) {


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


[36/50] [abbrv] hadoop git commit: HDFS-10701. TestDFSStripedOutputStreamWithFailure#testBlockTokenExpired occasionally fails. Contributed by SammiChen.

Posted by ae...@apache.org.
HDFS-10701. TestDFSStripedOutputStreamWithFailure#testBlockTokenExpired occasionally fails. Contributed by SammiChen.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ef8cd5dc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ef8cd5dc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ef8cd5dc

Branch: refs/heads/HDFS-7240
Commit: ef8cd5dc565f901b4954befe784675e130e84c3c
Parents: 1a84c24
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Sep 15 16:20:36 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Sep 15 16:20:36 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java     | 6 ++----
 1 file changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef8cd5dc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index ea889e3..57da439 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -260,8 +260,6 @@ public class TestDFSStripedOutputStreamWithFailure {
 
   @Test(timeout=240000)
   public void testBlockTokenExpired() throws Exception {
-    // TODO: this is very flaky, re-enable it later. See HDFS-12417.
-    assumeTrue("Test has been temporarily disabled. See HDFS-12417.", false);
     final int length = dataBlocks * (blockSize - cellSize);
     final HdfsConfiguration conf = newHdfsConfiguration();
 
@@ -494,8 +492,8 @@ public class TestDFSStripedOutputStreamWithFailure {
       final BlockManager bm = nn.getNamesystem().getBlockManager();
       final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager();
 
-      // set a short token lifetime (1 second)
-      SecurityTestUtil.setBlockTokenLifetime(sm, 1000L);
+      // set a short token lifetime (6 second)
+      SecurityTestUtil.setBlockTokenLifetime(sm, 6000L);
     }
 
     final AtomicInteger pos = new AtomicInteger();


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


[06/50] [abbrv] hadoop git commit: HADOOP-14738 Remove S3N and obsolete bits of S3A; rework docs. Contributed by Steve Loughran.

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/InMemoryNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/InMemoryNativeFileSystemStore.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/InMemoryNativeFileSystemStore.java
deleted file mode 100644
index c082493..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/InMemoryNativeFileSystemStore.java
+++ /dev/null
@@ -1,213 +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.s3native;
-
-import static org.apache.hadoop.fs.s3native.NativeS3FileSystem.PATH_DELIMITER;
-import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_BUFFER_DIR_KEY;
-import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.addDeprecatedConfigKeys;
-
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.SortedMap;
-import java.util.SortedSet;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.Time;
-
-/**
- * <p>
- * A stub implementation of {@link NativeFileSystemStore} for testing
- * {@link NativeS3FileSystem} without actually connecting to S3.
- * </p>
- */
-public class InMemoryNativeFileSystemStore implements NativeFileSystemStore {
-
-  static {
-    // Add the deprecated config keys
-    addDeprecatedConfigKeys();
-  }
-
-  private Configuration conf;
-  
-  private SortedMap<String, FileMetadata> metadataMap =
-    new TreeMap<String, FileMetadata>();
-  private SortedMap<String, byte[]> dataMap = new TreeMap<String, byte[]>();
-
-  @Override
-  public void initialize(URI uri, Configuration conf) throws IOException {
-    this.conf = conf;
-  }
-
-  @Override
-  public void storeEmptyFile(String key) throws IOException {
-    metadataMap.put(key, new FileMetadata(key, 0, Time.now()));
-    dataMap.put(key, new byte[0]);
-  }
-
-  @Override
-  public void storeFile(String key, File file, byte[] md5Hash)
-    throws IOException {
-    
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    byte[] buf = new byte[8192];
-    int numRead;
-    BufferedInputStream in = null;
-    try {
-      in = new BufferedInputStream(new FileInputStream(file));
-      while ((numRead = in.read(buf)) >= 0) {
-        out.write(buf, 0, numRead);
-      }
-    } finally {
-      if (in != null) {
-        in.close();
-      }
-    }
-    metadataMap.put(key,
-        new FileMetadata(key, file.length(), Time.now()));
-    dataMap.put(key, out.toByteArray());
-  }
-
-  @Override
-  public InputStream retrieve(String key) throws IOException {
-    return retrieve(key, 0);
-  }
-  
-  @Override
-  public InputStream retrieve(String key, long byteRangeStart)
-    throws IOException {
-    
-    byte[] data = dataMap.get(key);
-    File file = createTempFile();
-    BufferedOutputStream out = null;
-    try {
-      out = new BufferedOutputStream(new FileOutputStream(file));
-      out.write(data, (int) byteRangeStart,
-          data.length - (int) byteRangeStart);
-    } finally {
-      if (out != null) {
-        out.close();
-      }
-    }
-    return new FileInputStream(file);
-  }
-  
-  private File createTempFile() throws IOException {
-    File dir = new File(conf.get(S3_NATIVE_BUFFER_DIR_KEY));
-    if (!dir.exists() && !dir.mkdirs()) {
-      throw new IOException("Cannot create S3 buffer directory: " + dir);
-    }
-    File result = File.createTempFile("test-", ".tmp", dir);
-    result.deleteOnExit();
-    return result;
-  }
-
-  @Override
-  public FileMetadata retrieveMetadata(String key) throws IOException {
-    return metadataMap.get(key);
-  }
-
-  @Override
-  public PartialListing list(String prefix, int maxListingLength)
-      throws IOException {
-    return list(prefix, maxListingLength, null, false);
-  }
-
-  @Override
-  public PartialListing list(String prefix, int maxListingLength,
-      String priorLastKey, boolean recursive) throws IOException {
-
-    return list(prefix, recursive ? null : PATH_DELIMITER, maxListingLength, priorLastKey);
-  }
-
-  private PartialListing list(String prefix, String delimiter,
-      int maxListingLength, String priorLastKey) throws IOException {
-
-    if (prefix.length() > 0 && !prefix.endsWith(PATH_DELIMITER)) {
-      prefix += PATH_DELIMITER;
-    }
-    
-    List<FileMetadata> metadata = new ArrayList<FileMetadata>();
-    SortedSet<String> commonPrefixes = new TreeSet<String>();
-    for (String key : dataMap.keySet()) {
-      if (key.startsWith(prefix)) {
-        if (delimiter == null) {
-          metadata.add(retrieveMetadata(key));
-        } else {
-          int delimIndex = key.indexOf(delimiter, prefix.length());
-          if (delimIndex == -1) {
-            metadata.add(retrieveMetadata(key));
-          } else {
-            String commonPrefix = key.substring(0, delimIndex);
-            commonPrefixes.add(commonPrefix);
-          }
-        }
-      }
-      if (metadata.size() + commonPrefixes.size() == maxListingLength) {
-        new PartialListing(key, metadata.toArray(new FileMetadata[0]),
-            commonPrefixes.toArray(new String[0]));
-      }
-    }
-    return new PartialListing(null, metadata.toArray(new FileMetadata[0]),
-        commonPrefixes.toArray(new String[0]));
-  }
-
-  @Override
-  public void delete(String key) throws IOException {
-    metadataMap.remove(key);
-    dataMap.remove(key);
-  }
-
-  @Override
-  public void copy(String srcKey, String dstKey) throws IOException {
-    metadataMap.put(dstKey, metadataMap.get(srcKey));
-    dataMap.put(dstKey, dataMap.get(srcKey));
-  }
-  
-  @Override
-  public void purge(String prefix) throws IOException {
-    Iterator<Entry<String, FileMetadata>> i =
-      metadataMap.entrySet().iterator();
-    while (i.hasNext()) {
-      Entry<String, FileMetadata> entry = i.next();
-      if (entry.getKey().startsWith(prefix)) {
-        dataMap.remove(entry.getKey());
-        i.remove();
-      }
-    }
-  }
-
-  @Override
-  public void dump() throws IOException {
-    System.out.println(metadataMap.values());
-    System.out.println(dataMap.keySet());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/NativeS3FileSystemContractBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/NativeS3FileSystemContractBaseTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/NativeS3FileSystemContractBaseTest.java
deleted file mode 100644
index bfbca71..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/NativeS3FileSystemContractBaseTest.java
+++ /dev/null
@@ -1,266 +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.s3native;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URI;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystemContractBaseTest;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3native.NativeS3FileSystem.NativeS3FsInputStream;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.internal.AssumptionViolatedException;
-import static org.junit.Assert.*;
-
-public abstract class NativeS3FileSystemContractBaseTest
-  extends FileSystemContractBaseTest {
-  public static final String KEY_TEST_FS = "test.fs.s3n.name";
-  private NativeFileSystemStore store;
-  
-  abstract NativeFileSystemStore getNativeFileSystemStore() throws IOException;
-
-  @Before
-  public void setUp() throws Exception {
-    Configuration conf = new Configuration();
-    String fsname = conf.get(KEY_TEST_FS);
-    if (StringUtils.isEmpty(fsname)) {
-      throw new AssumptionViolatedException(
-          "No test FS defined in :" + KEY_TEST_FS);
-    }
-    store = getNativeFileSystemStore();
-    fs = new NativeS3FileSystem(store);
-    fs.initialize(URI.create(fsname), conf);
-  }
-  
-  @After
-  public void tearDown() throws Exception {
-    if (store != null) {
-      store.purge("test");
-    }
-  }
-
-  @Test
-  public void testCanonicalName() throws Exception {
-    assertNull("s3n doesn't support security token and shouldn't have canonical name",
-               fs.getCanonicalServiceName());
-  }
-
-  @Test
-  public void testListStatusForRoot() throws Exception {
-    FileStatus[] paths = fs.listStatus(path("/"));
-    assertEquals("Root directory is not empty; ", 0, paths.length);
-    
-    Path testDir = path("/test");
-    assertTrue(fs.mkdirs(testDir));
-    
-    paths = fs.listStatus(path("/"));
-    assertEquals(1, paths.length);
-    assertEquals(path("/test"), paths[0].getPath());
-  }
-
-  @Test
-  public void testNoTrailingBackslashOnBucket() throws Exception {
-    assertTrue(fs.getFileStatus(new Path(fs.getUri().toString())).isDirectory());
-  }
-
-  private void createTestFiles(String base) throws IOException {
-    store.storeEmptyFile(base + "/file1");
-    store.storeEmptyFile(base + "/dir/file2");
-    store.storeEmptyFile(base + "/dir/file3");
-  }
-
-  @Test
-  public void testDirWithDifferentMarkersWorks() throws Exception {
-
-    for (int i = 0; i <= 3; i++) {
-      String base = "test/hadoop" + i;
-      Path path = path("/" + base);
-
-      createTestFiles(base);
-
-      if (i == 0 ) {
-        //do nothing, we are testing correctness with no markers
-      }
-      else if (i == 1) {
-        // test for _$folder$ marker
-        store.storeEmptyFile(base + "_$folder$");
-        store.storeEmptyFile(base + "/dir_$folder$");
-      }
-      else if (i == 2) {
-        // test the end slash file marker
-        store.storeEmptyFile(base + "/");
-        store.storeEmptyFile(base + "/dir/");
-      }
-      else if (i == 3) {
-        // test both markers
-        store.storeEmptyFile(base + "_$folder$");
-        store.storeEmptyFile(base + "/dir_$folder$");
-        store.storeEmptyFile(base + "/");
-        store.storeEmptyFile(base + "/dir/");
-      }
-
-      assertTrue(fs.getFileStatus(path).isDirectory());
-      assertEquals(2, fs.listStatus(path).length);
-    }
-  }
-
-  @Test
-  public void testDeleteWithNoMarker() throws Exception {
-    String base = "test/hadoop";
-    Path path = path("/" + base);
-
-    createTestFiles(base);
-
-    fs.delete(path, true);
-
-    path = path("/test");
-    assertTrue(fs.getFileStatus(path).isDirectory());
-    assertEquals(0, fs.listStatus(path).length);
-  }
-
-  @Test
-  public void testRenameWithNoMarker() throws Exception {
-    String base = "test/hadoop";
-    Path dest = path("/test/hadoop2");
-
-    createTestFiles(base);
-
-    fs.rename(path("/" + base), dest);
-
-    Path path = path("/test");
-    assertTrue(fs.getFileStatus(path).isDirectory());
-    assertEquals(1, fs.listStatus(path).length);
-    assertTrue(fs.getFileStatus(dest).isDirectory());
-    assertEquals(2, fs.listStatus(dest).length);
-  }
-
-  @Test
-  public void testEmptyFile() throws Exception {
-    store.storeEmptyFile("test/hadoop/file1");
-    fs.open(path("/test/hadoop/file1")).close();
-  }
-
-  @Test
-  public void testBlockSize() throws Exception {
-    Path file = path("/test/hadoop/file");
-    createFile(file);
-    assertEquals("Default block size", fs.getDefaultBlockSize(file),
-    fs.getFileStatus(file).getBlockSize());
-
-    // Block size is determined at read time
-    long newBlockSize = fs.getDefaultBlockSize(file) * 2;
-    fs.getConf().setLong("fs.s3n.block.size", newBlockSize);
-    assertEquals("Double default block size", newBlockSize,
-    fs.getFileStatus(file).getBlockSize());
-  }
-
-  @Test
-  public void testRetryOnIoException() throws Exception {
-    class TestInputStream extends InputStream {
-      boolean shouldThrow = true;
-      int throwCount = 0;
-      int pos = 0;
-      byte[] bytes;
-      boolean threwException = false;
-      
-      public TestInputStream() {
-        bytes = new byte[256];
-        for (int i = pos; i < 256; i++) {
-          bytes[i] = (byte)i;
-        }
-      }
-      
-      @Override
-      public int read() throws IOException {
-        shouldThrow = !shouldThrow;
-        if (shouldThrow) {
-          throwCount++;
-          threwException = true;
-          throw new IOException();
-        }
-        assertFalse("IOException was thrown. InputStream should be reopened", threwException);
-        return pos++;
-      }
-      
-      @Override
-      public int read(byte[] b, int off, int len) throws IOException {
-        shouldThrow = !shouldThrow;
-        if (shouldThrow) {
-          throwCount++;
-          threwException = true;
-          throw new IOException();
-        }
-        assertFalse("IOException was thrown. InputStream should be reopened", threwException);
-        int sizeToRead = Math.min(len, 256 - pos);
-        for (int i = 0; i < sizeToRead; i++) {
-          b[i] = bytes[pos + i];
-        }
-        pos += sizeToRead;
-        return sizeToRead;
-      }
-
-      public void reopenAt(long byteRangeStart) {
-        threwException = false;
-        pos = Long.valueOf(byteRangeStart).intValue();
-      }
-
-    }
-    
-    final TestInputStream is = new TestInputStream();
-    
-    class MockNativeFileSystemStore extends Jets3tNativeFileSystemStore {
-      @Override
-      public InputStream retrieve(String key, long byteRangeStart) throws IOException {
-        is.reopenAt(byteRangeStart);
-        return is;
-      }
-    }
-    
-    NativeS3FsInputStream stream = new NativeS3FsInputStream(new MockNativeFileSystemStore(), null, is, "");
-    
-    // Test reading methods.
-    byte[] result = new byte[256];
-    for (int i = 0; i < 128; i++) {
-      result[i] = (byte)stream.read();
-    }
-    for (int i = 128; i < 256; i += 8) {
-      byte[] temp = new byte[8];
-      int read = stream.read(temp, 0, 8);
-      assertEquals(8, read);
-      System.arraycopy(temp, 0, result, i, 8);
-    }
-    
-    // Assert correct
-    for (int i = 0; i < 256; i++) {
-      assertEquals((byte)i, result[i]);
-    }
-    
-    // Test to make sure the throw path was exercised.
-    // every read should have thrown 1 IOException except for the first read
-    // 144 = 128 - 1 + (128 / 8)
-    assertEquals(143, ((TestInputStream)is).throwCount);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/S3NInMemoryFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/S3NInMemoryFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/S3NInMemoryFileSystem.java
deleted file mode 100644
index c0ea85b..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/S3NInMemoryFileSystem.java
+++ /dev/null
@@ -1,32 +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.s3native;
-
-import org.apache.hadoop.fs.s3native.NativeS3FileSystem;
-import org.apache.hadoop.fs.s3native.InMemoryNativeFileSystemStore;
-
-/**
- * A helper implementation of {@link NativeS3FileSystem}
- * without actually connecting to S3 for unit testing.
- */
-public class S3NInMemoryFileSystem extends NativeS3FileSystem {
-    public S3NInMemoryFileSystem() {
-        super(new InMemoryNativeFileSystemStore());
-    }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3Credentials.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3Credentials.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3Credentials.java
deleted file mode 100644
index 17b78c7..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3Credentials.java
+++ /dev/null
@@ -1,129 +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.s3native;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.ProviderUtils;
-import org.apache.hadoop.security.alias.CredentialProvider;
-import org.apache.hadoop.security.alias.CredentialProviderFactory;
-
-import java.io.File;
-import java.net.URI;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.rules.TestName;
-
-import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_AWS_ACCESS_KEY_ID;
-import static org.apache.hadoop.fs.s3native.S3NativeFileSystemConfigKeys.S3_NATIVE_AWS_SECRET_ACCESS_KEY;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-/**
- * This is to test the {@link S3Credentials} class for extracting AWS
- * credentials.
- */
-public class TestS3Credentials {
-  public static final Log LOG = LogFactory.getLog(TestS3Credentials.class);
-
-  @Rule
-  public final TestName test = new TestName();
-
-  @Before
-  public void announce() {
-    LOG.info("Running test " + test.getMethodName());
-  }
-
-  private static final String EXAMPLE_ID = "AKASOMEACCESSKEY";
-  private static final String EXAMPLE_KEY =
-      "RGV0cm9pdCBSZ/WQgY2xl/YW5lZCB1cAEXAMPLE";
-
-  @Test
-  public void testInvalidHostnameWithUnderscores() throws Exception {
-    S3Credentials s3Credentials = new S3Credentials();
-    try {
-      s3Credentials.initialize(new URI("s3n://a:b@c_d"), new Configuration());
-      fail("Should throw IllegalArgumentException");
-    } catch (IllegalArgumentException e) {
-      assertEquals("Invalid hostname in URI s3n://a:b@c_d", e.getMessage());
-    }
-  }
-
-  @Test
-  public void testPlaintextConfigPassword() throws Exception {
-    S3Credentials s3Credentials = new S3Credentials();
-    Configuration conf = new Configuration();
-    conf.set(S3_NATIVE_AWS_ACCESS_KEY_ID, EXAMPLE_ID);
-    conf.set(S3_NATIVE_AWS_SECRET_ACCESS_KEY, EXAMPLE_KEY);
-    s3Credentials.initialize(new URI("s3n://foobar"), conf);
-    assertEquals("Could not retrieve proper access key", EXAMPLE_ID,
-        s3Credentials.getAccessKey());
-    assertEquals("Could not retrieve proper secret", EXAMPLE_KEY,
-        s3Credentials.getSecretAccessKey());
-  }
-
-  @Test
-  public void testPlaintextConfigPasswordWithWhitespace() throws Exception {
-    S3Credentials s3Credentials = new S3Credentials();
-    Configuration conf = new Configuration();
-    conf.set(S3_NATIVE_AWS_ACCESS_KEY_ID, "\r\n " + EXAMPLE_ID +
-        " \r\n");
-    conf.set(S3_NATIVE_AWS_SECRET_ACCESS_KEY, "\r\n " + EXAMPLE_KEY +
-        " \r\n");
-    s3Credentials.initialize(new URI("s3n://foobar"), conf);
-    assertEquals("Could not retrieve proper access key", EXAMPLE_ID,
-        s3Credentials.getAccessKey());
-    assertEquals("Could not retrieve proper secret", EXAMPLE_KEY,
-        s3Credentials.getSecretAccessKey());
-  }
-
-  @Rule
-  public final TemporaryFolder tempDir = new TemporaryFolder();
-
-  @Test
-  public void testCredentialProvider() throws Exception {
-    // set up conf to have a cred provider
-    final Configuration conf = new Configuration();
-    final File file = tempDir.newFile("test.jks");
-    final URI jks = ProviderUtils.nestURIForLocalJavaKeyStoreProvider(
-        file.toURI());
-    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
-        jks.toString());
-
-    // add our creds to the provider
-    final CredentialProvider provider =
-        CredentialProviderFactory.getProviders(conf).get(0);
-    provider.createCredentialEntry(S3_NATIVE_AWS_SECRET_ACCESS_KEY,
-        EXAMPLE_KEY.toCharArray());
-    provider.flush();
-
-    // make sure S3Creds can retrieve things.
-    S3Credentials s3Credentials = new S3Credentials();
-    conf.set(S3_NATIVE_AWS_ACCESS_KEY_ID, EXAMPLE_ID);
-    s3Credentials.initialize(new URI("s3n://foobar"), conf);
-    assertEquals("Could not retrieve proper access key", EXAMPLE_ID,
-        s3Credentials.getAccessKey());
-    assertEquals("Could not retrieve proper secret", EXAMPLE_KEY,
-        s3Credentials.getSecretAccessKey());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3NInMemoryFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3NInMemoryFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3NInMemoryFileSystem.java
deleted file mode 100644
index b457df2..0000000
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3NInMemoryFileSystem.java
+++ /dev/null
@@ -1,69 +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.s3native;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.net.URI;
-
-import junit.framework.TestCase;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.Path;
-
-public class TestS3NInMemoryFileSystem extends TestCase {
-
-  private static final String TEST_PATH = "s3n://test/data.txt";
-  
-  private static final String TEST_DATA = "Sample data for testing.";
-  
-  private S3NInMemoryFileSystem fs;
-  
-  @Override
-  public void setUp() throws IOException {
-    fs = new S3NInMemoryFileSystem();
-    fs.initialize(URI.create("s3n://test/"), new Configuration());
-  }
- 
-  public void testBasicReadWriteIO() throws IOException {
-    FSDataOutputStream writeData = fs.create(new Path(TEST_PATH));
-    writeData.write(TEST_DATA.getBytes());
-    writeData.flush();
-    writeData.close();
-    
-    FSDataInputStream readData = fs.open(new Path(TEST_PATH));
-    BufferedReader br = new BufferedReader(new InputStreamReader(readData));
-    String line = "";
-    StringBuffer stringBuffer = new StringBuffer();
-    while ((line = br.readLine()) != null) {
-        stringBuffer.append(line);
-    }
-    br.close();
-    
-    assert(TEST_DATA.equals(stringBuffer.toString()));
-  }
-  
-  @Override
-  public void tearDown() throws IOException {
-    fs.close();  
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/resources/contract/s3n.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/resources/contract/s3n.xml b/hadoop-tools/hadoop-aws/src/test/resources/contract/s3n.xml
deleted file mode 100644
index 0c6b8c6..0000000
--- a/hadoop-tools/hadoop-aws/src/test/resources/contract/s3n.xml
+++ /dev/null
@@ -1,110 +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.
-  -->
-
-<configuration>
-  <!--
-  S3N is a blobstore, with very different behavior than a
-  classic filesystem.
-  -->
-
-  <property>
-    <name>fs.contract.test.root-tests-enabled</name>
-    <value>true</value>
-  </property>
-
-  <property>
-    <name>fs.contract.test.random-seek-count</name>
-    <value>10</value>
-  </property>
-
-  <property>
-    <name>fs.contract.is-blobstore</name>
-    <value>true</value>
-  </property>
-
-  <property>
-    <name>fs.contract.create-overwrites-directory</name>
-    <value>true</value>
-  </property>
-
-  <property>
-    <name>fs.contract.create-visibility-delayed</name>
-    <value>true</value>
-  </property>
-
-  <property>
-    <name>fs.contract.is-case-sensitive</name>
-    <value>true</value>
-  </property>
-
-  <property>
-    <name>fs.contract.rename-returns-false-if-source-missing</name>
-    <value>true</value>
-  </property>
-
-  <property>
-    <name>fs.contract.supports-append</name>
-    <value>false</value>
-  </property>
-
-  <property>
-    <name>fs.contract.supports-atomic-directory-delete</name>
-    <value>false</value>
-  </property>
-
-  <property>
-    <name>fs.contract.supports-atomic-rename</name>
-    <value>false</value>
-  </property>
-
-  <property>
-    <name>fs.contract.supports-block-locality</name>
-    <value>false</value>
-  </property>
-
-  <property>
-    <name>fs.contract.supports-concat</name>
-    <value>false</value>
-  </property>
-
-  <property>
-    <name>fs.contract.supports-seek</name>
-    <value>true</value>
-  </property>
-
-  <property>
-    <name>fs.contract.supports-seek-on-closed-file</name>
-    <value>true</value>
-  </property>
-
-  <property>
-    <name>fs.contract.rejects-seek-past-eof</name>
-    <value>true</value>
-  </property>
-
-  <property>
-    <name>fs.contract.supports-strict-exceptions</name>
-    <value>true</value>
-  </property>
-
-  <property>
-    <name>fs.contract.supports-unix-permissions</name>
-    <value>false</value>
-  </property>
-
-</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49467165/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
index 9376ebd..acbe7f1 100644
--- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
+++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties
@@ -32,3 +32,6 @@ log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
 # Log all HTTP requests made; includes S3 interaction. This may
 # include sensitive information such as account IDs in HTTP headers.
 #log4j.logger.com.amazonaws.request=DEBUG
+
+# Turn on low level HTTP protocol debugging
+#log4j.logger.com.amazonaws.thirdparty.apache.http=DEBUG


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


[38/50] [abbrv] hadoop git commit: YARN-7149. Cross-queue preemption sometimes starves an underserved queue. (Eric Payne via wangda)

Posted by ae...@apache.org.
YARN-7149. Cross-queue preemption sometimes starves an underserved queue. (Eric Payne via wangda)

Change-Id: Ib269991dbebce160378e8372ee6d24849c4a5ed6
(cherry picked from commit 3dfa937a1fadfc62947755872515f549b3b15e6a)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/38c14ef8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/38c14ef8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/38c14ef8

Branch: refs/heads/HDFS-7240
Commit: 38c14ef8d8a094a7101917eb77d90f5e62324f61
Parents: 958e8c0
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Sep 15 21:25:21 2017 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Sep 15 21:29:39 2017 -0700

----------------------------------------------------------------------
 .../scheduler/capacity/UsersManager.java        |  4 +-
 .../capacity/TestContainerAllocation.java       | 50 ++++++++++++++++++++
 .../scheduler/capacity/TestLeafQueue.java       |  8 ++--
 3 files changed, 57 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c14ef8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java
index 5f7d185..33f30b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java
@@ -731,7 +731,9 @@ public class UsersManager implements AbstractUsersManager {
      * should be higher than queue-hard-limit * ulMin
      */
     float usersSummedByWeight = activeUsersTimesWeights;
-    Resource resourceUsed = totalResUsageForActiveUsers.getUsed(nodePartition);
+    Resource resourceUsed = Resources.add(
+                            totalResUsageForActiveUsers.getUsed(nodePartition),
+                            required);
 
     // For non-activeUser calculation, consider all users count.
     if (!activeUser) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c14ef8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
index dd6b25b..906febf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
@@ -24,6 +24,7 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.SecurityUtilTestHelper;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -887,4 +888,53 @@ public class TestContainerAllocation {
 
     rm1.close();
   }
+
+
+
+  @Test(timeout = 60000)
+  public void testUserLimitAllocationMultipleContainers() throws Exception {
+    CapacitySchedulerConfiguration newConf =
+        (CapacitySchedulerConfiguration) TestUtils
+            .getConfigurationWithMultipleQueues(conf);
+    newConf.setUserLimit("root.c", 50);
+    MockRM rm1 = new MockRM(newConf);
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 1000 * GB);
+
+    // launch app from 1st user to queue C, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(2 * GB, "app", "user1", null, "c");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    // launch app from 2nd user to queue C, AM container should be launched in nm1
+    RMApp app2 = rm1.submitApp(2 * GB, "app", "user2", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
+
+    // Each application asks 1000 * 5GB containers
+    am1.allocate("*", 5 * GB, 1000, null);
+    am1.allocate("h1", 5 * GB, 1000, null);
+    am1.allocate(NetworkTopology.DEFAULT_RACK, 5 * GB, 1000, null);
+
+    // Each application asks 1000 * 5GB containers
+    am2.allocate("*", 5 * GB, 1000, null);
+    am2.allocate("h1", 5 * GB, 1000, null);
+    am2.allocate(NetworkTopology.DEFAULT_RACK, 5 * GB, 1000, null);
+
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+
+    FiCaSchedulerApp schedulerApp1 =
+        cs.getApplicationAttempt(am1.getApplicationAttemptId());
+    FiCaSchedulerApp schedulerApp2 =
+        cs.getApplicationAttempt(am2.getApplicationAttemptId());
+
+    // container will be allocated to am1
+    // App1 will get 2 container allocated (plus AM container)
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    Assert.assertEquals(101, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(100, schedulerApp2.getLiveContainers().size());
+
+    rm1.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c14ef8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
index d45f756..a32352b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
@@ -1252,7 +1252,7 @@ public class TestLeafQueue {
     //app4 is user 0
     //maxqueue 16G, userlimit 7G, used 8G, headroom 5G
     //(8G used is 6G from this test case - app4, 2 from last test case, app_1)
-    assertEquals(0*GB, app_4.getHeadroom().getMemorySize());
+    assertEquals(1*GB, app_4.getHeadroom().getMemorySize());
   }
 
   @Test
@@ -1436,7 +1436,7 @@ public class TestLeafQueue {
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
     // TODO, fix headroom in the future patch
-    assertEquals(0*GB, app_0.getHeadroom().getMemorySize());
+    assertEquals(1*GB, app_0.getHeadroom().getMemorySize());
       // User limit = 2G, 2 in use
     assertEquals(0*GB, app_1.getHeadroom().getMemorySize());
       // the application is not yet active
@@ -1449,8 +1449,8 @@ public class TestLeafQueue {
     assertEquals(3*GB, a.getUsedResources().getMemorySize());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize());
     assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize());
-    assertEquals(0*GB, app_0.getHeadroom().getMemorySize()); // 4G - 3G
-    assertEquals(0*GB, app_1.getHeadroom().getMemorySize()); // 4G - 3G
+    assertEquals(1*GB, app_0.getHeadroom().getMemorySize()); // 4G - 3G
+    assertEquals(1*GB, app_1.getHeadroom().getMemorySize()); // 4G - 3G
     
     // Submit requests for app_1 and set max-cap
     a.setMaxCapacity(.1f);


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


[35/50] [abbrv] hadoop git commit: YARN-7174. Add retry logic in LogsCLI when fetch running application logs. Contributed by Xuan Gong.

Posted by ae...@apache.org.
YARN-7174. Add retry logic in LogsCLI when fetch running application logs. Contributed by Xuan Gong.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1a84c24b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1a84c24b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1a84c24b

Branch: refs/heads/HDFS-7240
Commit: 1a84c24b0cf6674fa755403971fa57d8e412b320
Parents: 90894c7
Author: Junping Du <ju...@apache.org>
Authored: Fri Sep 15 15:33:24 2017 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Fri Sep 15 15:33:24 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  | 175 +++++++++++++++-
 .../hadoop/yarn/client/cli/TestLogsCLI.java     | 205 +++++++++++++------
 2 files changed, 309 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a84c24b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
index 1a3db26..9a8ba4a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
@@ -22,6 +22,9 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
+import java.net.ConnectException;
+import java.net.SocketException;
+import java.net.SocketTimeoutException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -75,9 +78,11 @@ import org.codehaus.jettison.json.JSONObject;
 import com.google.common.annotations.VisibleForTesting;
 import com.sun.jersey.api.client.Client;
 import com.sun.jersey.api.client.ClientHandlerException;
+import com.sun.jersey.api.client.ClientRequest;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.UniformInterfaceException;
 import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.filter.ClientFilter;
 
 @Public
 @Evolving
@@ -98,14 +103,27 @@ public class LogsCLI extends Configured implements Tool {
       = "show_container_log_info";
   private static final String OUT_OPTION = "out";
   private static final String SIZE_OPTION = "size";
+  private static final String CLIENT_MAX_RETRY_OPTION = "client_max_retries";
+  private static final String CLIENT_RETRY_INTERVAL_OPTION
+      = "client_retry_interval_ms";
   public static final String HELP_CMD = "help";
+
   private PrintStream outStream = System.out;
   private YarnClient yarnClient = null;
+  private Client webServiceClient = null;
+
+  private static final int DEFAULT_MAX_RETRIES = 30;
+  private static final long DEFAULT_RETRY_INTERVAL = 1000;
+
+  @Private
+  @VisibleForTesting
+  ClientConnectionRetry connectionRetry;
 
   @Override
   public int run(String[] args) throws Exception {
     try {
       yarnClient = createYarnClient();
+      webServiceClient = Client.create();
       return runCommand(args);
     } finally {
       if (yarnClient != null) {
@@ -140,6 +158,8 @@ public class LogsCLI extends Configured implements Tool {
     List<String> amContainersList = new ArrayList<String>();
     String localDir = null;
     long bytes = Long.MAX_VALUE;
+    int maxRetries = DEFAULT_MAX_RETRIES;
+    long retryInterval = DEFAULT_RETRY_INTERVAL;
     try {
       CommandLine commandLine = parser.parse(opts, args, false);
       appIdStr = commandLine.getOptionValue(APPLICATION_ID_OPTION);
@@ -171,6 +191,14 @@ public class LogsCLI extends Configured implements Tool {
       if (commandLine.hasOption(SIZE_OPTION)) {
         bytes = Long.parseLong(commandLine.getOptionValue(SIZE_OPTION));
       }
+      if (commandLine.hasOption(CLIENT_MAX_RETRY_OPTION)) {
+        maxRetries = Integer.parseInt(commandLine.getOptionValue(
+            CLIENT_MAX_RETRY_OPTION));
+      }
+      if (commandLine.hasOption(CLIENT_RETRY_INTERVAL_OPTION)) {
+        retryInterval = Long.parseLong(commandLine.getOptionValue(
+            CLIENT_RETRY_INTERVAL_OPTION));
+      }
     } catch (ParseException e) {
       System.err.println("options parsing failed: " + e.getMessage());
       printHelpMessage(printOpts);
@@ -232,6 +260,11 @@ public class LogsCLI extends Configured implements Tool {
       }
     }
 
+    // Set up Retry WebService Client
+    connectionRetry = new ClientConnectionRetry(maxRetries, retryInterval);
+    ClientJerseyRetryFilter retryFilter = new ClientJerseyRetryFilter();
+    webServiceClient.addFilter(retryFilter);
+
     LogCLIHelpers logCliHelper = new LogCLIHelpers();
     logCliHelper.setConf(getConf());
 
@@ -342,7 +375,6 @@ public class LogsCLI extends Configured implements Tool {
   protected List<JSONObject> getAMContainerInfoForRMWebService(
       Configuration conf, String appId) throws ClientHandlerException,
       UniformInterfaceException, JSONException {
-    Client webServiceClient = Client.create();
     String webAppAddress = WebAppUtils.getRMWebAppURLWithScheme(conf);
 
     WebResource webResource = webServiceClient.resource(webAppAddress);
@@ -364,7 +396,6 @@ public class LogsCLI extends Configured implements Tool {
   private List<JSONObject> getAMContainerInfoForAHSWebService(
       Configuration conf, String appId) throws ClientHandlerException,
       UniformInterfaceException, JSONException {
-    Client webServiceClient = Client.create();
     String webAppAddress =
         WebAppUtils.getHttpSchemePrefix(conf)
             + WebAppUtils.getAHSWebAppURLWithoutScheme(conf);
@@ -417,7 +448,6 @@ public class LogsCLI extends Configured implements Tool {
       throws IOException {
     List<Pair<ContainerLogFileInfo, String>> logFileInfos
         = new ArrayList<>();
-    Client webServiceClient = Client.create();
     try {
       WebResource webResource = webServiceClient
           .resource(WebAppUtils.getHttpSchemePrefix(conf) + nodeHttpAddress);
@@ -490,7 +520,6 @@ public class LogsCLI extends Configured implements Tool {
         lastModificationTime);
   }
 
-
   @Private
   @VisibleForTesting
   public int printContainerLogsFromRunningApplication(Configuration conf,
@@ -521,7 +550,6 @@ public class LogsCLI extends Configured implements Tool {
       ContainerLogsRequest newOptions = new ContainerLogsRequest(request);
       newOptions.setLogTypes(matchedFiles);
 
-      Client webServiceClient = Client.create();
       boolean foundAnyLogs = false;
       byte[] buffer = new byte[65536];
       for (String logFile : newOptions.getLogTypes()) {
@@ -797,6 +825,10 @@ public class LogsCLI extends Configured implements Tool {
     }
   }
 
+  /**
+   * Create Command Options.
+   * @return the command options
+   */
   private Options createCommandOpts() {
     Options opts = new Options();
     opts.addOption(HELP_CMD, false, "Displays help for all commands.");
@@ -859,6 +891,13 @@ public class LogsCLI extends Configured implements Tool {
     opts.addOption(SIZE_OPTION, true, "Prints the log file's first 'n' bytes "
         + "or the last 'n' bytes. Use negative values as bytes to read from "
         + "the end and positive values as bytes to read from the beginning.");
+    opts.addOption(CLIENT_MAX_RETRY_OPTION, true, "Set max retry number for a"
+        + " retry client to get the container logs for the running "
+        + "applications. Use a negative value to make retry forever. "
+        + "The default value is 30.");
+    opts.addOption(CLIENT_RETRY_INTERVAL_OPTION, true,
+        "Work with --client_max_retries to create a retry client. "
+        + "The default value is 1000.");
     opts.getOption(APPLICATION_ID_OPTION).setArgName("Application ID");
     opts.getOption(CONTAINER_ID_OPTION).setArgName("Container ID");
     opts.getOption(NODE_ADDRESS_OPTION).setArgName("Node Address");
@@ -866,9 +905,17 @@ public class LogsCLI extends Configured implements Tool {
     opts.getOption(AM_CONTAINER_OPTION).setArgName("AM Containers");
     opts.getOption(OUT_OPTION).setArgName("Local Directory");
     opts.getOption(SIZE_OPTION).setArgName("size");
+    opts.getOption(CLIENT_MAX_RETRY_OPTION).setArgName("Max Retries");
+    opts.getOption(CLIENT_RETRY_INTERVAL_OPTION)
+        .setArgName("Retry Interval");
     return opts;
   }
 
+  /**
+   * Create Print options for helper message.
+   * @param commandOpts the options
+   * @return the print options
+   */
   private Options createPrintOpts(Options commandOpts) {
     Options printOpts = new Options();
     printOpts.addOption(commandOpts.getOption(HELP_CMD));
@@ -884,6 +931,8 @@ public class LogsCLI extends Configured implements Tool {
     printOpts.addOption(commandOpts.getOption(SIZE_OPTION));
     printOpts.addOption(commandOpts.getOption(
         PER_CONTAINER_LOG_FILES_REGEX_OPTION));
+    printOpts.addOption(commandOpts.getOption(CLIENT_MAX_RETRY_OPTION));
+    printOpts.addOption(commandOpts.getOption(CLIENT_RETRY_INTERVAL_OPTION));
     return printOpts;
   }
 
@@ -1286,4 +1335,120 @@ public class LogsCLI extends Configured implements Tool {
     return nodeInfo.has("nodeHTTPAddress") ?
         nodeInfo.getString("nodeHTTPAddress") : null;
   }
+
+  // Class to handle retry
+  static class ClientConnectionRetry {
+
+    // maxRetries < 0 means keep trying
+    @Private
+    @VisibleForTesting
+    public int maxRetries;
+
+    @Private
+    @VisibleForTesting
+    public long retryInterval;
+
+    // Indicates if retries happened last time. Only tests should read it.
+    // In unit tests, retryOn() calls should _not_ be concurrent.
+    private boolean retried = false;
+
+    @Private
+    @VisibleForTesting
+    boolean getRetired() {
+      return retried;
+    }
+
+    // Constructor with default retry settings
+    public ClientConnectionRetry(int inputMaxRetries,
+        long inputRetryInterval) {
+      this.maxRetries = inputMaxRetries;
+      this.retryInterval = inputRetryInterval;
+    }
+
+    public Object retryOn(ClientRetryOp op)
+        throws RuntimeException, IOException {
+      int leftRetries = maxRetries;
+      retried = false;
+
+      // keep trying
+      while (true) {
+        try {
+          // try perform the op, if fail, keep retrying
+          return op.run();
+        } catch (IOException | RuntimeException e) {
+          // break if there's no retries left
+          if (leftRetries == 0) {
+            break;
+          }
+          if (op.shouldRetryOn(e)) {
+            logException(e, leftRetries);
+          } else {
+            throw e;
+          }
+        }
+        if (leftRetries > 0) {
+          leftRetries--;
+        }
+        retried = true;
+        try {
+          // sleep for the given time interval
+          Thread.sleep(retryInterval);
+        } catch (InterruptedException ie) {
+          System.out.println("Client retry sleep interrupted! ");
+        }
+      }
+      throw new RuntimeException("Connection retries limit exceeded.");
+    };
+
+    private void logException(Exception e, int leftRetries) {
+      if (leftRetries > 0) {
+        System.out.println("Exception caught by ClientConnectionRetry,"
+              + " will try " + leftRetries + " more time(s).\nMessage: "
+              + e.getMessage());
+      } else {
+        // note that maxRetries may be -1 at the very beginning
+        System.out.println("ConnectionException caught by ClientConnectionRetry,"
+            + " will keep retrying.\nMessage: "
+            + e.getMessage());
+      }
+    }
+  }
+
+  private class ClientJerseyRetryFilter extends ClientFilter {
+    @Override
+    public ClientResponse handle(final ClientRequest cr)
+        throws ClientHandlerException {
+      // Set up the retry operation
+      ClientRetryOp jerseyRetryOp = new ClientRetryOp() {
+        @Override
+        public Object run() {
+          // Try pass the request, if fail, keep retrying
+          return getNext().handle(cr);
+        }
+
+        @Override
+        public boolean shouldRetryOn(Exception e) {
+          // Only retry on connection exceptions
+          return (e instanceof ClientHandlerException)
+              && (e.getCause() instanceof ConnectException ||
+                  e.getCause() instanceof SocketTimeoutException ||
+                  e.getCause() instanceof SocketException);
+        }
+      };
+      try {
+        return (ClientResponse) connectionRetry.retryOn(jerseyRetryOp);
+      } catch (IOException e) {
+        throw new ClientHandlerException("Jersey retry failed!\nMessage: "
+              + e.getMessage());
+      }
+    }
+  }
+
+  // Abstract class for an operation that should be retried by client
+  private static abstract class ClientRetryOp {
+    // The operation that should be retried
+    public abstract Object run() throws IOException;
+    // The method to indicate if we should retry given the incoming exception
+    public abstract boolean shouldRetryOn(Exception e);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a84c24b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
index d58732e..fed7488 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
@@ -196,7 +196,7 @@ public class TestLogsCLI {
         "Unable to get ApplicationState"));
   }
 
-  @Test(timeout = 5000l)
+  @Test (timeout = 10000)
   public void testHelpMessage() throws Exception {
     Configuration conf = new YarnConfiguration();
     YarnClient mockYarnClient = createMockYarnClient(
@@ -207,79 +207,102 @@ public class TestLogsCLI {
 
     int exitCode = dumper.run(new String[]{});
     assertTrue(exitCode == -1);
+
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     PrintWriter pw = new PrintWriter(baos);
     pw.println("Retrieve logs for YARN applications.");
     pw.println("usage: yarn logs -applicationId <application ID> [OPTIONS]");
     pw.println();
     pw.println("general options are:");
-    pw.println(" -am <AM Containers>                     Prints the AM Container logs for");
-    pw.println("                                         this application. Specify");
-    pw.println("                                         comma-separated value to get logs");
-    pw.println("                                         for related AM Container. For");
-    pw.println("                                         example, If we specify -am 1,2,");
-    pw.println("                                         we will get the logs for the");
-    pw.println("                                         first AM Container as well as the");
-    pw.println("                                         second AM Container. To get logs");
-    pw.println("                                         for all AM Containers, use -am");
-    pw.println("                                         ALL. To get logs for the latest");
-    pw.println("                                         AM Container, use -am -1. By");
-    pw.println("                                         default, it will print all");
-    pw.println("                                         available logs. Work with");
-    pw.println("                                         -log_files to get only specific");
-    pw.println("                                         logs.");
-    pw.println(" -appOwner <Application Owner>           AppOwner (assumed to be current");
-    pw.println("                                         user if not specified)");
-    pw.println(" -containerId <Container ID>             ContainerId. By default, it will");
-    pw.println("                                         print all available logs. Work");
-    pw.println("                                         with -log_files to get only");
-    pw.println("                                         specific logs. If specified, the");
-    pw.println("                                         applicationId can be omitted");
-    pw.println(" -help                                   Displays help for all commands.");
-    pw.println(" -list_nodes                             Show the list of nodes that");
-    pw.println("                                         successfully aggregated logs.");
-    pw.println("                                         This option can only be used with");
-    pw.println("                                         finished applications.");
-    pw.println(" -log_files <Log File Name>              Specify comma-separated value to");
-    pw.println("                                         get exact matched log files. Use");
-    pw.println("                                         \"ALL\" or \"*\" to fetch all the log");
-    pw.println("                                         files for the container.");
-    pw.println(" -log_files_pattern <Log File Pattern>   Specify comma-separated value to");
-    pw.println("                                         get matched log files by using");
-    pw.println("                                         java regex. Use \".*\" to fetch all");
-    pw.println("                                         the log files for the container.");
-    pw.println(" -nodeAddress <Node Address>             NodeAddress in the format");
-    pw.println("                                         nodename:port");
-    pw.println(" -out <Local Directory>                  Local directory for storing");
-    pw.println("                                         individual container logs. The");
-    pw.println("                                         container logs will be stored");
-    pw.println("                                         based on the node the container");
-    pw.println("                                         ran on.");
-    pw.println(" -show_application_log_info              Show the containerIds which");
-    pw.println("                                         belong to the specific");
-    pw.println("                                         Application. You can combine this");
-    pw.println("                                         with --nodeAddress to get");
-    pw.println("                                         containerIds for all the");
-    pw.println("                                         containers on the specific");
-    pw.println("                                         NodeManager.");
-    pw.println(" -show_container_log_info                Show the container log metadata,");
-    pw.println("                                         including log-file names, the");
-    pw.println("                                         size of the log files. You can");
-    pw.println("                                         combine this with --containerId");
-    pw.println("                                         to get log metadata for the");
-    pw.println("                                         specific container, or with");
-    pw.println("                                         --nodeAddress to get log metadata");
-    pw.println("                                         for all the containers on the");
-    pw.println("                                         specific NodeManager.");
-    pw.println(" -size <size>                            Prints the log file's first 'n'");
-    pw.println("                                         bytes or the last 'n' bytes. Use");
-    pw.println("                                         negative values as bytes to read");
-    pw.println("                                         from the end and positive values");
-    pw.println("                                         as bytes to read from the");
-    pw.println("                                         beginning.");
+    pw.println(" -am <AM Containers>                          Prints the AM Container logs");
+    pw.println("                                              for this application.");
+    pw.println("                                              Specify comma-separated");
+    pw.println("                                              value to get logs for");
+    pw.println("                                              related AM Container. For");
+    pw.println("                                              example, If we specify -am");
+    pw.println("                                              1,2, we will get the logs");
+    pw.println("                                              for the first AM Container");
+    pw.println("                                              as well as the second AM");
+    pw.println("                                              Container. To get logs for");
+    pw.println("                                              all AM Containers, use -am");
+    pw.println("                                              ALL. To get logs for the");
+    pw.println("                                              latest AM Container, use -am");
+    pw.println("                                              -1. By default, it will");
+    pw.println("                                              print all available logs.");
+    pw.println("                                              Work with -log_files to get");
+    pw.println("                                              only specific logs.");
+    pw.println(" -appOwner <Application Owner>                AppOwner (assumed to be");
+    pw.println("                                              current user if not");
+    pw.println("                                              specified)");
+    pw.println(" -client_max_retries <Max Retries>            Set max retry number for a");
+    pw.println("                                              retry client to get the");
+    pw.println("                                              container logs for the");
+    pw.println("                                              running applications. Use a");
+    pw.println("                                              negative value to make retry");
+    pw.println("                                              forever. The default value");
+    pw.println("                                              is 30.");
+    pw.println(" -client_retry_interval_ms <Retry Interval>   Work with");
+    pw.println("                                              --client_max_retries to");
+    pw.println("                                              create a retry client. The");
+    pw.println("                                              default value is 1000.");
+    pw.println(" -containerId <Container ID>                  ContainerId. By default, it");
+    pw.println("                                              will print all available");
+    pw.println("                                              logs. Work with -log_files");
+    pw.println("                                              to get only specific logs.");
+    pw.println("                                              If specified, the");
+    pw.println("                                              applicationId can be omitted");
+    pw.println(" -help                                        Displays help for all");
+    pw.println("                                              commands.");
+    pw.println(" -list_nodes                                  Show the list of nodes that");
+    pw.println("                                              successfully aggregated");
+    pw.println("                                              logs. This option can only");
+    pw.println("                                              be used with finished");
+    pw.println("                                              applications.");
+    pw.println(" -log_files <Log File Name>                   Specify comma-separated");
+    pw.println("                                              value to get exact matched");
+    pw.println("                                              log files. Use \"ALL\" or \"*\"");
+    pw.println("                                              to fetch all the log files");
+    pw.println("                                              for the container.");
+    pw.println(" -log_files_pattern <Log File Pattern>        Specify comma-separated");
+    pw.println("                                              value to get matched log");
+    pw.println("                                              files by using java regex.");
+    pw.println("                                              Use \".*\" to fetch all the");
+    pw.println("                                              log files for the container.");
+    pw.println(" -nodeAddress <Node Address>                  NodeAddress in the format");
+    pw.println("                                              nodename:port");
+    pw.println(" -out <Local Directory>                       Local directory for storing");
+    pw.println("                                              individual container logs.");
+    pw.println("                                              The container logs will be");
+    pw.println("                                              stored based on the node the");
+    pw.println("                                              container ran on.");
+    pw.println(" -show_application_log_info                   Show the containerIds which");
+    pw.println("                                              belong to the specific");
+    pw.println("                                              Application. You can combine");
+    pw.println("                                              this with --nodeAddress to");
+    pw.println("                                              get containerIds for all the");
+    pw.println("                                              containers on the specific");
+    pw.println("                                              NodeManager.");
+    pw.println(" -show_container_log_info                     Show the container log");
+    pw.println("                                              metadata, including log-file");
+    pw.println("                                              names, the size of the log");
+    pw.println("                                              files. You can combine this");
+    pw.println("                                              with --containerId to get");
+    pw.println("                                              log metadata for the");
+    pw.println("                                              specific container, or with");
+    pw.println("                                              --nodeAddress to get log");
+    pw.println("                                              metadata for all the");
+    pw.println("                                              containers on the specific");
+    pw.println("                                              NodeManager.");
+    pw.println(" -size <size>                                 Prints the log file's first");
+    pw.println("                                              'n' bytes or the last 'n'");
+    pw.println("                                              bytes. Use negative values");
+    pw.println("                                              as bytes to read from the");
+    pw.println("                                              end and positive values as");
+    pw.println("                                              bytes to read from the");
+    pw.println("                                              beginning.");
     pw.close();
     String appReportStr = baos.toString("UTF-8");
-    Assert.assertEquals(appReportStr, sysOutStream.toString());
+    Assert.assertTrue(sysOutStream.toString().contains(appReportStr));
   }
 
   @Test (timeout = 15000)
@@ -609,6 +632,56 @@ public class TestLogsCLI {
     fs.delete(new Path(rootLogDir), true);
   }
 
+  @Test
+  public void testCheckRetryCount() throws Exception {
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+
+    NodeId nodeId = NodeId.newInstance("localhost", 1234);
+    ApplicationId appId = ApplicationId.newInstance(0, 1);
+    ApplicationAttemptId appAttemptId = ApplicationAttemptId
+        .newInstance(appId, 1);
+
+    // Create a mock ApplicationAttempt Report
+    ApplicationAttemptReport mockAttemptReport = mock(
+        ApplicationAttemptReport.class);
+    doReturn(appAttemptId).when(mockAttemptReport).getApplicationAttemptId();
+    List<ApplicationAttemptReport> attemptReports = Arrays.asList(
+        mockAttemptReport);
+
+    // Create one mock containerReport
+    ContainerId containerId1 = ContainerId.newContainerId(appAttemptId, 1);
+    ContainerReport mockContainerReport1 = mock(ContainerReport.class);
+    doReturn(containerId1).when(mockContainerReport1).getContainerId();
+    doReturn(nodeId).when(mockContainerReport1).getAssignedNode();
+    doReturn("http://localhost:2345").when(mockContainerReport1)
+        .getNodeHttpAddress();
+    doReturn(ContainerState.RUNNING).when(mockContainerReport1)
+        .getContainerState();
+    List<ContainerReport> containerReports = Arrays.asList(
+        mockContainerReport1);
+    // Mock the YarnClient, and it would report the previous created
+    // mockAttemptReport and previous two created mockContainerReports
+    YarnClient mockYarnClient = createMockYarnClient(
+        YarnApplicationState.RUNNING, ugi.getShortUserName(), true,
+        attemptReports, containerReports);
+    doReturn(mockContainerReport1).when(mockYarnClient).getContainerReport(
+        any(ContainerId.class));
+    LogsCLI cli = new LogsCLIForTest(mockYarnClient);
+    cli.setConf(new YarnConfiguration());
+    try {
+      cli.run(new String[] {"-containerId",
+          containerId1.toString(), "-client_max_retries", "5"});
+      Assert.fail("Exception expected! "
+          + "NodeManager should be off to run this test. ");
+    } catch (RuntimeException ce) {
+      Assert.assertTrue(
+          "Handler exception for reason other than retry: " + ce.getMessage(),
+          ce.getMessage().contains("Connection retries limit exceeded"));
+      Assert.assertTrue("Retry filter didn't perform any retries! ", cli
+           .connectionRetry.getRetired());
+    }
+  }
+
   @Test (timeout = 5000)
   public void testGetRunningContainerLogs() throws Exception {
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();


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


[19/50] [abbrv] hadoop git commit: MAPREDUCE-6956 FileOutputCommitter to gain abstract superclass PathOutputCommitter. Contributed by Steve Loughran

Posted by ae...@apache.org.
MAPREDUCE-6956 FileOutputCommitter to gain abstract superclass PathOutputCommitter.
Contributed by Steve Loughran


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/11390c2d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/11390c2d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/11390c2d

Branch: refs/heads/HDFS-7240
Commit: 11390c2d111910b01d9c4d3e39dee49babae272f
Parents: 78bdf10
Author: Steve Loughran <st...@apache.org>
Authored: Fri Sep 15 16:59:04 2017 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Sep 15 16:59:04 2017 +0100

----------------------------------------------------------------------
 .../lib/output/FileOutputCommitter.java         |  25 +-
 .../mapreduce/lib/output/FileOutputFormat.java  |  57 ++-
 .../lib/output/PathOutputCommitter.java         |  91 +++++
 .../hadoop/mapreduce/task/JobContextImpl.java   |  10 +-
 .../mapreduce/task/TaskAttemptContextImpl.java  |  13 +-
 .../lib/output/TestPathOutputCommitter.java     | 377 +++++++++++++++++++
 6 files changed, 553 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/11390c2d/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java
index 9e750be..0061406 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputCommitter.java
@@ -39,13 +39,14 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 
 /** An {@link OutputCommitter} that commits files specified 
  * in job output directory i.e. ${mapreduce.output.fileoutputformat.outputdir}.
  **/
 @InterfaceAudience.Public
 @InterfaceStability.Stable
-public class FileOutputCommitter extends OutputCommitter {
+public class FileOutputCommitter extends PathOutputCommitter {
   private static final Log LOG = LogFactory.getLog(FileOutputCommitter.class);
 
   /** 
@@ -101,8 +102,11 @@ public class FileOutputCommitter extends OutputCommitter {
   public FileOutputCommitter(Path outputPath, 
                              TaskAttemptContext context) throws IOException {
     this(outputPath, (JobContext)context);
-    if (outputPath != null) {
-      workPath = getTaskAttemptPath(context, outputPath);
+    if (getOutputPath() != null) {
+      workPath = Preconditions.checkNotNull(
+          getTaskAttemptPath(context, getOutputPath()),
+          "Null task attempt path in %s and output path %s",
+          context, outputPath);
     }
   }
   
@@ -116,6 +120,7 @@ public class FileOutputCommitter extends OutputCommitter {
   @Private
   public FileOutputCommitter(Path outputPath, 
                              JobContext context) throws IOException {
+    super(outputPath, context);
     Configuration conf = context.getConfiguration();
     algorithmVersion =
         conf.getInt(FILEOUTPUTCOMMITTER_ALGORITHM_VERSION,
@@ -705,4 +710,18 @@ public class FileOutputCommitter extends OutputCommitter {
       LOG.warn("Output Path is null in recoverTask()");
     }
   }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "FileOutputCommitter{");
+    sb.append(super.toString()).append("; ");
+    sb.append("outputPath=").append(outputPath);
+    sb.append(", workPath=").append(workPath);
+    sb.append(", algorithmVersion=").append(algorithmVersion);
+    sb.append(", skipCleanup=").append(skipCleanup);
+    sb.append(", ignoreCleanupFailures=").append(ignoreCleanupFailures);
+    sb.append('}');
+    return sb.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11390c2d/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java
index c11f8d8..0e7efa3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.mapreduce.lib.output;
 import java.io.IOException;
 import java.text.NumberFormat;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -38,11 +39,15 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskInputOutputContext;
 import org.apache.hadoop.mapreduce.security.TokenCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /** A base class for {@link OutputFormat}s that read from {@link FileSystem}s.*/
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public abstract class FileOutputFormat<K, V> extends OutputFormat<K, V> {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FileOutputFormat.class);
 
   /** Construct output file names so that, when an output directory listing is
    * sorted lexicographically, positions correspond to output partitions.*/
@@ -53,12 +58,25 @@ public abstract class FileOutputFormat<K, V> extends OutputFormat<K, V> {
     NUMBER_FORMAT.setMinimumIntegerDigits(5);
     NUMBER_FORMAT.setGroupingUsed(false);
   }
-  private FileOutputCommitter committer = null;
-public static final String COMPRESS ="mapreduce.output.fileoutputformat.compress";
-public static final String COMPRESS_CODEC = 
-"mapreduce.output.fileoutputformat.compress.codec";
-public static final String COMPRESS_TYPE = "mapreduce.output.fileoutputformat.compress.type";
-public static final String OUTDIR = "mapreduce.output.fileoutputformat.outputdir";
+  private PathOutputCommitter committer = null;
+
+  /** Configuration option: should output be compressed? {@value}. */
+  public static final String COMPRESS =
+      "mapreduce.output.fileoutputformat.compress";
+
+  /** If compression is enabled, name of codec: {@value}. */
+  public static final String COMPRESS_CODEC =
+      "mapreduce.output.fileoutputformat.compress.codec";
+  /**
+   * Type of compression {@value}: NONE, RECORD, BLOCK.
+   * Generally only used in {@code SequenceFileOutputFormat}.
+   */
+  public static final String COMPRESS_TYPE =
+      "mapreduce.output.fileoutputformat.compress.type";
+
+  /** Destination directory of work: {@value}. */
+  public static final String OUTDIR =
+      "mapreduce.output.fileoutputformat.outputdir";
 
   @Deprecated
   public enum Counter {
@@ -110,14 +128,14 @@ public static final String OUTDIR = "mapreduce.output.fileoutputformat.outputdir
    */
   public static Class<? extends CompressionCodec> 
   getOutputCompressorClass(JobContext job, 
-		                       Class<? extends CompressionCodec> defaultValue) {
+                       Class<? extends CompressionCodec> defaultValue) {
     Class<? extends CompressionCodec> codecClass = defaultValue;
     Configuration conf = job.getConfiguration();
     String name = conf.get(FileOutputFormat.COMPRESS_CODEC);
     if (name != null) {
       try {
-        codecClass = 
-        	conf.getClassByName(name).asSubclass(CompressionCodec.class);
+        codecClass =
+            conf.getClassByName(name).asSubclass(CompressionCodec.class);
       } catch (ClassNotFoundException e) {
         throw new IllegalArgumentException("Compression codec " + name + 
                                            " was not found.", e);
@@ -219,9 +237,11 @@ public static final String OUTDIR = "mapreduce.output.fileoutputformat.outputdir
   public static Path getWorkOutputPath(TaskInputOutputContext<?,?,?,?> context
                                        ) throws IOException, 
                                                 InterruptedException {
-    FileOutputCommitter committer = (FileOutputCommitter) 
+    PathOutputCommitter committer = (PathOutputCommitter)
       context.getOutputCommitter();
-    return committer.getWorkPath();
+    Path workPath = committer.getWorkPath();
+    LOG.debug("Work path is {}", workPath);
+    return workPath;
   }
 
   /**
@@ -281,10 +301,17 @@ public static final String OUTDIR = "mapreduce.output.fileoutputformat.outputdir
    */
   public Path getDefaultWorkFile(TaskAttemptContext context,
                                  String extension) throws IOException{
-    FileOutputCommitter committer = 
-      (FileOutputCommitter) getOutputCommitter(context);
-    return new Path(committer.getWorkPath(), getUniqueFile(context, 
-      getOutputName(context), extension));
+    OutputCommitter c = getOutputCommitter(context);
+    Preconditions.checkState(c instanceof PathOutputCommitter,
+        "Committer %s is not a PathOutputCommitter", c);
+    Path workPath = ((PathOutputCommitter) c).getWorkPath();
+    Preconditions.checkNotNull(workPath,
+        "Null workPath returned by committer %s", c);
+    Path workFile = new Path(workPath,
+        getUniqueFile(context, getOutputName(context), extension));
+    LOG.debug("Work file for {} extension '{}' is {}",
+        context, extension, workFile);
+    return workFile;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11390c2d/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitter.java
new file mode 100644
index 0000000..2df30ba
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitter.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.mapreduce.lib.output;
+
+import java.io.IOException;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * A committer which somehow commits data written to a working directory
+ * to the final directory during the commit process. The reference
+ * implementation of this is the {@link FileOutputCommitter}.
+ *
+ * There are two constructors, both of which do nothing but long and
+ * validate their arguments.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class PathOutputCommitter extends OutputCommitter {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(PathOutputCommitter.class);
+
+  private final JobContext context;
+
+  /**
+   * Constructor for a task attempt.
+   * Subclasses should provide a public constructor with this signature.
+   * @param outputPath output path: may be null
+   * @param context task context
+   * @throws IOException IO problem
+   */
+  protected PathOutputCommitter(Path outputPath,
+      TaskAttemptContext context) throws IOException {
+    this.context = Preconditions.checkNotNull(context, "Null context");
+    LOG.debug("Creating committer with output path {} and task context"
+        + " {}", outputPath, context);
+  }
+
+  /**
+   * Constructor for a job attempt.
+   * Subclasses should provide a public constructor with this signature.
+   * @param outputPath output path: may be null
+   * @param context task context
+   * @throws IOException IO problem
+   */
+  protected PathOutputCommitter(Path outputPath,
+      JobContext context) throws IOException {
+    this.context = Preconditions.checkNotNull(context, "Null context");
+    LOG.debug("Creating committer with output path {} and job context"
+        + " {}", outputPath, context);
+  }
+
+  /**
+   * Get the directory that the task should write results into.
+   * Warning: there's no guarantee that this work path is on the same
+   * FS as the final output, or that it's visible across machines.
+   * @return the work directory
+   * @throws IOException IO problem
+   */
+  public abstract Path getWorkPath() throws IOException;
+
+  @Override
+  public String toString() {
+    return "PathOutputCommitter{context=" + context + '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11390c2d/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/JobContextImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/JobContextImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/JobContextImpl.java
index b9014ef..1696246 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/JobContextImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/JobContextImpl.java
@@ -451,5 +451,13 @@ public class JobContextImpl implements JobContext {
   public Credentials getCredentials() {
     return credentials;
   }
-  
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "JobContextImpl{");
+    sb.append("jobId=").append(jobId);
+    sb.append('}');
+    return sb.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11390c2d/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/TaskAttemptContextImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/TaskAttemptContextImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/TaskAttemptContextImpl.java
index 333f57b..a622d3a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/TaskAttemptContextImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/TaskAttemptContextImpl.java
@@ -118,4 +118,15 @@ public class TaskAttemptContextImpl extends JobContextImpl
   public float getProgress() {
     return reporter.getProgress();
   }
-}
\ No newline at end of file
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "TaskAttemptContextImpl{");
+    sb.append(super.toString());
+    sb.append("; taskId=").append(taskId);
+    sb.append(", status='").append(status).append('\'');
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/11390c2d/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestPathOutputCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestPathOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestPathOutputCommitter.java
new file mode 100644
index 0000000..9cff82f
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestPathOutputCommitter.java
@@ -0,0 +1,377 @@
+/*
+ * 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.mapreduce.lib.output;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.Partitioner;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskInputOutputContext;
+import org.apache.hadoop.security.Credentials;
+
+/**
+ * Test the path output committer binding to FileOutputFormat.
+ */
+public class TestPathOutputCommitter extends Assert {
+
+  @Test
+  public void testFileOutputCommitterOverrride() throws Throwable {
+    TaskContext context = new TaskContext();
+    Path workPath = new Path("file:///work");
+    context.setOutputCommitter(
+        new SimpleCommitter(new Path("/"), context, workPath));
+    assertEquals(workPath, FileOutputFormat.getWorkOutputPath(context));
+  }
+
+  @Test
+  public void testFileOutputCommitterNullWorkPath() throws Throwable {
+    TaskContext context = new TaskContext();
+    context.setOutputCommitter(
+        new SimpleCommitter(new Path("/"), context, null));
+    assertNull(FileOutputFormat.getWorkOutputPath(context));
+  }
+
+  private static class SimpleCommitter extends PathOutputCommitter {
+
+    private final Path workPath;
+
+    SimpleCommitter(Path outputPath,
+        TaskAttemptContext context, Path workPath) throws IOException {
+      super(outputPath, context);
+      this.workPath = workPath;
+    }
+
+    SimpleCommitter(Path outputPath,
+        JobContext context, Path workPath) throws IOException {
+      super(outputPath, context);
+      this.workPath = workPath;
+    }
+
+    @Override
+    public Path getWorkPath() throws IOException {
+      return workPath;
+    }
+
+    @Override
+    public void setupJob(JobContext jobContext) throws IOException {
+
+    }
+
+    @Override
+    public void setupTask(TaskAttemptContext taskContext) throws IOException {
+
+    }
+
+    @Override
+    public boolean needsTaskCommit(TaskAttemptContext taskContext)
+        throws IOException {
+      return false;
+    }
+
+    @Override
+    public void commitTask(TaskAttemptContext taskContext) throws IOException {
+
+    }
+
+    @Override
+    public void abortTask(TaskAttemptContext taskContext) throws IOException {
+
+    }
+  }
+
+  /**
+   * Stub task context.
+   */
+  public class TaskContext
+      implements TaskInputOutputContext<String, String, String, String> {
+
+    private OutputCommitter outputCommitter;
+
+    public void setOutputCommitter(OutputCommitter outputCommitter) {
+      this.outputCommitter = outputCommitter;
+    }
+
+    @Override
+    public OutputCommitter getOutputCommitter() {
+      return outputCommitter;
+    }
+
+    @Override
+    public boolean nextKeyValue() throws IOException, InterruptedException {
+      return false;
+    }
+
+    @Override
+    public String getCurrentKey() throws IOException, InterruptedException {
+      return null;
+    }
+
+    @Override
+    public String getCurrentValue() throws IOException, InterruptedException {
+      return null;
+    }
+
+    @Override
+    public void write(String key, String value)
+        throws IOException, InterruptedException {
+    }
+
+
+    @Override
+    public TaskAttemptID getTaskAttemptID() {
+      return null;
+    }
+
+    @Override
+    public void setStatus(String msg) {
+    }
+
+    @Override
+    public String getStatus() {
+      return null;
+    }
+
+    @Override
+    public float getProgress() {
+      return 0;
+    }
+
+    @Override
+    public Counter getCounter(Enum<?> counterName) {
+      return null;
+    }
+
+    @Override
+    public Counter getCounter(String groupName, String counterName) {
+      return null;
+    }
+
+    @Override
+    public Configuration getConfiguration() {
+      return null;
+    }
+
+    @Override
+    public Credentials getCredentials() {
+      return null;
+    }
+
+    @Override
+    public JobID getJobID() {
+      return null;
+    }
+
+    @Override
+    public int getNumReduceTasks() {
+      return 0;
+    }
+
+    @Override
+    public Path getWorkingDirectory() throws IOException {
+      return null;
+    }
+
+    @Override
+    public Class<?> getOutputKeyClass() {
+      return null;
+    }
+
+    @Override
+    public Class<?> getOutputValueClass() {
+      return null;
+    }
+
+    @Override
+    public Class<?> getMapOutputKeyClass() {
+      return null;
+    }
+
+    @Override
+    public Class<?> getMapOutputValueClass() {
+      return null;
+    }
+
+    @Override
+    public String getJobName() {
+      return null;
+    }
+
+    @Override
+    public Class<? extends InputFormat<?, ?>> getInputFormatClass()
+        throws ClassNotFoundException {
+      return null;
+    }
+
+    @Override
+    public Class<? extends Mapper<?, ?, ?, ?>> getMapperClass()
+        throws ClassNotFoundException {
+      return null;
+    }
+
+    @Override
+    public Class<? extends Reducer<?, ?, ?, ?>> getCombinerClass()
+        throws ClassNotFoundException {
+      return null;
+    }
+
+    @Override
+    public Class<? extends Reducer<?, ?, ?, ?>> getReducerClass()
+        throws ClassNotFoundException {
+      return null;
+    }
+
+    @Override
+    public Class<? extends OutputFormat<?, ?>> getOutputFormatClass()
+        throws ClassNotFoundException {
+      return null;
+    }
+
+    @Override
+    public Class<? extends Partitioner<?, ?>> getPartitionerClass()
+        throws ClassNotFoundException {
+      return null;
+    }
+
+    @Override
+    public RawComparator<?> getSortComparator() {
+      return null;
+    }
+
+    @Override
+    public String getJar() {
+      return null;
+    }
+
+    @Override
+    public RawComparator<?> getCombinerKeyGroupingComparator() {
+      return null;
+    }
+
+    @Override
+    public RawComparator<?> getGroupingComparator() {
+      return null;
+    }
+
+    @Override
+    public boolean getJobSetupCleanupNeeded() {
+      return false;
+    }
+
+    @Override
+    public boolean getTaskCleanupNeeded() {
+      return false;
+    }
+
+    @Override
+    public boolean getProfileEnabled() {
+      return false;
+    }
+
+    @Override
+    public String getProfileParams() {
+      return null;
+    }
+
+    @Override
+    public Configuration.IntegerRanges getProfileTaskRange(boolean isMap) {
+      return null;
+    }
+
+    @Override
+    public String getUser() {
+      return null;
+    }
+
+    @Override
+    public boolean getSymlink() {
+      return false;
+    }
+
+    @Override
+    public Path[] getArchiveClassPaths() {
+      return new Path[0];
+    }
+
+    @Override
+    public URI[] getCacheArchives() throws IOException {
+      return new URI[0];
+    }
+
+    @Override
+    public URI[] getCacheFiles() throws IOException {
+      return new URI[0];
+    }
+
+    @Override
+    public Path[] getLocalCacheArchives() throws IOException {
+      return new Path[0];
+    }
+
+    @Override
+    public Path[] getLocalCacheFiles() throws IOException {
+      return new Path[0];
+    }
+
+    @Override
+    public Path[] getFileClassPaths() {
+      return new Path[0];
+    }
+
+    @Override
+    public String[] getArchiveTimestamps() {
+      return new String[0];
+    }
+
+    @Override
+    public String[] getFileTimestamps() {
+      return new String[0];
+    }
+
+    @Override
+    public int getMaxMapAttempts() {
+      return 0;
+    }
+
+    @Override
+    public int getMaxReduceAttempts() {
+      return 0;
+    }
+
+    @Override
+    public void progress() {
+    }
+  }
+
+}


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


[20/50] [abbrv] hadoop git commit: HADOOP-14553. Add (parallelized) integration tests to hadoop-azure Contributed by Steve Loughran

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestAzureFileSystemInstrumentation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestAzureFileSystemInstrumentation.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestAzureFileSystemInstrumentation.java
deleted file mode 100644
index 818a844..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/metrics/TestAzureFileSystemInstrumentation.java
+++ /dev/null
@@ -1,579 +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.azure.metrics;
-
-import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_CLIENT_ERRORS;
-import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_DIRECTORIES_CREATED;
-import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_DOWNLOAD_LATENCY;
-import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_DOWNLOAD_RATE;
-import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_FILES_CREATED;
-import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_FILES_DELETED;
-import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_SERVER_ERRORS;
-import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_UPLOAD_LATENCY;
-import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_UPLOAD_RATE;
-import static org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation.WASB_WEB_RESPONSES;
-import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
-import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assume.assumeNotNull;
-import static org.mockito.Matchers.argThat;
-import static org.mockito.Mockito.verify;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Arrays;
-import java.util.Date;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
-import org.apache.hadoop.fs.azure.AzureException;
-import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore;
-import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.metrics2.MetricsRecordBuilder;
-import org.apache.hadoop.metrics2.MetricsTag;
-import org.hamcrest.BaseMatcher;
-import org.hamcrest.Description;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestAzureFileSystemInstrumentation {
-  private FileSystem fs;
-  private AzureBlobStorageTestAccount testAccount;
-
-  @Before
-  public void setUp() throws Exception {
-    testAccount = AzureBlobStorageTestAccount.create();
-    if (testAccount != null) {
-      fs = testAccount.getFileSystem();
-    }
-    assumeNotNull(testAccount);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (testAccount != null) {
-      testAccount.cleanup();
-      testAccount = null;
-      fs = null;
-    }
-  }
-
-  @Test
-  public void testMetricTags() throws Exception {
-    String accountName =
-        testAccount.getRealAccount().getBlobEndpoint()
-        .getAuthority();
-    String containerName =
-        testAccount.getRealContainer().getName();
-    MetricsRecordBuilder myMetrics = getMyMetrics();
-    verify(myMetrics).add(argThat(
-        new TagMatcher("accountName", accountName)
-        ));
-    verify(myMetrics).add(argThat(
-        new TagMatcher("containerName", containerName)
-        ));
-    verify(myMetrics).add(argThat(
-        new TagMatcher("Context", "azureFileSystem")
-        ));
-    verify(myMetrics).add(argThat(
-        new TagExistsMatcher("wasbFileSystemId")
-        ));
-  }
-  
-
-  @Test
-  public void testMetricsOnMkdirList() throws Exception {
-    long base = getBaseWebResponses();
-
-    // Create a directory
-    assertTrue(fs.mkdirs(new Path("a")));
-    // At the time of writing
-    // getAncestor uses 2 calls for each folder level /user/<name>/a
-    // plus 1 call made by checkContainer
-    // mkdir checks the hierarchy with 2 calls per level
-    // mkdirs calls storeEmptyDir to create the empty folder, which makes 5 calls
-    // For a total of 7 + 6 + 5 = 18 web responses
-    base = assertWebResponsesInRange(base, 1, 18);
-    assertEquals(1,
-        AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_DIRECTORIES_CREATED));
-
-    // List the root contents
-    assertEquals(1, fs.listStatus(new Path("/")).length);
-    base = assertWebResponsesEquals(base, 1);
-
-    assertNoErrors();
-  }
-
-  private BandwidthGaugeUpdater getBandwidthGaugeUpdater() {
-    NativeAzureFileSystem azureFs = (NativeAzureFileSystem)fs;
-    AzureNativeFileSystemStore azureStore = azureFs.getStore();
-    return azureStore.getBandwidthGaugeUpdater();
-  }
-
-  private static byte[] nonZeroByteArray(int size) {
-    byte[] data = new byte[size];
-    Arrays.fill(data, (byte)5);
-    return data;
-  }
-
-  @Test
-  public void testMetricsOnFileCreateRead() throws Exception {
-    long base = getBaseWebResponses();
-
-    assertEquals(0, AzureMetricsTestUtil.getCurrentBytesWritten(getInstrumentation()));
-
-    Path filePath = new Path("/metricsTest_webResponses");
-    final int FILE_SIZE = 1000;
-
-    // Suppress auto-update of bandwidth metrics so we get
-    // to update them exactly when we want to.
-    getBandwidthGaugeUpdater().suppressAutoUpdate();
-
-    // Create a file
-    Date start = new Date();
-    OutputStream outputStream = fs.create(filePath);
-    outputStream.write(nonZeroByteArray(FILE_SIZE));
-    outputStream.close();
-    long uploadDurationMs = new Date().getTime() - start.getTime();
-
-    // The exact number of requests/responses that happen to create a file
-    // can vary  - at the time of writing this code it takes 10
-    // requests/responses for the 1000 byte file (33 for 100 MB),
-    // plus the initial container-check request but that
-    // can very easily change in the future. Just assert that we do roughly
-    // more than 2 but less than 15.
-    logOpResponseCount("Creating a 1K file", base);
-    base = assertWebResponsesInRange(base, 2, 15);
-    getBandwidthGaugeUpdater().triggerUpdate(true);
-    long bytesWritten = AzureMetricsTestUtil.getCurrentBytesWritten(getInstrumentation());
-    assertTrue("The bytes written in the last second " + bytesWritten +
-        " is pretty far from the expected range of around " + FILE_SIZE +
-        " bytes plus a little overhead.",
-        bytesWritten > (FILE_SIZE / 2) && bytesWritten < (FILE_SIZE * 2));
-    long totalBytesWritten = AzureMetricsTestUtil.getCurrentTotalBytesWritten(getInstrumentation());
-    assertTrue("The total bytes written  " + totalBytesWritten +
-        " is pretty far from the expected range of around " + FILE_SIZE +
-        " bytes plus a little overhead.",
-        totalBytesWritten >= FILE_SIZE && totalBytesWritten < (FILE_SIZE * 2));
-    long uploadRate = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(), WASB_UPLOAD_RATE);
-    System.out.println("Upload rate: " + uploadRate + " bytes/second.");
-    long expectedRate = (FILE_SIZE * 1000L) / uploadDurationMs;
-    assertTrue("The upload rate " + uploadRate +
-        " is below the expected range of around " + expectedRate +
-        " bytes/second that the unit test observed. This should never be" +
-        " the case since the test underestimates the rate by looking at " +
-        " end-to-end time instead of just block upload time.",
-        uploadRate >= expectedRate);
-    long uploadLatency = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(),
-        WASB_UPLOAD_LATENCY);
-    System.out.println("Upload latency: " + uploadLatency);
-    long expectedLatency = uploadDurationMs; // We're uploading less than a block.
-    assertTrue("The upload latency " + uploadLatency +
-        " should be greater than zero now that I've just uploaded a file.",
-        uploadLatency > 0);
-    assertTrue("The upload latency " + uploadLatency +
-        " is more than the expected range of around " + expectedLatency +
-        " milliseconds that the unit test observed. This should never be" +
-        " the case since the test overestimates the latency by looking at " +
-        " end-to-end time instead of just block upload time.",
-        uploadLatency <= expectedLatency);
-
-    // Read the file
-    start = new Date();
-    InputStream inputStream = fs.open(filePath);
-    int count = 0;
-    while (inputStream.read() >= 0) {
-      count++;
-    }
-    inputStream.close();
-    long downloadDurationMs = new Date().getTime() - start.getTime();
-    assertEquals(FILE_SIZE, count);
-
-    // Again, exact number varies. At the time of writing this code
-    // it takes 4 request/responses, so just assert a rough range between
-    // 1 and 10.
-    logOpResponseCount("Reading a 1K file", base);
-    base = assertWebResponsesInRange(base, 1, 10);
-    getBandwidthGaugeUpdater().triggerUpdate(false);
-    long totalBytesRead = AzureMetricsTestUtil.getCurrentTotalBytesRead(getInstrumentation());
-    assertEquals(FILE_SIZE, totalBytesRead);
-    long bytesRead = AzureMetricsTestUtil.getCurrentBytesRead(getInstrumentation());
-    assertTrue("The bytes read in the last second " + bytesRead +
-        " is pretty far from the expected range of around " + FILE_SIZE +
-        " bytes plus a little overhead.",
-        bytesRead > (FILE_SIZE / 2) && bytesRead < (FILE_SIZE * 2));
-    long downloadRate = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(), WASB_DOWNLOAD_RATE);
-    System.out.println("Download rate: " + downloadRate + " bytes/second.");
-    expectedRate = (FILE_SIZE * 1000L) / downloadDurationMs;
-    assertTrue("The download rate " + downloadRate +
-        " is below the expected range of around " + expectedRate +
-        " bytes/second that the unit test observed. This should never be" +
-        " the case since the test underestimates the rate by looking at " +
-        " end-to-end time instead of just block download time.",
-        downloadRate >= expectedRate);
-    long downloadLatency = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(),
-        WASB_DOWNLOAD_LATENCY);
-    System.out.println("Download latency: " + downloadLatency);
-    expectedLatency = downloadDurationMs; // We're downloading less than a block.
-    assertTrue("The download latency " + downloadLatency +
-        " should be greater than zero now that I've just downloaded a file.",
-        downloadLatency > 0);
-    assertTrue("The download latency " + downloadLatency +
-        " is more than the expected range of around " + expectedLatency +
-        " milliseconds that the unit test observed. This should never be" +
-        " the case since the test overestimates the latency by looking at " +
-        " end-to-end time instead of just block download time.",
-        downloadLatency <= expectedLatency);
-
-    assertNoErrors();
-  }
-
-  @Test
-  public void testMetricsOnBigFileCreateRead() throws Exception {
-    long base = getBaseWebResponses();
-
-    assertEquals(0, AzureMetricsTestUtil.getCurrentBytesWritten(getInstrumentation()));
-
-    Path filePath = new Path("/metricsTest_webResponses");
-    final int FILE_SIZE = 100 * 1024 * 1024;
-
-    // Suppress auto-update of bandwidth metrics so we get
-    // to update them exactly when we want to.
-    getBandwidthGaugeUpdater().suppressAutoUpdate();
-
-    // Create a file
-    OutputStream outputStream = fs.create(filePath);
-    outputStream.write(new byte[FILE_SIZE]);
-    outputStream.close();
-
-    // The exact number of requests/responses that happen to create a file
-    // can vary  - at the time of writing this code it takes 34
-    // requests/responses for the 100 MB file,
-    // plus the initial container check request, but that
-    // can very easily change in the future. Just assert that we do roughly
-    // more than 20 but less than 50.
-    logOpResponseCount("Creating a 100 MB file", base);
-    base = assertWebResponsesInRange(base, 20, 50);
-    getBandwidthGaugeUpdater().triggerUpdate(true);
-    long totalBytesWritten = AzureMetricsTestUtil.getCurrentTotalBytesWritten(getInstrumentation());
-    assertTrue("The total bytes written  " + totalBytesWritten +
-        " is pretty far from the expected range of around " + FILE_SIZE +
-        " bytes plus a little overhead.",
-        totalBytesWritten >= FILE_SIZE && totalBytesWritten < (FILE_SIZE * 2));
-    long uploadRate = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(), WASB_UPLOAD_RATE);
-    System.out.println("Upload rate: " + uploadRate + " bytes/second.");
-    long uploadLatency = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(),
-        WASB_UPLOAD_LATENCY);
-    System.out.println("Upload latency: " + uploadLatency);
-    assertTrue("The upload latency " + uploadLatency +
-        " should be greater than zero now that I've just uploaded a file.",
-        uploadLatency > 0);
-
-    // Read the file
-    InputStream inputStream = fs.open(filePath);
-    int count = 0;
-    while (inputStream.read() >= 0) {
-      count++;
-    }
-    inputStream.close();
-    assertEquals(FILE_SIZE, count);
-
-    // Again, exact number varies. At the time of writing this code
-    // it takes 27 request/responses, so just assert a rough range between
-    // 20 and 40.
-    logOpResponseCount("Reading a 100 MB file", base);
-    base = assertWebResponsesInRange(base, 20, 40);
-    getBandwidthGaugeUpdater().triggerUpdate(false);
-    long totalBytesRead = AzureMetricsTestUtil.getCurrentTotalBytesRead(getInstrumentation());
-    assertEquals(FILE_SIZE, totalBytesRead);
-    long downloadRate = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(), WASB_DOWNLOAD_RATE);
-    System.out.println("Download rate: " + downloadRate + " bytes/second.");
-    long downloadLatency = AzureMetricsTestUtil.getLongGaugeValue(getInstrumentation(),
-        WASB_DOWNLOAD_LATENCY);
-    System.out.println("Download latency: " + downloadLatency);
-    assertTrue("The download latency " + downloadLatency +
-        " should be greater than zero now that I've just downloaded a file.",
-        downloadLatency > 0);
-  }
-
-  @Test
-  public void testMetricsOnFileRename() throws Exception {
-    long base = getBaseWebResponses();
-
-    Path originalPath = new Path("/metricsTest_RenameStart");
-    Path destinationPath = new Path("/metricsTest_RenameFinal");
-
-    // Create an empty file
-    assertEquals(0, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_FILES_CREATED));
-    assertTrue(fs.createNewFile(originalPath));
-    logOpResponseCount("Creating an empty file", base);
-    base = assertWebResponsesInRange(base, 2, 20);
-    assertEquals(1, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_FILES_CREATED));
-
-    // Rename the file
-    assertTrue(fs.rename(originalPath, destinationPath));
-    // Varies: at the time of writing this code it takes 7 requests/responses.
-    logOpResponseCount("Renaming a file", base);
-    base = assertWebResponsesInRange(base, 2, 15);
-
-    assertNoErrors();
-  }
-
-  @Test
-  public void testMetricsOnFileExistsDelete() throws Exception {
-    long base = getBaseWebResponses();
-
-    Path filePath = new Path("/metricsTest_delete");
-
-    // Check existence
-    assertFalse(fs.exists(filePath));
-    // At the time of writing this code it takes 2 requests/responses to
-    // check existence, which seems excessive, plus initial request for
-    // container check.
-    logOpResponseCount("Checking file existence for non-existent file", base);
-    base = assertWebResponsesInRange(base, 1, 3);
-
-    // Create an empty file
-    assertTrue(fs.createNewFile(filePath));
-    base = getCurrentWebResponses();
-
-    // Check existence again
-    assertTrue(fs.exists(filePath));
-    logOpResponseCount("Checking file existence for existent file", base);
-    base = assertWebResponsesInRange(base, 1, 2);
-
-    // Delete the file
-    assertEquals(0, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_FILES_DELETED));
-    assertTrue(fs.delete(filePath, false));
-    // At the time of writing this code it takes 4 requests/responses to
-    // delete, which seems excessive. Check for range 1-4 for now.
-    logOpResponseCount("Deleting a file", base);
-    base = assertWebResponsesInRange(base, 1, 4);
-    assertEquals(1, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_FILES_DELETED));
-
-    assertNoErrors();
-  }
-
-  @Test
-  public void testMetricsOnDirRename() throws Exception {
-    long base = getBaseWebResponses();
-
-    Path originalDirName = new Path("/metricsTestDirectory_RenameStart");
-    Path innerFileName = new Path(originalDirName, "innerFile");
-    Path destDirName = new Path("/metricsTestDirectory_RenameFinal");
-
-    // Create an empty directory
-    assertTrue(fs.mkdirs(originalDirName));
-    base = getCurrentWebResponses();
-
-    // Create an inner file
-    assertTrue(fs.createNewFile(innerFileName));
-    base = getCurrentWebResponses();
-
-    // Rename the directory
-    assertTrue(fs.rename(originalDirName, destDirName));
-    // At the time of writing this code it takes 11 requests/responses
-    // to rename the directory with one file. Check for range 1-20 for now.
-    logOpResponseCount("Renaming a directory", base);
-    base = assertWebResponsesInRange(base, 1, 20);
-
-    assertNoErrors();
-  }
-
-  @Test
-  public void testClientErrorMetrics() throws Exception {
-    String fileName = "metricsTestFile_ClientError";
-    Path filePath = new Path("/"+fileName);
-    final int FILE_SIZE = 100;
-    OutputStream outputStream = null;
-    String leaseID = null;
-    try {
-      // Create a file
-      outputStream = fs.create(filePath);
-      leaseID = testAccount.acquireShortLease(fileName);
-      try {
-        outputStream.write(new byte[FILE_SIZE]);
-        outputStream.close();
-        assertTrue("Should've thrown", false);
-      } catch (AzureException ex) {
-        assertTrue("Unexpected exception: " + ex,
-          ex.getMessage().contains("lease"));
-      }
-      assertEquals(1, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_CLIENT_ERRORS));
-      assertEquals(0, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_SERVER_ERRORS));
-    } finally {
-      if(leaseID != null){
-        testAccount.releaseLease(leaseID, fileName);
-      }
-      IOUtils.closeStream(outputStream);
-    }
-  }
-
-  private void logOpResponseCount(String opName, long base) {
-    System.out.println(opName + " took " + (getCurrentWebResponses() - base) +
-        " web responses to complete.");
-  }
-
-  /**
-   * Gets (and asserts) the value of the wasb_web_responses counter just
-   * after the creation of the file system object.
-   */
-  private long getBaseWebResponses() {
-    // The number of requests should start at 0
-    return assertWebResponsesEquals(0, 0);
-  }
-
-  /**
-   * Gets the current value of the wasb_web_responses counter.
-   */
-  private long getCurrentWebResponses() {
-	    return AzureMetricsTestUtil.getCurrentWebResponses(getInstrumentation());
-  }
-
-  /**
-   * Checks that the wasb_web_responses counter is at the given value.
-   * @param base The base value (before the operation of interest).
-   * @param expected The expected value for the operation of interest.
-   * @return The new base value now.
-   */
-  private long assertWebResponsesEquals(long base, long expected) {
-    assertCounter(WASB_WEB_RESPONSES, base + expected, getMyMetrics());
-    return base + expected;
-  }
-
-  private void assertNoErrors() {
-    assertEquals(0, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_CLIENT_ERRORS));
-    assertEquals(0, AzureMetricsTestUtil.getLongCounterValue(getInstrumentation(), WASB_SERVER_ERRORS));
-  }
-
-  /**
-   * Checks that the wasb_web_responses counter is in the given range.
-   * @param base The base value (before the operation of interest).
-   * @param inclusiveLowerLimit The lower limit for what it should increase by.
-   * @param inclusiveUpperLimit The upper limit for what it should increase by.
-   * @return The new base value now.
-   */
-  private long assertWebResponsesInRange(long base,
-      long inclusiveLowerLimit,
-      long inclusiveUpperLimit) {
-    long currentResponses = getCurrentWebResponses();
-    long justOperation = currentResponses - base;
-    assertTrue(String.format(
-        "Web responses expected in range [%d, %d], but was %d.",
-        inclusiveLowerLimit, inclusiveUpperLimit, justOperation),
-        justOperation >= inclusiveLowerLimit &&
-        justOperation <= inclusiveUpperLimit);
-    return currentResponses;
-  }  
-
-  /**
-   * Gets the metrics for the file system object.
-   * @return The metrics record.
-   */
-  private MetricsRecordBuilder getMyMetrics() {
-    return getMetrics(getInstrumentation());
-  }
-
-  private AzureFileSystemInstrumentation getInstrumentation() {
-    return ((NativeAzureFileSystem)fs).getInstrumentation();
-  }
-
-  /**
-   * A matcher class for asserting that we got a tag with a given
-   * value.
-   */
-  private static class TagMatcher extends TagExistsMatcher {
-    private final String tagValue;
-
-    public TagMatcher(String tagName, String tagValue) {
-      super(tagName);
-      this.tagValue = tagValue;
-    }
-
-    @Override
-    public boolean matches(MetricsTag toMatch) {
-      return toMatch.value().equals(tagValue);
-    }
-
-    @Override
-    public void describeTo(Description desc) {
-      super.describeTo(desc);
-      desc.appendText(" with value " + tagValue);
-    }
-  }
-
-  /**
-   * A matcher class for asserting that we got a tag with any value.
-   */
-  private static class TagExistsMatcher extends BaseMatcher<MetricsTag> {
-    private final String tagName;
-
-    public TagExistsMatcher(String tagName) {
-      this.tagName = tagName;
-    }
-
-    @Override
-    public boolean matches(Object toMatch) {
-      MetricsTag asTag = (MetricsTag)toMatch;
-      return asTag.name().equals(tagName) && matches(asTag);
-    }
-
-    protected boolean matches(MetricsTag toMatch) {
-      return true;
-    }
-
-    @Override
-    public void describeTo(Description desc) {
-      desc.appendText("Has tag " + tagName);
-    }
-  }
-
-  /**
-   * A matcher class for asserting that a long value is in a
-   * given range.
-   */
-  private static class InRange extends BaseMatcher<Long> {
-    private final long inclusiveLowerLimit;
-    private final long inclusiveUpperLimit;
-    private long obtained;
-
-    public InRange(long inclusiveLowerLimit, long inclusiveUpperLimit) {
-      this.inclusiveLowerLimit = inclusiveLowerLimit;
-      this.inclusiveUpperLimit = inclusiveUpperLimit;
-    }
-
-    @Override
-    public boolean matches(Object number) {
-      obtained = (Long)number;
-      return obtained >= inclusiveLowerLimit &&
-          obtained <= inclusiveUpperLimit;
-    }
-
-    @Override
-    public void describeTo(Description description) {
-      description.appendText("Between " + inclusiveLowerLimit +
-          " and " + inclusiveUpperLimit + " inclusively");
-    }
-  }
-}


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


[44/50] [abbrv] hadoop git commit: HDFS-12470. DiskBalancer: Some tests create plan files under system directory. Contributed by Hanisha Koneru.

Posted by ae...@apache.org.
HDFS-12470. DiskBalancer: Some tests create plan files under system directory. Contributed by Hanisha Koneru.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a2dcba18
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a2dcba18
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a2dcba18

Branch: refs/heads/HDFS-7240
Commit: a2dcba18531c6fa4b76325f5132773f12ddfc6d5
Parents: a4f9c7c
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Sep 18 09:53:24 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Mon Sep 18 09:53:24 2017 -0700

----------------------------------------------------------------------
 .../server/diskbalancer/command/TestDiskBalancerCommand.java    | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2dcba18/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
index b0b0b0c..1cebae0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
@@ -476,9 +476,12 @@ public class TestDiskBalancerCommand {
   public void testPlanJsonNode() throws Exception {
     final String planArg = String.format("-%s %s", PLAN,
         "a87654a9-54c7-4693-8dd9-c9c7021dc340");
+    final Path testPath = new Path(
+        PathUtils.getTestPath(getClass()),
+        GenericTestUtils.getMethodName());
     final String cmdLine = String
         .format(
-            "hdfs diskbalancer %s", planArg);
+            "hdfs diskbalancer -out %s %s", testPath, planArg);
     runCommand(cmdLine);
   }
 


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


[22/50] [abbrv] hadoop git commit: HADOOP-14553. Add (parallelized) integration tests to hadoop-azure Contributed by Steve Loughran

Posted by ae...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbRemoteCallHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbRemoteCallHelper.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbRemoteCallHelper.java
deleted file mode 100644
index 8aad9e9..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbRemoteCallHelper.java
+++ /dev/null
@@ -1,569 +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.azure;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.apache.hadoop.io.retry.RetryUtils;
-import org.apache.http.Header;
-import org.apache.http.HttpResponse;
-import org.apache.http.HttpEntity;
-import org.apache.http.HttpStatus;
-import org.apache.http.StatusLine;
-import org.apache.http.ProtocolVersion;
-import org.apache.http.ParseException;
-import org.apache.http.HeaderElement;
-import org.apache.http.client.HttpClient;
-import org.apache.http.client.methods.HttpGet;
-import org.hamcrest.Description;
-import org.hamcrest.TypeSafeMatcher;
-import org.junit.Assume;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.mockito.ArgumentMatcher;
-import org.mockito.Mockito;
-
-import java.io.ByteArrayInputStream;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.nio.charset.StandardCharsets;
-
-import static org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.KEY_USE_SECURE_MODE;
-import static org.mockito.Matchers.argThat;
-import static org.mockito.Mockito.atLeast;
-import static org.mockito.Mockito.times;
-
-/**
- * Test class to hold all WasbRemoteCallHelper tests
- */
-public class TestWasbRemoteCallHelper
-    extends AbstractWasbTestBase {
-  public static final String EMPTY_STRING = "";
-  private static final int INVALID_HTTP_STATUS_CODE_999 = 999;
-
-  @Override
-  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
-    Configuration conf = new Configuration();
-    conf.set(NativeAzureFileSystem.KEY_AZURE_AUTHORIZATION, "true");
-    conf.set(RemoteWasbAuthorizerImpl.KEY_REMOTE_AUTH_SERVICE_URLS, "http://localhost1/,http://localhost2/,http://localhost:8080");
-    return AzureBlobStorageTestAccount.create(conf);
-  }
-
-  @Before
-  public void beforeMethod() {
-    boolean useSecureMode = fs.getConf().getBoolean(KEY_USE_SECURE_MODE, false);
-    boolean useAuthorization = fs.getConf().getBoolean(NativeAzureFileSystem.KEY_AZURE_AUTHORIZATION, false);
-    Assume.assumeTrue("Test valid when both SecureMode and Authorization are enabled .. skipping",
-        useSecureMode && useAuthorization);
-
-    Assume.assumeTrue(
-        useSecureMode && useAuthorization
-    );
-  }
-
-  @Rule
-  public ExpectedException expectedEx = ExpectedException.none();
-
-  /**
-   * Test invalid status-code
-   * @throws Throwable
-   */
-  @Test // (expected = WasbAuthorizationException.class)
-  public void testInvalidStatusCode() throws Throwable {
-
-    setupExpectations();
-
-    // set up mocks
-    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
-    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
-    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
-    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(INVALID_HTTP_STATUS_CODE_999));
-    // finished setting up mocks
-
-    performop(mockHttpClient);
-  }
-
-  /**
-   * Test invalid Content-Type
-   * @throws Throwable
-   */
-  @Test // (expected = WasbAuthorizationException.class)
-  public void testInvalidContentType() throws Throwable {
-
-    setupExpectations();
-
-    // set up mocks
-    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
-    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
-    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
-    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "text/plain"));
-    // finished setting up mocks
-
-    performop(mockHttpClient);
-  }
-
-  /**
-   * Test missing Content-Length
-   * @throws Throwable
-   */
-  @Test // (expected = WasbAuthorizationException.class)
-  public void testMissingContentLength() throws Throwable {
-
-    setupExpectations();
-
-    // set up mocks
-    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
-    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
-    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
-    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "application/json"));
-    // finished setting up mocks
-
-    performop(mockHttpClient);
-  }
-
-  /**
-   * Test Content-Length exceeds max
-   * @throws Throwable
-   */
-  @Test // (expected = WasbAuthorizationException.class)
-  public void testContentLengthExceedsMax() throws Throwable {
-
-    setupExpectations();
-
-    // set up mocks
-    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
-    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
-    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
-    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "application/json"));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
-        .thenReturn(newHeader("Content-Length", "2048"));
-    // finished setting up mocks
-
-    performop(mockHttpClient);
-  }
-
-  /**
-   * Test invalid Content-Length value
-   * @throws Throwable
-   */
-  @Test // (expected = WasbAuthorizationException.class)
-  public void testInvalidContentLengthValue() throws Throwable {
-
-    setupExpectations();
-
-    // set up mocks
-    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
-    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
-    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
-    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "application/json"));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
-        .thenReturn(newHeader("Content-Length", "20abc48"));
-    // finished setting up mocks
-
-    performop(mockHttpClient);
-  }
-
-  /**
-   * Test valid JSON response
-   * @throws Throwable
-   */
-  @Test
-  public void testValidJSONResponse() throws Throwable {
-
-    // set up mocks
-    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
-
-    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
-    HttpEntity mockHttpEntity = Mockito.mock(HttpEntity.class);
-
-    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
-    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "application/json"));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
-        .thenReturn(newHeader("Content-Length", "1024"));
-    Mockito.when(mockHttpResponse.getEntity()).thenReturn(mockHttpEntity);
-    Mockito.when(mockHttpEntity.getContent())
-        .thenReturn(new ByteArrayInputStream(validJsonResponse().getBytes(StandardCharsets.UTF_8)))
-        .thenReturn(new ByteArrayInputStream(validJsonResponse().getBytes(StandardCharsets.UTF_8)))
-        .thenReturn(new ByteArrayInputStream(validJsonResponse().getBytes(StandardCharsets.UTF_8)));
-    // finished setting up mocks
-
-    performop(mockHttpClient);
-  }
-
-  /**
-   * Test malformed JSON response
-   * @throws Throwable
-   */
-  @Test // (expected = WasbAuthorizationException.class)
-  public void testMalFormedJSONResponse() throws Throwable {
-
-    expectedEx.expect(WasbAuthorizationException.class);
-    expectedEx.expectMessage("com.fasterxml.jackson.core.JsonParseException: Unexpected end-of-input in FIELD_NAME");
-
-    // set up mocks
-    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
-
-    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
-    HttpEntity mockHttpEntity = Mockito.mock(HttpEntity.class);
-
-    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
-    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "application/json"));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
-        .thenReturn(newHeader("Content-Length", "1024"));
-    Mockito.when(mockHttpResponse.getEntity()).thenReturn(mockHttpEntity);
-    Mockito.when(mockHttpEntity.getContent())
-        .thenReturn(new ByteArrayInputStream(malformedJsonResponse().getBytes(StandardCharsets.UTF_8)));
-    // finished setting up mocks
-
-    performop(mockHttpClient);
-  }
-
-  /**
-   * Test valid JSON response failure response code
-   * @throws Throwable
-   */
-  @Test // (expected = WasbAuthorizationException.class)
-  public void testFailureCodeJSONResponse() throws Throwable {
-
-    expectedEx.expect(WasbAuthorizationException.class);
-    expectedEx.expectMessage("Remote authorization service encountered an error Unauthorized");
-
-    // set up mocks
-    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
-
-    HttpResponse mockHttpResponse = Mockito.mock(HttpResponse.class);
-    HttpEntity mockHttpEntity = Mockito.mock(HttpEntity.class);
-
-    Mockito.when(mockHttpClient.execute(Mockito.<HttpGet>any())).thenReturn(mockHttpResponse);
-    Mockito.when(mockHttpResponse.getStatusLine()).thenReturn(newStatusLine(HttpStatus.SC_OK));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "application/json"));
-    Mockito.when(mockHttpResponse.getFirstHeader("Content-Length"))
-        .thenReturn(newHeader("Content-Length", "1024"));
-    Mockito.when(mockHttpResponse.getEntity()).thenReturn(mockHttpEntity);
-    Mockito.when(mockHttpEntity.getContent())
-        .thenReturn(new ByteArrayInputStream(failureCodeJsonResponse().getBytes(StandardCharsets.UTF_8)));
-    // finished setting up mocks
-
-    performop(mockHttpClient);
-  }
-
-  @Test
-  public void testWhenOneInstanceIsDown() throws Throwable {
-
-    boolean isAuthorizationCachingEnabled = fs.getConf().getBoolean(CachingAuthorizer.KEY_AUTH_SERVICE_CACHING_ENABLE, false);
-
-    // set up mocks
-    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
-    HttpEntity mockHttpEntity = Mockito.mock(HttpEntity.class);
-
-    HttpResponse mockHttpResponseService1 = Mockito.mock(HttpResponse.class);
-    Mockito.when(mockHttpResponseService1.getStatusLine())
-        .thenReturn(newStatusLine(HttpStatus.SC_INTERNAL_SERVER_ERROR));
-    Mockito.when(mockHttpResponseService1.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "application/json"));
-    Mockito.when(mockHttpResponseService1.getFirstHeader("Content-Length"))
-        .thenReturn(newHeader("Content-Length", "1024"));
-    Mockito.when(mockHttpResponseService1.getEntity())
-        .thenReturn(mockHttpEntity);
-
-    HttpResponse mockHttpResponseService2 = Mockito.mock(HttpResponse.class);
-    Mockito.when(mockHttpResponseService2.getStatusLine())
-        .thenReturn(newStatusLine(HttpStatus.SC_OK));
-    Mockito.when(mockHttpResponseService2.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "application/json"));
-    Mockito.when(mockHttpResponseService2.getFirstHeader("Content-Length"))
-        .thenReturn(newHeader("Content-Length", "1024"));
-    Mockito.when(mockHttpResponseService2.getEntity())
-        .thenReturn(mockHttpEntity);
-
-    HttpResponse mockHttpResponseServiceLocal = Mockito.mock(HttpResponse.class);
-    Mockito.when(mockHttpResponseServiceLocal.getStatusLine())
-        .thenReturn(newStatusLine(HttpStatus.SC_INTERNAL_SERVER_ERROR));
-    Mockito.when(mockHttpResponseServiceLocal.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "application/json"));
-    Mockito.when(mockHttpResponseServiceLocal.getFirstHeader("Content-Length"))
-        .thenReturn(newHeader("Content-Length", "1024"));
-    Mockito.when(mockHttpResponseServiceLocal.getEntity())
-        .thenReturn(mockHttpEntity);
-
-
-
-    class HttpGetForService1 extends ArgumentMatcher<HttpGet>{
-      @Override public boolean matches(Object o) {
-        return checkHttpGetMatchHost((HttpGet) o, "localhost1");
-      }
-    }
-    class HttpGetForService2 extends ArgumentMatcher<HttpGet>{
-      @Override public boolean matches(Object o) {
-        return checkHttpGetMatchHost((HttpGet) o, "localhost2");
-      }
-    }
-    class HttpGetForServiceLocal extends ArgumentMatcher<HttpGet>{
-      @Override public boolean matches(Object o) {
-        try {
-          return checkHttpGetMatchHost((HttpGet) o, InetAddress.getLocalHost().getCanonicalHostName());
-        } catch (UnknownHostException e) {
-          return checkHttpGetMatchHost((HttpGet) o, "localhost");
-        }
-      }
-    }
-    Mockito.when(mockHttpClient.execute(argThat(new HttpGetForService1())))
-        .thenReturn(mockHttpResponseService1);
-    Mockito.when(mockHttpClient.execute(argThat(new HttpGetForService2())))
-        .thenReturn(mockHttpResponseService2);
-    Mockito.when(mockHttpClient.execute(argThat(new HttpGetForServiceLocal())))
-        .thenReturn(mockHttpResponseServiceLocal);
-
-    //Need 2 times because performop()  does 2 fs operations.
-    Mockito.when(mockHttpEntity.getContent())
-        .thenReturn(new ByteArrayInputStream(validJsonResponse()
-            .getBytes(StandardCharsets.UTF_8)))
-        .thenReturn(new ByteArrayInputStream(validJsonResponse()
-            .getBytes(StandardCharsets.UTF_8)))
-        .thenReturn(new ByteArrayInputStream(validJsonResponse()
-            .getBytes(StandardCharsets.UTF_8)));
-    // finished setting up mocks
-
-    performop(mockHttpClient);
-
-    int expectedNumberOfInvocations = isAuthorizationCachingEnabled ? 1 : 2;
-    Mockito.verify(mockHttpClient, times(expectedNumberOfInvocations)).execute(Mockito.argThat(new HttpGetForServiceLocal()));
-    Mockito.verify(mockHttpClient, times(expectedNumberOfInvocations)).execute(Mockito.argThat(new HttpGetForService2()));
-  }
-
-  @Test
-  public void testWhenServiceInstancesAreDown() throws Throwable {
-    //expectedEx.expect(WasbAuthorizationException.class);
-    // set up mocks
-    HttpClient mockHttpClient = Mockito.mock(HttpClient.class);
-    HttpEntity mockHttpEntity = Mockito.mock(HttpEntity.class);
-
-    HttpResponse mockHttpResponseService1 = Mockito.mock(HttpResponse.class);
-    Mockito.when(mockHttpResponseService1.getStatusLine())
-        .thenReturn(newStatusLine(HttpStatus.SC_INTERNAL_SERVER_ERROR));
-    Mockito.when(mockHttpResponseService1.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "application/json"));
-    Mockito.when(mockHttpResponseService1.getFirstHeader("Content-Length"))
-        .thenReturn(newHeader("Content-Length", "1024"));
-    Mockito.when(mockHttpResponseService1.getEntity())
-        .thenReturn(mockHttpEntity);
-
-    HttpResponse mockHttpResponseService2 = Mockito.mock(HttpResponse.class);
-    Mockito.when(mockHttpResponseService2.getStatusLine())
-        .thenReturn(newStatusLine(
-        HttpStatus.SC_INTERNAL_SERVER_ERROR));
-    Mockito.when(mockHttpResponseService2.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "application/json"));
-    Mockito.when(mockHttpResponseService2.getFirstHeader("Content-Length"))
-        .thenReturn(newHeader("Content-Length", "1024"));
-    Mockito.when(mockHttpResponseService2.getEntity())
-        .thenReturn(mockHttpEntity);
-
-    HttpResponse mockHttpResponseService3 = Mockito.mock(HttpResponse.class);
-    Mockito.when(mockHttpResponseService3.getStatusLine())
-        .thenReturn(newStatusLine(
-            HttpStatus.SC_INTERNAL_SERVER_ERROR));
-    Mockito.when(mockHttpResponseService3.getFirstHeader("Content-Type"))
-        .thenReturn(newHeader("Content-Type", "application/json"));
-    Mockito.when(mockHttpResponseService3.getFirstHeader("Content-Length"))
-        .thenReturn(newHeader("Content-Length", "1024"));
-    Mockito.when(mockHttpResponseService3.getEntity())
-        .thenReturn(mockHttpEntity);
-
-    class HttpGetForService1 extends ArgumentMatcher<HttpGet>{
-      @Override public boolean matches(Object o) {
-        return checkHttpGetMatchHost((HttpGet) o, "localhost1");
-      }
-    }
-    class HttpGetForService2 extends ArgumentMatcher<HttpGet>{
-      @Override public boolean matches(Object o) {
-        return checkHttpGetMatchHost((HttpGet) o, "localhost2");
-      }
-    }
-    class HttpGetForService3 extends ArgumentMatcher<HttpGet> {
-      @Override public boolean matches(Object o){
-        try {
-          return checkHttpGetMatchHost((HttpGet) o, InetAddress.getLocalHost().getCanonicalHostName());
-        } catch (UnknownHostException e) {
-          return checkHttpGetMatchHost((HttpGet) o, "localhost");
-        }
-      }
-    }
-    Mockito.when(mockHttpClient.execute(argThat(new HttpGetForService1())))
-        .thenReturn(mockHttpResponseService1);
-    Mockito.when(mockHttpClient.execute(argThat(new HttpGetForService2())))
-        .thenReturn(mockHttpResponseService2);
-    Mockito.when(mockHttpClient.execute(argThat(new HttpGetForService3())))
-        .thenReturn(mockHttpResponseService3);
-
-    //Need 3 times because performop()  does 3 fs operations.
-    Mockito.when(mockHttpEntity.getContent())
-        .thenReturn(new ByteArrayInputStream(
-            validJsonResponse().getBytes(StandardCharsets.UTF_8)))
-        .thenReturn(new ByteArrayInputStream(
-            validJsonResponse().getBytes(StandardCharsets.UTF_8)))
-        .thenReturn(new ByteArrayInputStream(
-            validJsonResponse().getBytes(StandardCharsets.UTF_8)));
-    // finished setting up mocks
-    try {
-      performop(mockHttpClient);
-    }catch (WasbAuthorizationException e){
-      e.printStackTrace();
-      Mockito.verify(mockHttpClient, atLeast(2))
-          .execute(argThat(new HttpGetForService1()));
-      Mockito.verify(mockHttpClient, atLeast(2))
-          .execute(argThat(new HttpGetForService2()));
-      Mockito.verify(mockHttpClient, atLeast(3))
-          .execute(argThat(new HttpGetForService3()));
-      Mockito.verify(mockHttpClient, times(7)).execute(Mockito.<HttpGet>any());
-    }
-  }
-
-  private void setupExpectations() {
-    expectedEx.expect(WasbAuthorizationException.class);
-
-    class MatchesPattern extends TypeSafeMatcher<String> {
-      private String pattern;
-
-      MatchesPattern(String pattern) {
-        this.pattern = pattern;
-      }
-
-      @Override protected boolean matchesSafely(String item) {
-        return item.matches(pattern);
-      }
-
-      @Override public void describeTo(Description description) {
-        description.appendText("matches pattern ").appendValue(pattern);
-      }
-
-      @Override protected void describeMismatchSafely(String item,
-          Description mismatchDescription) {
-        mismatchDescription.appendText("does not match");
-      }
-    }
-
-    expectedEx.expectMessage(new MatchesPattern(
-        "org\\.apache\\.hadoop\\.fs\\.azure\\.WasbRemoteCallException: "
-            + "Encountered error while making remote call to "
-            + "http:\\/\\/localhost1\\/,http:\\/\\/localhost2\\/,http:\\/\\/localhost:8080 retried 6 time\\(s\\)\\."));
-  }
-
-  private void performop(HttpClient mockHttpClient) throws Throwable {
-
-    Path testPath = new Path("/", "test.dat");
-
-    RemoteWasbAuthorizerImpl authorizer = new RemoteWasbAuthorizerImpl();
-    authorizer.init(fs.getConf());
-    WasbRemoteCallHelper mockWasbRemoteCallHelper = new WasbRemoteCallHelper(
-        RetryUtils.getMultipleLinearRandomRetry(new Configuration(),
-            EMPTY_STRING, true,
-            EMPTY_STRING, "1000,3,10000,2"));
-    mockWasbRemoteCallHelper.updateHttpClient(mockHttpClient);
-    authorizer.updateWasbRemoteCallHelper(mockWasbRemoteCallHelper);
-    fs.updateWasbAuthorizer(authorizer);
-
-    fs.create(testPath);
-    ContractTestUtils.assertPathExists(fs, "testPath was not created", testPath);
-    fs.delete(testPath, false);
-  }
-
-  private String validJsonResponse() {
-    return "{"
-        + "\"responseCode\": 0,"
-        + "\"authorizationResult\": true,"
-        + "\"responseMessage\": \"Authorized\""
-        + "}";
-  }
-
-  private String malformedJsonResponse() {
-    return "{"
-        + "\"responseCode\": 0,"
-        + "\"authorizationResult\": true,"
-        + "\"responseMessage\":";
-  }
-
-  private String failureCodeJsonResponse() {
-    return "{"
-        + "\"responseCode\": 1,"
-        + "\"authorizationResult\": false,"
-        + "\"responseMessage\": \"Unauthorized\""
-        + "}";
-  }
-
-  private StatusLine newStatusLine(int statusCode) {
-    return new StatusLine() {
-      @Override
-      public ProtocolVersion getProtocolVersion() {
-        return new ProtocolVersion("HTTP", 1, 1);
-      }
-
-      @Override
-      public int getStatusCode() {
-        return statusCode;
-      }
-
-      @Override
-      public String getReasonPhrase() {
-        return "Reason Phrase";
-      }
-    };
-  }
-
-  private Header newHeader(String name, String value) {
-    return new Header() {
-      @Override
-      public String getName() {
-        return name;
-      }
-
-      @Override
-      public String getValue() {
-        return value;
-      }
-
-      @Override
-      public HeaderElement[] getElements() throws ParseException {
-        return new HeaderElement[0];
-      }
-    };
-  }
-
-  /** Check that a HttpGet request is with given remote host. */
-  private static boolean checkHttpGetMatchHost(HttpGet g, String h) {
-    return g != null && g.getURI().getHost().equals(h);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbUriAndConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbUriAndConfiguration.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbUriAndConfiguration.java
deleted file mode 100644
index 672ed9c..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbUriAndConfiguration.java
+++ /dev/null
@@ -1,617 +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.azure;
-
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.junit.Assume.assumeNotNull;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.URI;
-import java.util.Date;
-import java.util.EnumSet;
-import java.io.File;
-
-import org.apache.hadoop.security.ProviderUtils;
-import org.apache.hadoop.security.alias.CredentialProvider;
-import org.apache.hadoop.security.alias.CredentialProviderFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.AbstractFileSystem;
-import org.apache.hadoop.fs.FileContext;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.UnsupportedFileSystemException;
-import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.CreateOptions;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Assume;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import com.microsoft.azure.storage.blob.CloudBlobContainer;
-import com.microsoft.azure.storage.blob.CloudBlockBlob;
-
-public class TestWasbUriAndConfiguration {
-
-  private static final int FILE_SIZE = 4096;
-  private static final String PATH_DELIMITER = "/";
-
-  protected String accountName;
-  protected String accountKey;
-  protected static Configuration conf = null;
-  private boolean runningInSASMode = false;
-  @Rule
-  public final TemporaryFolder tempDir = new TemporaryFolder();
-
-  private AzureBlobStorageTestAccount testAccount;
-
-  @After
-  public void tearDown() throws Exception {
-    if (testAccount != null) {
-      testAccount.cleanup();
-      testAccount = null;
-    }
-  }
-
-  @Before
-  public void setMode() {
-    runningInSASMode = AzureBlobStorageTestAccount.createTestConfiguration().
-        getBoolean(AzureNativeFileSystemStore.KEY_USE_SECURE_MODE, false);
-  }
-
-  private boolean validateIOStreams(Path filePath) throws IOException {
-    // Capture the file system from the test account.
-    FileSystem fs = testAccount.getFileSystem();
-    return validateIOStreams(fs, filePath);
-  }
-
-  private boolean validateIOStreams(FileSystem fs, Path filePath)
-      throws IOException {
-
-    // Create and write a file
-    OutputStream outputStream = fs.create(filePath);
-    outputStream.write(new byte[FILE_SIZE]);
-    outputStream.close();
-
-    // Return true if the the count is equivalent to the file size.
-    return (FILE_SIZE == readInputStream(fs, filePath));
-  }
-
-  private int readInputStream(Path filePath) throws IOException {
-    // Capture the file system from the test account.
-    FileSystem fs = testAccount.getFileSystem();
-    return readInputStream(fs, filePath);
-  }
-
-  private int readInputStream(FileSystem fs, Path filePath) throws IOException {
-    // Read the file
-    InputStream inputStream = fs.open(filePath);
-    int count = 0;
-    while (inputStream.read() >= 0) {
-      count++;
-    }
-    inputStream.close();
-
-    // Return true if the the count is equivalent to the file size.
-    return count;
-  }
-
-  // Positive tests to exercise making a connection with to Azure account using
-  // account key.
-  @Test
-  public void testConnectUsingKey() throws Exception {
-
-    testAccount = AzureBlobStorageTestAccount.create();
-    assumeNotNull(testAccount);
-
-    // Validate input and output on the connection.
-    assertTrue(validateIOStreams(new Path("/wasb_scheme")));
-  }
-
-  @Test
-  public void testConnectUsingSAS() throws Exception {
-
-    Assume.assumeFalse(runningInSASMode);
-    // Create the test account with SAS credentials.
-    testAccount = AzureBlobStorageTestAccount.create("",
-        EnumSet.of(CreateOptions.UseSas, CreateOptions.CreateContainer));
-    assumeNotNull(testAccount);
-    // Validate input and output on the connection.
-    // NOTE: As of 4/15/2013, Azure Storage has a deficiency that prevents the
-    // full scenario from working (CopyFromBlob doesn't work with SAS), so
-    // just do a minor check until that is corrected.
-    assertFalse(testAccount.getFileSystem().exists(new Path("/IDontExist")));
-    //assertTrue(validateIOStreams(new Path("/sastest.txt")));
-  }
-
-  @Test
-  public void testConnectUsingSASReadonly() throws Exception {
-
-    Assume.assumeFalse(runningInSASMode);
-    // Create the test account with SAS credentials.
-    testAccount = AzureBlobStorageTestAccount.create("", EnumSet.of(
-        CreateOptions.UseSas, CreateOptions.CreateContainer,
-        CreateOptions.Readonly));
-    assumeNotNull(testAccount);
-
-    // Create a blob in there
-    final String blobKey = "blobForReadonly";
-    CloudBlobContainer container = testAccount.getRealContainer();
-    CloudBlockBlob blob = container.getBlockBlobReference(blobKey);
-    ByteArrayInputStream inputStream = new ByteArrayInputStream(new byte[] { 1,
-        2, 3 });
-    blob.upload(inputStream, 3);
-    inputStream.close();
-
-    // Make sure we can read it from the file system
-    Path filePath = new Path("/" + blobKey);
-    FileSystem fs = testAccount.getFileSystem();
-    assertTrue(fs.exists(filePath));
-    byte[] obtained = new byte[3];
-    DataInputStream obtainedInputStream = fs.open(filePath);
-    obtainedInputStream.readFully(obtained);
-    obtainedInputStream.close();
-    assertEquals(3, obtained[2]);
-  }
-
-  @Test
-  public void testConnectUsingAnonymous() throws Exception {
-
-    // Create test account with anonymous credentials
-    testAccount = AzureBlobStorageTestAccount.createAnonymous("testWasb.txt",
-        FILE_SIZE);
-    assumeNotNull(testAccount);
-
-    // Read the file from the public folder using anonymous credentials.
-    assertEquals(FILE_SIZE, readInputStream(new Path("/testWasb.txt")));
-  }
-
-  @Test
-  public void testConnectToEmulator() throws Exception {
-    testAccount = AzureBlobStorageTestAccount.createForEmulator();
-    assumeNotNull(testAccount);
-    assertTrue(validateIOStreams(new Path("/testFile")));
-  }
-
-  /**
-   * Tests that we can connect to fully qualified accounts outside of
-   * blob.core.windows.net
-   */
-  @Test
-  public void testConnectToFullyQualifiedAccountMock() throws Exception {
-    Configuration conf = new Configuration();
-    AzureBlobStorageTestAccount.setMockAccountKey(conf,
-        "mockAccount.mock.authority.net");
-    AzureNativeFileSystemStore store = new AzureNativeFileSystemStore();
-    MockStorageInterface mockStorage = new MockStorageInterface();
-    store.setAzureStorageInteractionLayer(mockStorage);
-    NativeAzureFileSystem fs = new NativeAzureFileSystem(store);
-    fs.initialize(
-        new URI("wasb://mockContainer@mockAccount.mock.authority.net"), conf);
-    fs.createNewFile(new Path("/x"));
-    assertTrue(mockStorage.getBackingStore().exists(
-        "http://mockAccount.mock.authority.net/mockContainer/x"));
-    fs.close();
-  }
-
-  public void testConnectToRoot() throws Exception {
-
-    // Set up blob names.
-    final String blobPrefix = String.format("wasbtests-%s-%tQ-blob",
-        System.getProperty("user.name"), new Date());
-    final String inblobName = blobPrefix + "_In" + ".txt";
-    final String outblobName = blobPrefix + "_Out" + ".txt";
-
-    // Create test account with default root access.
-    testAccount = AzureBlobStorageTestAccount.createRoot(inblobName, FILE_SIZE);
-    assumeNotNull(testAccount);
-
-    // Read the file from the default container.
-    assertEquals(FILE_SIZE, readInputStream(new Path(PATH_DELIMITER
-        + inblobName)));
-
-    try {
-      // Capture file system.
-      FileSystem fs = testAccount.getFileSystem();
-
-      // Create output path and open an output stream to the root folder.
-      Path outputPath = new Path(PATH_DELIMITER + outblobName);
-      OutputStream outputStream = fs.create(outputPath);
-      fail("Expected an AzureException when writing to root folder.");
-      outputStream.write(new byte[FILE_SIZE]);
-      outputStream.close();
-    } catch (AzureException e) {
-      assertTrue(true);
-    } catch (Exception e) {
-      String errMsg = String.format(
-          "Expected AzureException but got %s instead.", e);
-      assertTrue(errMsg, false);
-    }
-  }
-
-  // Positive tests to exercise throttling I/O path. Connections are made to an
-  // Azure account using account key.
-  //
-  public void testConnectWithThrottling() throws Exception {
-
-    testAccount = AzureBlobStorageTestAccount.createThrottled();
-
-    // Validate input and output on the connection.
-    assertTrue(validateIOStreams(new Path("/wasb_scheme")));
-  }
-
-  /**
-   * Creates a file and writes a single byte with the given value in it.
-   */
-  private static void writeSingleByte(FileSystem fs, Path testFile, int toWrite)
-      throws Exception {
-    OutputStream outputStream = fs.create(testFile);
-    outputStream.write(toWrite);
-    outputStream.close();
-  }
-
-  /**
-   * Reads the file given and makes sure that it's a single-byte file with the
-   * given value in it.
-   */
-  private static void assertSingleByteValue(FileSystem fs, Path testFile,
-      int expectedValue) throws Exception {
-    InputStream inputStream = fs.open(testFile);
-    int byteRead = inputStream.read();
-    assertTrue("File unexpectedly empty: " + testFile, byteRead >= 0);
-    assertTrue("File has more than a single byte: " + testFile,
-        inputStream.read() < 0);
-    inputStream.close();
-    assertEquals("Unxpected content in: " + testFile, expectedValue, byteRead);
-  }
-
-  @Test
-  public void testMultipleContainers() throws Exception {
-    AzureBlobStorageTestAccount firstAccount = AzureBlobStorageTestAccount
-        .create("first"), secondAccount = AzureBlobStorageTestAccount
-        .create("second");
-    assumeNotNull(firstAccount);
-    assumeNotNull(secondAccount);
-    try {
-      FileSystem firstFs = firstAccount.getFileSystem(),
-          secondFs = secondAccount.getFileSystem();
-      Path testFile = new Path("/testWasb");
-      assertTrue(validateIOStreams(firstFs, testFile));
-      assertTrue(validateIOStreams(secondFs, testFile));
-      // Make sure that we're really dealing with two file systems here.
-      writeSingleByte(firstFs, testFile, 5);
-      writeSingleByte(secondFs, testFile, 7);
-      assertSingleByteValue(firstFs, testFile, 5);
-      assertSingleByteValue(secondFs, testFile, 7);
-    } finally {
-      firstAccount.cleanup();
-      secondAccount.cleanup();
-    }
-  }
-
-  @Test
-  public void testDefaultKeyProvider() throws Exception {
-    Configuration conf = new Configuration();
-    String account = "testacct";
-    String key = "testkey";
-
-    conf.set(SimpleKeyProvider.KEY_ACCOUNT_KEY_PREFIX + account, key);
-
-    String result = AzureNativeFileSystemStore.getAccountKeyFromConfiguration(
-        account, conf);
-    assertEquals(key, result);
-  }
-
-  @Test
-  public void testCredsFromCredentialProvider() throws Exception {
-
-    Assume.assumeFalse(runningInSASMode);
-    String account = "testacct";
-    String key = "testkey";
-    // set up conf to have a cred provider
-    final Configuration conf = new Configuration();
-    final File file = tempDir.newFile("test.jks");
-    final URI jks = ProviderUtils.nestURIForLocalJavaKeyStoreProvider(
-        file.toURI());
-    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
-        jks.toString());
-
-    provisionAccountKey(conf, account, key);
-
-    // also add to configuration as clear text that should be overridden
-    conf.set(SimpleKeyProvider.KEY_ACCOUNT_KEY_PREFIX + account,
-        key + "cleartext");
-
-    String result = AzureNativeFileSystemStore.getAccountKeyFromConfiguration(
-        account, conf);
-    // result should contain the credential provider key not the config key
-    assertEquals("AccountKey incorrect.", key, result);
-  }
-
-  void provisionAccountKey(
-      final Configuration conf, String account, String key) throws Exception {
-    // add our creds to the provider
-    final CredentialProvider provider =
-        CredentialProviderFactory.getProviders(conf).get(0);
-    provider.createCredentialEntry(
-        SimpleKeyProvider.KEY_ACCOUNT_KEY_PREFIX + account, key.toCharArray());
-    provider.flush();
-  }
-
-  @Test
-  public void testValidKeyProvider() throws Exception {
-    Configuration conf = new Configuration();
-    String account = "testacct";
-    String key = "testkey";
-
-    conf.set(SimpleKeyProvider.KEY_ACCOUNT_KEY_PREFIX + account, key);
-    conf.setClass("fs.azure.account.keyprovider." + account,
-        SimpleKeyProvider.class, KeyProvider.class);
-    String result = AzureNativeFileSystemStore.getAccountKeyFromConfiguration(
-        account, conf);
-    assertEquals(key, result);
-  }
-
-  @Test
-  public void testInvalidKeyProviderNonexistantClass() throws Exception {
-    Configuration conf = new Configuration();
-    String account = "testacct";
-
-    conf.set("fs.azure.account.keyprovider." + account,
-        "org.apache.Nonexistant.Class");
-    try {
-      AzureNativeFileSystemStore.getAccountKeyFromConfiguration(account, conf);
-      Assert.fail("Nonexistant key provider class should have thrown a "
-          + "KeyProviderException");
-    } catch (KeyProviderException e) {
-    }
-  }
-
-  @Test
-  public void testInvalidKeyProviderWrongClass() throws Exception {
-    Configuration conf = new Configuration();
-    String account = "testacct";
-
-    conf.set("fs.azure.account.keyprovider." + account, "java.lang.String");
-    try {
-      AzureNativeFileSystemStore.getAccountKeyFromConfiguration(account, conf);
-      Assert.fail("Key provider class that doesn't implement KeyProvider "
-          + "should have thrown a KeyProviderException");
-    } catch (KeyProviderException e) {
-    }
-  }
-
-  /**
-   * Tests the cases when the URI is specified with no authority, i.e.
-   * wasb:///path/to/file.
-   */
-  @Test
-  public void testNoUriAuthority() throws Exception {
-    // For any combination of default FS being asv(s)/wasb(s)://c@a/ and
-    // the actual URI being asv(s)/wasb(s):///, it should work.
-
-    String[] wasbAliases = new String[] { "wasb", "wasbs" };
-    for (String defaultScheme : wasbAliases) {
-      for (String wantedScheme : wasbAliases) {
-        testAccount = AzureBlobStorageTestAccount.createMock();
-        Configuration conf = testAccount.getFileSystem().getConf();
-        String authority = testAccount.getFileSystem().getUri().getAuthority();
-        URI defaultUri = new URI(defaultScheme, authority, null, null, null);
-        conf.set(FS_DEFAULT_NAME_KEY, defaultUri.toString());
-        // Add references to file system implementations for wasb and wasbs.
-        conf.addResource("azure-test.xml");
-        URI wantedUri = new URI(wantedScheme + ":///random/path");
-        NativeAzureFileSystem obtained = (NativeAzureFileSystem) FileSystem
-            .get(wantedUri, conf);
-        assertNotNull(obtained);
-        assertEquals(new URI(wantedScheme, authority, null, null, null),
-            obtained.getUri());
-        // Make sure makeQualified works as expected
-        Path qualified = obtained.makeQualified(new Path(wantedUri));
-        assertEquals(new URI(wantedScheme, authority, wantedUri.getPath(),
-            null, null), qualified.toUri());
-        // Cleanup for the next iteration to not cache anything in FS
-        testAccount.cleanup();
-        FileSystem.closeAll();
-      }
-    }
-    // If the default FS is not a WASB FS, then specifying a URI without
-    // authority for the Azure file system should throw.
-    testAccount = AzureBlobStorageTestAccount.createMock();
-    Configuration conf = testAccount.getFileSystem().getConf();
-    conf.set(FS_DEFAULT_NAME_KEY, "file:///");
-    try {
-      FileSystem.get(new URI("wasb:///random/path"), conf);
-      fail("Should've thrown.");
-    } catch (IllegalArgumentException e) {
-    }
-  }
-
-  @Test
-  public void testWasbAsDefaultFileSystemHasNoPort() throws Exception {
-    try {
-      testAccount = AzureBlobStorageTestAccount.createMock();
-      Configuration conf = testAccount.getFileSystem().getConf();
-      String authority = testAccount.getFileSystem().getUri().getAuthority();
-      URI defaultUri = new URI("wasb", authority, null, null, null);
-      conf.set(FS_DEFAULT_NAME_KEY, defaultUri.toString());
-      conf.addResource("azure-test.xml");
-
-      FileSystem fs = FileSystem.get(conf);
-      assertTrue(fs instanceof NativeAzureFileSystem);
-      assertEquals(-1, fs.getUri().getPort());
-
-      AbstractFileSystem afs = FileContext.getFileContext(conf)
-          .getDefaultFileSystem();
-      assertTrue(afs instanceof Wasb);
-      assertEquals(-1, afs.getUri().getPort());
-    } finally {
-      testAccount.cleanup();
-      FileSystem.closeAll();
-    }
-  }
-
-   /**
-   * Tests the cases when the scheme specified is 'wasbs'.
-   */
-  @Test
-  public void testAbstractFileSystemImplementationForWasbsScheme() throws Exception {
-    try {
-      testAccount = AzureBlobStorageTestAccount.createMock();
-      Configuration conf = testAccount.getFileSystem().getConf();
-      String authority = testAccount.getFileSystem().getUri().getAuthority();
-      URI defaultUri = new URI("wasbs", authority, null, null, null);
-      conf.set(FS_DEFAULT_NAME_KEY, defaultUri.toString());
-      conf.set("fs.AbstractFileSystem.wasbs.impl", "org.apache.hadoop.fs.azure.Wasbs");
-      conf.addResource("azure-test.xml");
-
-      FileSystem fs = FileSystem.get(conf);
-      assertTrue(fs instanceof NativeAzureFileSystem);
-      assertEquals("wasbs", fs.getScheme());
-
-      AbstractFileSystem afs = FileContext.getFileContext(conf)
-          .getDefaultFileSystem();
-      assertTrue(afs instanceof Wasbs);
-      assertEquals(-1, afs.getUri().getPort());
-      assertEquals("wasbs", afs.getUri().getScheme());
-    } finally {
-      testAccount.cleanup();
-      FileSystem.closeAll();
-    }
-  }
-
-  @Test
-  public void testNoAbstractFileSystemImplementationSpecifiedForWasbsScheme() throws Exception {
-    try {
-      testAccount = AzureBlobStorageTestAccount.createMock();
-      Configuration conf = testAccount.getFileSystem().getConf();
-      String authority = testAccount.getFileSystem().getUri().getAuthority();
-      URI defaultUri = new URI("wasbs", authority, null, null, null);
-      conf.set(FS_DEFAULT_NAME_KEY, defaultUri.toString());
-
-      FileSystem fs = FileSystem.get(conf);
-      assertTrue(fs instanceof NativeAzureFileSystem);
-      assertEquals("wasbs", fs.getScheme());
-
-      // should throw if 'fs.AbstractFileSystem.wasbs.impl'' is not specified
-      try{
-        FileContext.getFileContext(conf).getDefaultFileSystem();
-        fail("Should've thrown.");
-      }catch(UnsupportedFileSystemException e){
-      }
-
-    } finally {
-      testAccount.cleanup();
-      FileSystem.closeAll();
-    }
-  }
-
-  @Test
-  public void testCredentialProviderPathExclusions() throws Exception {
-    String providerPath =
-        "user:///,jceks://wasb/user/hrt_qa/sqoopdbpasswd.jceks," +
-        "jceks://hdfs@nn1.example.com/my/path/test.jceks";
-    Configuration config = new Configuration();
-    config.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
-        providerPath);
-    String newPath = "user:///,jceks://hdfs@nn1.example.com/my/path/test.jceks";
-
-    excludeAndTestExpectations(config, newPath);
-  }
-
-  @Test
-  public void testExcludeAllProviderTypesFromConfig() throws Exception {
-    String providerPath =
-        "jceks://wasb/tmp/test.jceks," +
-        "jceks://wasb@/my/path/test.jceks";
-    Configuration config = new Configuration();
-    config.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
-        providerPath);
-    String newPath = null;
-
-    excludeAndTestExpectations(config, newPath);
-  }
-
-  void excludeAndTestExpectations(Configuration config, String newPath)
-    throws Exception {
-    Configuration conf = ProviderUtils.excludeIncompatibleCredentialProviders(
-        config, NativeAzureFileSystem.class);
-    String effectivePath = conf.get(
-        CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, null);
-    assertEquals(newPath, effectivePath);
-  }
-
-  @Test
-  public void testUserAgentConfig() throws Exception {
-    // Set the user agent
-    try {
-      testAccount = AzureBlobStorageTestAccount.createMock();
-      Configuration conf = testAccount.getFileSystem().getConf();
-      String authority = testAccount.getFileSystem().getUri().getAuthority();
-      URI defaultUri = new URI("wasbs", authority, null, null, null);
-      conf.set(FS_DEFAULT_NAME_KEY, defaultUri.toString());
-      conf.set("fs.AbstractFileSystem.wasbs.impl", "org.apache.hadoop.fs.azure.Wasbs");
-
-      conf.set(AzureNativeFileSystemStore.USER_AGENT_ID_KEY, "TestClient");
-
-      FileSystem fs = FileSystem.get(conf);
-      AbstractFileSystem afs = FileContext.getFileContext(conf).getDefaultFileSystem();
-
-      assertTrue(afs instanceof Wasbs);
-      assertEquals(-1, afs.getUri().getPort());
-      assertEquals("wasbs", afs.getUri().getScheme());
-
-    } finally {
-      testAccount.cleanup();
-      FileSystem.closeAll();
-    }
-
-    // Unset the user agent
-    try {
-      testAccount = AzureBlobStorageTestAccount.createMock();
-      Configuration conf = testAccount.getFileSystem().getConf();
-      String authority = testAccount.getFileSystem().getUri().getAuthority();
-      URI defaultUri = new URI("wasbs", authority, null, null, null);
-      conf.set(FS_DEFAULT_NAME_KEY, defaultUri.toString());
-      conf.set("fs.AbstractFileSystem.wasbs.impl", "org.apache.hadoop.fs.azure.Wasbs");
-
-      conf.unset(AzureNativeFileSystemStore.USER_AGENT_ID_KEY);
-
-      FileSystem fs = FileSystem.get(conf);
-      AbstractFileSystem afs = FileContext.getFileContext(conf).getDefaultFileSystem();
-      assertTrue(afs instanceof Wasbs);
-      assertEquals(-1, afs.getUri().getPort());
-      assertEquals("wasbs", afs.getUri().getScheme());
-
-    } finally {
-      testAccount.cleanup();
-      FileSystem.closeAll();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractAppend.java
new file mode 100644
index 0000000..fd21bd2
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractAppend.java
@@ -0,0 +1,41 @@
+/**
+ * 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.azure.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractAppendTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
+
+/**
+ * Append test, skipping one of them.
+ */
+
+public class ITestAzureNativeContractAppend extends AbstractContractAppendTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new NativeAzureFileSystemContract(conf);
+  }
+
+  @Override
+  public void testRenameFileBeingAppended() throws Throwable {
+    skip("Skipping as renaming an opened file is not supported");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractCreate.java
new file mode 100644
index 0000000..0ac046a
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractCreate.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test.
+ */
+public class ITestAzureNativeContractCreate extends AbstractContractCreateTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new NativeAzureFileSystemContract(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractDelete.java
new file mode 100644
index 0000000..4c6dd48
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractDelete.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test.
+ */
+public class ITestAzureNativeContractDelete extends AbstractContractDeleteTest {
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new NativeAzureFileSystemContract(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractDistCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractDistCp.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractDistCp.java
new file mode 100644
index 0000000..7769570
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractDistCp.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azure.integration.AzureTestConstants;
+import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
+
+import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.assumeScaleTestsEnabled;
+
+/**
+ * Contract test suite covering WASB integration with DistCp.
+ */
+public class ITestAzureNativeContractDistCp extends AbstractContractDistCpTest {
+
+  @Override
+  protected int getTestTimeoutMillis() {
+    return AzureTestConstants.SCALE_TEST_TIMEOUT_MILLIS;
+  }
+
+  @Override
+  protected NativeAzureFileSystemContract createContract(Configuration conf) {
+    return new NativeAzureFileSystemContract(conf);
+  }
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    assumeScaleTestsEnabled(getContract().getConf());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractGetFileStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractGetFileStatus.java
new file mode 100644
index 0000000..9c09c0d
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractGetFileStatus.java
@@ -0,0 +1,35 @@
+/**
+ * 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.azure.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test.
+ */
+public class ITestAzureNativeContractGetFileStatus
+    extends AbstractContractGetFileStatusTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new NativeAzureFileSystemContract(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractMkdir.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractMkdir.java
new file mode 100644
index 0000000..71654b8
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractMkdir.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test.
+ */
+public class ITestAzureNativeContractMkdir extends AbstractContractMkdirTest {
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new NativeAzureFileSystemContract(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractOpen.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractOpen.java
new file mode 100644
index 0000000..0b174e6
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractOpen.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test.
+ */
+public class ITestAzureNativeContractOpen extends AbstractContractOpenTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new NativeAzureFileSystemContract(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractRename.java
new file mode 100644
index 0000000..474b874
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractRename.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test.
+ */
+public class ITestAzureNativeContractRename extends AbstractContractRenameTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new NativeAzureFileSystemContract(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractSeek.java
new file mode 100644
index 0000000..673d5f8
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/ITestAzureNativeContractSeek.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Contract test.
+ */
+public class ITestAzureNativeContractSeek extends AbstractContractSeekTest{
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new NativeAzureFileSystemContract(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/NativeAzureFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/NativeAzureFileSystemContract.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/NativeAzureFileSystemContract.java
index 28c13ea..a264aca 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/NativeAzureFileSystemContract.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/NativeAzureFileSystemContract.java
@@ -18,15 +18,21 @@
 
 package org.apache.hadoop.fs.azure.contract;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azure.integration.AzureTestUtils;
 import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
 
+/**
+ * Azure Contract. Test paths are created using any maven fork
+ * identifier, if defined. This guarantees paths unique to tests
+ * running in parallel.
+ */
 public class NativeAzureFileSystemContract extends AbstractBondedFSContract {
 
   public static final String CONTRACT_XML = "wasb.xml";
 
-  protected NativeAzureFileSystemContract(Configuration conf) {
-    super(conf);
-    //insert the base features
+  public NativeAzureFileSystemContract(Configuration conf) {
+    super(conf); //insert the base features
     addConfResource(CONTRACT_XML);
   }
 
@@ -34,4 +40,9 @@ public class NativeAzureFileSystemContract extends AbstractBondedFSContract {
   public String getScheme() {
     return "wasb";
   }
-}
\ No newline at end of file
+
+  @Override
+  public Path getTestPath() {
+    return AzureTestUtils.createTestPath(super.getTestPath());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractAppend.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractAppend.java
deleted file mode 100644
index 8a2341e..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractAppend.java
+++ /dev/null
@@ -1,37 +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.azure.contract;
-
-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;
-
-public class TestAzureNativeContractAppend extends AbstractContractAppendTest {
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new NativeAzureFileSystemContract(conf);
-  }
-
-  @Override
-  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/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractCreate.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractCreate.java
deleted file mode 100644
index 531552d..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractCreate.java
+++ /dev/null
@@ -1,30 +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.azure.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-public class TestAzureNativeContractCreate extends AbstractContractCreateTest{
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new NativeAzureFileSystemContract(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractDelete.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractDelete.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractDelete.java
deleted file mode 100644
index 5e5c13b..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractDelete.java
+++ /dev/null
@@ -1,30 +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.azure.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-public class TestAzureNativeContractDelete extends AbstractContractDeleteTest {
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new NativeAzureFileSystemContract(conf);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractDistCp.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractDistCp.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractDistCp.java
deleted file mode 100644
index a3750d4..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractDistCp.java
+++ /dev/null
@@ -1,33 +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.azure.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
-
-/**
- * Contract test suite covering WASB integration with DistCp.
- */
-public class TestAzureNativeContractDistCp extends AbstractContractDistCpTest {
-
-  @Override
-  protected NativeAzureFileSystemContract createContract(Configuration conf) {
-    return new NativeAzureFileSystemContract(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractGetFileStatus.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractGetFileStatus.java
deleted file mode 100644
index b0c59ee..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractGetFileStatus.java
+++ /dev/null
@@ -1,30 +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.azure.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-public class TestAzureNativeContractGetFileStatus extends AbstractContractGetFileStatusTest {
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new NativeAzureFileSystemContract(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractMkdir.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractMkdir.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractMkdir.java
deleted file mode 100644
index 36df041..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractMkdir.java
+++ /dev/null
@@ -1,30 +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.azure.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-public class TestAzureNativeContractMkdir extends AbstractContractMkdirTest {
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new NativeAzureFileSystemContract(conf);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractOpen.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractOpen.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractOpen.java
deleted file mode 100644
index d5147ac..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractOpen.java
+++ /dev/null
@@ -1,30 +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.azure.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-public class TestAzureNativeContractOpen extends AbstractContractOpenTest {
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new NativeAzureFileSystemContract(conf);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractRename.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractRename.java
deleted file mode 100644
index 4d8b2b5..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractRename.java
+++ /dev/null
@@ -1,30 +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.azure.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-public class TestAzureNativeContractRename extends AbstractContractRenameTest {
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new NativeAzureFileSystemContract(conf);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractSeek.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractSeek.java
deleted file mode 100644
index 30046dc..0000000
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/contract/TestAzureNativeContractSeek.java
+++ /dev/null
@@ -1,30 +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.azure.contract;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
-import org.apache.hadoop.fs.contract.AbstractFSContract;
-
-public class TestAzureNativeContractSeek extends AbstractContractSeekTest{
-  @Override
-  protected AbstractFSContract createContract(Configuration conf) {
-    return new NativeAzureFileSystemContract(conf);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2d2d97fa/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AbstractAzureScaleTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AbstractAzureScaleTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AbstractAzureScaleTest.java
new file mode 100644
index 0000000..062d073
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/integration/AbstractAzureScaleTest.java
@@ -0,0 +1,66 @@
+/*
+ * 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.azure.integration;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.azure.AbstractWasbTestBase;
+import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
+
+import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.*;
+
+/**
+ * Scale tests are only executed if the scale profile
+ * is set; the setup method will check this and skip
+ * tests if not.
+ *
+ */
+public abstract class AbstractAzureScaleTest
+    extends AbstractWasbTestBase implements Sizes {
+
+  protected static final Logger LOG =
+      LoggerFactory.getLogger(AbstractAzureScaleTest.class);
+
+  @Override
+  protected int getTestTimeoutMillis() {
+    return AzureTestConstants.SCALE_TEST_TIMEOUT_MILLIS;
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    LOG.debug("Scale test operation count = {}", getOperationCount());
+    assumeScaleTestsEnabled(getConfiguration());
+  }
+
+  /**
+   * Create the test account.
+   * @return a test account
+   * @throws Exception on any failure to create the account.
+   */
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.create(createConfiguration());
+  }
+
+  protected long getOperationCount() {
+    return getConfiguration().getLong(KEY_OPERATION_COUNT,
+        DEFAULT_OPERATION_COUNT);
+  }
+}


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


[12/50] [abbrv] hadoop git commit: Revert "HDFS-11156. Add new op GETFILEBLOCKLOCATIONS to WebHDFS REST API. Contributed by Weiwei Yang."

Posted by ae...@apache.org.
Revert "HDFS-11156. Add new op GETFILEBLOCKLOCATIONS to WebHDFS REST API. Contributed by Weiwei Yang."

This reverts commit 7fcc73fc0d248aae1edbd4e1514c5818f6198928.

 Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/390c2b5d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/390c2b5d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/390c2b5d

Branch: refs/heads/HDFS-7240
Commit: 390c2b5df0c4e25a2156d8daefd4670efe82b191
Parents: 2f86711
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Sep 14 15:12:26 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Sep 14 15:12:26 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |  53 ----
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  62 +----
 .../hadoop/hdfs/web/resources/GetOpParam.java   |  12 +-
 .../web/resources/NamenodeWebHdfsMethods.java   |  16 --
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    |  33 ---
 .../hadoop-hdfs/src/site/markdown/WebHDFS.md    | 188 +------------
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java | 276 -------------------
 7 files changed, 6 insertions(+), 634 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/390c2b5d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
index 7ec5fe5..dcd73bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
@@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectReader;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary.Builder;
 import org.apache.hadoop.fs.FileChecksum;
@@ -645,56 +644,4 @@ class JsonUtilClient {
     }
   }
 
-  static BlockLocation[] toBlockLocationArray(Map<?, ?> json)
-      throws IOException{
-    final Map<?, ?> rootmap =
-        (Map<?, ?>)json.get(BlockLocation.class.getSimpleName() + "s");
-    final List<?> array = JsonUtilClient.getList(rootmap,
-        BlockLocation.class.getSimpleName());
-
-    Preconditions.checkNotNull(array);
-    final BlockLocation[] locations = new BlockLocation[array.size()];
-    int i = 0;
-    for (Object object : array) {
-      final Map<?, ?> m = (Map<?, ?>) object;
-      locations[i++] = JsonUtilClient.toBlockLocation(m);
-    }
-    return locations;
-  }
-
-  /** Convert a Json map to BlockLocation. **/
-  static BlockLocation toBlockLocation(Map<?, ?> m)
-      throws IOException{
-    if(m == null) {
-      return null;
-    }
-
-    long length = ((Number) m.get("length")).longValue();
-    long offset = ((Number) m.get("offset")).longValue();
-    boolean corrupt = Boolean.
-        getBoolean(m.get("corrupt").toString());
-    String[] storageIds = toStringArray(getList(m, "storageIds"));
-    String[] cachedHosts = toStringArray(getList(m, "cachedHosts"));
-    String[] hosts = toStringArray(getList(m, "hosts"));
-    String[] names = toStringArray(getList(m, "names"));
-    String[] topologyPaths = toStringArray(getList(m, "topologyPaths"));
-    StorageType[] storageTypes = toStorageTypeArray(
-        getList(m, "storageTypes"));
-    return new BlockLocation(names, hosts, cachedHosts,
-        topologyPaths, storageIds, storageTypes,
-        offset, length, corrupt);
-  }
-
-  static String[] toStringArray(List<?> list) {
-    if (list == null) {
-      return null;
-    } else {
-      final String[] array = new String[list.size()];
-      int i = 0;
-      for (Object object : list) {
-        array[i++] = object.toString();
-      }
-      return array;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/390c2b5d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index 1159e50..ee8d5c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -1616,68 +1616,14 @@ public class WebHdfsFileSystem extends FileSystem
       final long offset, final long length) throws IOException {
     statistics.incrementReadOps(1);
     storageStatistics.incrementOpCounter(OpType.GET_FILE_BLOCK_LOCATIONS);
-    BlockLocation[] locations = null;
-    try {
-      locations = getFileBlockLocations(
-          GetOpParam.Op.GETFILEBLOCKLOCATIONS,
-          p, offset, length);
-    } catch (RemoteException e) {
-      // See the error message from ExceptionHandle
-      if(e.getMessage() != null &&
-          e.getMessage().contains(
-              "Invalid value for webhdfs parameter") &&
-          e.getMessage().contains(
-              GetOpParam.Op.GETFILEBLOCKLOCATIONS.toString())) {
-        // Old webhdfs server doesn't support GETFILEBLOCKLOCATIONS
-        // operation, fall back to query again using old API
-        // GET_BLOCK_LOCATIONS.
-        LOG.info("Invalid webhdfs operation parameter "
-            + GetOpParam.Op.GETFILEBLOCKLOCATIONS + ". Fallback to use "
-            + GetOpParam.Op.GET_BLOCK_LOCATIONS + " instead.");
-        locations = getFileBlockLocations(
-            GetOpParam.Op.GET_BLOCK_LOCATIONS,
-            p, offset, length);
-      }
-    }
-    return locations;
-  }
 
-  /**
-   * Get file block locations implementation. Provide a operation
-   * parameter to determine how to get block locations from a webhdfs
-   * server. Older server only supports <b>GET_BLOCK_LOCATIONS</b> but
-   * not <b>GETFILEBLOCKLOCATIONS</b>.
-   *
-   * @param path path to the file
-   * @param offset start offset in the given file
-   * @param length of the file to get locations for
-   * @param operation
-   *   Valid operation is either
-   *   {@link org.apache.hadoop.hdfs.web.resources.GetOpParam.Op
-   *   #GET_BLOCK_LOCATIONS} or
-   *   {@link org.apache.hadoop.hdfs.web.resources.GetOpParam.Op
-   *   #GET_BLOCK_LOCATIONS}
-   * @throws IOException
-   *   Http connection error, decoding error or given
-   *   operation is not valid
-   */
-  @VisibleForTesting
-  protected BlockLocation[] getFileBlockLocations(
-      GetOpParam.Op operation, final Path path,
-      final long offset, final long length) throws IOException {
-    return new FsPathResponseRunner<BlockLocation[]>(operation, path,
+    final HttpOpParam.Op op = GetOpParam.Op.GET_BLOCK_LOCATIONS;
+    return new FsPathResponseRunner<BlockLocation[]>(op, p,
         new OffsetParam(offset), new LengthParam(length)) {
       @Override
       BlockLocation[] decodeResponse(Map<?,?> json) throws IOException {
-        switch(operation) {
-        case GETFILEBLOCKLOCATIONS:
-          return JsonUtilClient.toBlockLocationArray(json);
-        case GET_BLOCK_LOCATIONS:
-          return DFSUtilClient.locatedBlocks2Locations(
-              JsonUtilClient.toLocatedBlocks(json));
-        default :
-          throw new IOException("Unknown operation " + operation.name());
-        }
+        return DFSUtilClient.locatedBlocks2Locations(
+            JsonUtilClient.toLocatedBlocks(json));
       }
     }.run();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/390c2b5d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
index d32af33..6c2c674 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
@@ -33,18 +33,8 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
     GETHOMEDIRECTORY(false, HttpURLConnection.HTTP_OK),
     GETDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK, true),
 
-    /**
-     * GET_BLOCK_LOCATIONS is a private/stable API op. It returns a
-     * {@link org.apache.hadoop.hdfs.protocol.LocatedBlocks}
-     * json object.
-     */
+    /** GET_BLOCK_LOCATIONS is a private unstable op. */
     GET_BLOCK_LOCATIONS(false, HttpURLConnection.HTTP_OK),
-    /**
-     * GETFILEBLOCKLOCATIONS is the public op that complies with
-     * {@link org.apache.hadoop.fs.FileSystem#getFileBlockLocations}
-     * interface.
-     */
-    GETFILEBLOCKLOCATIONS(false, HttpURLConnection.HTTP_OK),
     GETACLSTATUS(false, HttpURLConnection.HTTP_OK),
     GETXATTRS(false, HttpURLConnection.HTTP_OK),
     GETTRASHROOT(false, HttpURLConnection.HTTP_OK),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/390c2b5d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
index 139680c..e400847 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
@@ -54,7 +54,6 @@ import javax.ws.rs.core.StreamingOutput;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -993,21 +992,6 @@ public class NamenodeWebHdfsMethods {
         return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
       }
     }
-    case GETFILEBLOCKLOCATIONS:
-    {
-      final long offsetValue = offset.getValue();
-      final Long lengthValue = length.getValue();
-
-      FileSystem fs = FileSystem.get(conf != null ?
-          conf : new Configuration());
-      BlockLocation[] locations = fs.getFileBlockLocations(
-          new org.apache.hadoop.fs.Path(fullpath),
-          offsetValue,
-          lengthValue != null? lengthValue: Long.MAX_VALUE);
-      final String js = JsonUtil.toJsonString("BlockLocations",
-          JsonUtil.toJsonMap(locations));
-      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
-    }
     case GET_BLOCK_LOCATIONS:
     {
       final long offsetValue = offset.getValue();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/390c2b5d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
index 94752f5..5752948 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
@@ -470,37 +470,4 @@ public class JsonUtil {
   public static String toJsonString(BlockStoragePolicy storagePolicy) {
     return toJsonString(BlockStoragePolicy.class, toJsonMap(storagePolicy));
   }
-
-  public static Map<String, Object> toJsonMap(BlockLocation[] locations)
-      throws IOException {
-    if(locations == null) {
-      return null;
-    }
-    final Map<String, Object> m = new TreeMap<String, Object>();
-    Object[] blockLocations = new Object[locations.length];
-    for(int i=0; i<locations.length; i++) {
-      blockLocations[i] = toJsonMap(locations[i]);
-    }
-    m.put(BlockLocation.class.getSimpleName(), blockLocations);
-    return m;
-  }
-
-  public static Map<String, Object> toJsonMap(
-      final BlockLocation blockLocation) throws IOException {
-    if (blockLocation == null) {
-      return null;
-    }
-
-    final Map<String, Object> m = new TreeMap<String, Object>();
-    m.put("length", blockLocation.getLength());
-    m.put("offset", blockLocation.getOffset());
-    m.put("corrupt", blockLocation.isCorrupt());
-    m.put("storageTypes", toJsonArray(blockLocation.getStorageTypes()));
-    m.put("storageIds", blockLocation.getStorageIds());
-    m.put("cachedHosts", blockLocation.getCachedHosts());
-    m.put("hosts", blockLocation.getHosts());
-    m.put("names", blockLocation.getNames());
-    m.put("topologyPaths", blockLocation.getTopologyPaths());
-    return m;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/390c2b5d/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
index 03834eb..84e8a57 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
@@ -50,7 +50,6 @@ The HTTP REST API supports the complete [FileSystem](../../api/org/apache/hadoop
     * [`CHECKACCESS`](#Check_access) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).access)
     * [`GETALLSTORAGEPOLICY`](#Get_all_Storage_Policies) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getAllStoragePolicies)
     * [`GETSTORAGEPOLICY`](#Get_Storage_Policy) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getStoragePolicy)
-    * [`GETFILEBLOCKLOCATIONS`](#Get_File_Block_Locations) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getFileBlockLocations)
 *   HTTP PUT
     * [`CREATE`](#Create_and_Write_to_a_File) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).create)
     * [`MKDIRS`](#Make_a_Directory) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).mkdirs)
@@ -1069,7 +1068,7 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).unsetStor
         {
             "BlockStoragePolicy": {
                 "copyOnCreateFile": false,
-                "creationFallbacks": [],
+               "creationFallbacks": [],
                 "id":7,
                 "name":"HOT",
                 "replicationFallbacks":["ARCHIVE"],
@@ -1079,51 +1078,6 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).unsetStor
 
 See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getStoragePolicy
 
-### Get File Block Locations
-
-* Submit a HTTP GET request.
-
-        curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETFILEBLOCKLOCATIONS
-
-    The client receives a response with a [`BlockLocations` JSON Object](#Block_Locations_JSON_Schema):
-
-        HTTP/1.1 200 OK
-        Content-Type: application/json
-        Transfer-Encoding: chunked
-
-        {
-          "BlockLocations" :
-          {
-            "BlockLocation":
-            [
-              {
-                "cachedHosts" : [],
-                "corrupt" : false,
-                "hosts" : ["host"],
-                "length" : 134217728,                             // length of this block
-                "names" : ["host:ip"],
-                "offset" : 0,                                     // offset of the block in the file
-                "storageIds" : ["storageid"],
-                "storageTypes" : ["DISK"],                        // enum {RAM_DISK, SSD, DISK, ARCHIVE}
-                "topologyPaths" : ["/default-rack/hostname:ip"]
-              }, {
-                "cachedHosts" : [],
-                "corrupt" : false,
-                "hosts" : ["host"],
-                "length" : 62599364,
-                "names" : ["host:ip"],
-                "offset" : 134217728,
-                "storageIds" : ["storageid"],
-                "storageTypes" : ["DISK"],
-                "topologyPaths" : ["/default-rack/hostname:ip"]
-              },
-              ...
-            ]
-          }
-        }
-
-See also: [`offset`](#Offset), [`length`](#Length), [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getFileBlockLocations
-
 Extended Attributes(XAttrs) Operations
 --------------------------------------
 
@@ -2082,146 +2036,6 @@ A `BlockStoragePolicies` JSON object represents an array of `BlockStoragePolicy`
 }
 ```
 
-#### BlockLocations JSON Schema
-
-A `BlockLocations` JSON object represents an array of `BlockLocation` JSON objects.
-
-```json
-{
-  "name"      : "BlockLocations",
-  "properties":
-  {
-    "BlockLocations":
-    {
-      "type"      : "object",
-      "properties":
-      {
-        "BlockLocation":
-        {
-          "description": "An array of BlockLocation",
-          "type"       : "array",
-          "items"      : blockLocationProperties      //See BlockLocation Properties
-        }
-      }
-    }
-  }
-}
-```
-
-See also [`BlockLocation` Properties](#BlockLocation_Properties), [`GETFILEBLOCKLOCATIONS`](#Get_File_Block_Locations), [BlockLocation](../../api/org/apache/hadoop/fs/BlockLocation.html)
-
-### BlockLocation JSON Schema
-
-```json
-{
-  "name"      : "BlockLocation",
-  "properties":
-  {
-    "BlockLocation": blockLocationProperties      //See BlockLocation Properties
-  }
-}
-```
-
-See also [`BlockLocation` Properties](#BlockLocation_Properties), [`GETFILEBLOCKLOCATIONS`](#Get_File_Block_Locations), [BlockLocation](../../api/org/apache/hadoop/fs/BlockLocation.html)
-
-#### BlockLocation Properties
-
-JavaScript syntax is used to define `blockLocationProperties` so that it can be referred in both `BlockLocation` and `BlockLocations` JSON schemas.
-
-```javascript
-var blockLocationProperties =
-{
-  "type"      : "object",
-  "properties":
-  {
-    "cachedHosts":
-    {
-      "description": "Datanode hostnames with a cached replica",
-      "type"       : "array",
-      "required"   : "true",
-      "items"      :
-      {
-        "description": "A datanode hostname",
-        "type"       : "string"
-      }
-    },
-    "corrupt":
-    {
-      "description": "True if the block is corrupted",
-      "type"       : "boolean",
-      "required"   : "true"
-    },
-    "hosts":
-    {
-      "description": "Datanode hostnames store the block",
-      "type"       : "array",
-      "required"   : "true",
-      "items"      :
-      {
-        "description": "A datanode hostname",
-        "type"       : "string"
-      }
-    },
-    "length":
-    {
-      "description": "Length of the block",
-      "type"       : "integer",
-      "required"   : "true"
-    },
-    "names":
-    {
-      "description": "Datanode IP:xferPort for accessing the block",
-      "type"       : "array",
-      "required"   : "true",
-      "items"      :
-      {
-        "description": "DatanodeIP:xferPort",
-        "type"       : "string"
-      }
-    },
-    "offset":
-    {
-      "description": "Offset of the block in the file",
-      "type"       : "integer",
-      "required"   : "true"
-    },
-    "storageIds":
-    {
-      "description": "Storage ID of each replica",
-      "type"       : "array",
-      "required"   : "true",
-      "items"      :
-      {
-        "description": "Storage ID",
-        "type"       : "string"
-      }
-    },
-    "storageTypes":
-    {
-      "description": "Storage type of each replica",
-      "type"       : "array",
-      "required"   : "true",
-      "items"      :
-      {
-        "description": "Storage type",
-        "enum"       : ["RAM_DISK", "SSD", "DISK", "ARCHIVE"]
-      }
-    },
-    "topologyPaths":
-    {
-      "description": "Datanode addresses in network topology",
-      "type"       : "array",
-      "required"   : "true",
-      "items"      :
-      {
-        "description": "/rack/host:ip",
-        "type"       : "string"
-      }
-    }
-  }
-};
-```
-
 HTTP Query Parameter Dictionary
 -------------------------------
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/390c2b5d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
index 77e5be1..de051b3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
@@ -29,7 +29,6 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.io.PrintWriter;
 import java.net.HttpURLConnection;
 import java.net.InetSocketAddress;
 import java.net.SocketException;
@@ -39,16 +38,8 @@ import java.net.URISyntaxException;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
 import java.util.Random;
 
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import javax.ws.rs.core.MediaType;
-
 import com.google.common.collect.ImmutableList;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
@@ -80,12 +71,8 @@ import org.apache.hadoop.hdfs.TestDFSClientRetries;
 import org.apache.hadoop.hdfs.TestFileCreation;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
@@ -96,8 +83,6 @@ import org.apache.hadoop.hdfs.web.resources.LengthParam;
 import org.apache.hadoop.hdfs.web.resources.NoRedirectParam;
 import org.apache.hadoop.hdfs.web.resources.OffsetParam;
 import org.apache.hadoop.hdfs.web.resources.Param;
-import org.apache.hadoop.http.HttpServer2;
-import org.apache.hadoop.http.HttpServerFunctionalTest;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryPolicy.RetryAction;
 import org.apache.hadoop.io.retry.RetryPolicy.RetryAction.RetryDecision;
@@ -114,12 +99,8 @@ import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.type.MapType;
-
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.anyLong;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
@@ -974,76 +955,6 @@ public class TestWebHDFS {
         Assert.assertTrue(storageTypes != null && storageTypes.length > 0 &&
             storageTypes[0] == StorageType.DISK);
       }
-
-      // Query webhdfs REST API to get block locations
-      InetSocketAddress addr = cluster.getNameNode().getHttpAddress();
-
-      // Case 1
-      // URL without length or offset parameters
-      URL url1 = new URL("http", addr.getHostString(), addr.getPort(),
-          WebHdfsFileSystem.PATH_PREFIX + "/foo?op=GETFILEBLOCKLOCATIONS");
-      LOG.info("Sending GETFILEBLOCKLOCATIONS request " + url1);
-
-      String response1 = getResponse(url1, "GET");
-      LOG.info("The output of GETFILEBLOCKLOCATIONS request " + response1);
-      // Parse BlockLocation array from json output using object mapper
-      BlockLocation[] locationArray1 = toBlockLocationArray(response1);
-
-      // Verify the result from rest call is same as file system api
-      verifyEquals(locations, locationArray1);
-
-      // Case 2
-      // URL contains length and offset parameters
-      URL url2 = new URL("http", addr.getHostString(), addr.getPort(),
-          WebHdfsFileSystem.PATH_PREFIX + "/foo?op=GETFILEBLOCKLOCATIONS"
-              + "&length=" + LENGTH + "&offset=" + OFFSET);
-      LOG.info("Sending GETFILEBLOCKLOCATIONS request " + url2);
-
-      String response2 = getResponse(url2, "GET");
-      LOG.info("The output of GETFILEBLOCKLOCATIONS request " + response2);
-      BlockLocation[] locationArray2 = toBlockLocationArray(response2);
-
-      verifyEquals(locations, locationArray2);
-
-      // Case 3
-      // URL contains length parameter but without offset parameters
-      URL url3 = new URL("http", addr.getHostString(), addr.getPort(),
-          WebHdfsFileSystem.PATH_PREFIX + "/foo?op=GETFILEBLOCKLOCATIONS"
-              + "&length=" + LENGTH);
-      LOG.info("Sending GETFILEBLOCKLOCATIONS request " + url3);
-
-      String response3 = getResponse(url3, "GET");
-      LOG.info("The output of GETFILEBLOCKLOCATIONS request " + response3);
-      BlockLocation[] locationArray3 = toBlockLocationArray(response3);
-
-      verifyEquals(locations, locationArray3);
-
-      // Case 4
-      // URL contains offset parameter but without length parameter
-      URL url4 = new URL("http", addr.getHostString(), addr.getPort(),
-          WebHdfsFileSystem.PATH_PREFIX + "/foo?op=GETFILEBLOCKLOCATIONS"
-              + "&offset=" + OFFSET);
-      LOG.info("Sending GETFILEBLOCKLOCATIONS request " + url4);
-
-      String response4 = getResponse(url4, "GET");
-      LOG.info("The output of GETFILEBLOCKLOCATIONS request " + response4);
-      BlockLocation[] locationArray4 = toBlockLocationArray(response4);
-
-      verifyEquals(locations, locationArray4);
-
-      // Case 5
-      // URL specifies offset exceeds the file length
-      URL url5 = new URL("http", addr.getHostString(), addr.getPort(),
-          WebHdfsFileSystem.PATH_PREFIX + "/foo?op=GETFILEBLOCKLOCATIONS"
-              + "&offset=1200");
-      LOG.info("Sending GETFILEBLOCKLOCATIONS request " + url5);
-
-      String response5 = getResponse(url5, "GET");
-      LOG.info("The output of GETFILEBLOCKLOCATIONS request " + response5);
-      BlockLocation[] locationArray5 = toBlockLocationArray(response5);
-
-      // Expected an empty array of BlockLocation
-      verifyEquals(new BlockLocation[] {}, locationArray5);
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -1051,66 +962,6 @@ public class TestWebHDFS {
     }
   }
 
-  private BlockLocation[] toBlockLocationArray(String json)
-      throws IOException {
-    ObjectMapper mapper = new ObjectMapper();
-    MapType subType = mapper.getTypeFactory().constructMapType(
-        Map.class,
-        String.class,
-        BlockLocation[].class);
-    MapType rootType = mapper.getTypeFactory().constructMapType(
-        Map.class,
-        mapper.constructType(String.class),
-        mapper.constructType(subType));
-
-    Map<String, Map<String, BlockLocation[]>> jsonMap = mapper
-        .readValue(json, rootType);
-    Map<String, BlockLocation[]> locationMap = jsonMap
-        .get("BlockLocations");
-    BlockLocation[] locationArray = locationMap.get(
-        BlockLocation.class.getSimpleName());
-    return locationArray;
-  }
-
-  private void verifyEquals(BlockLocation[] locations1,
-      BlockLocation[] locations2) throws IOException {
-    for(int i=0; i<locations1.length; i++) {
-      BlockLocation location1 = locations1[i];
-      BlockLocation location2 = locations2[i];
-      Assert.assertEquals(location1.getLength(),
-          location2.getLength());
-      Assert.assertEquals(location1.getOffset(),
-          location2.getOffset());
-      Assert.assertArrayEquals(location1.getCachedHosts(),
-          location2.getCachedHosts());
-      Assert.assertArrayEquals(location1.getHosts(),
-          location2.getHosts());
-      Assert.assertArrayEquals(location1.getNames(),
-          location2.getNames());
-      Assert.assertArrayEquals(location1.getStorageIds(),
-          location2.getStorageIds());
-      Assert.assertArrayEquals(location1.getTopologyPaths(),
-          location2.getTopologyPaths());
-      Assert.assertArrayEquals(location1.getStorageTypes(),
-          location2.getStorageTypes());
-    }
-  }
-
-  private static String getResponse(URL url, String httpRequestType)
-      throws IOException {
-    HttpURLConnection conn = null;
-    try {
-      conn = (HttpURLConnection) url.openConnection();
-      conn.setRequestMethod(httpRequestType);
-      conn.setInstanceFollowRedirects(false);
-      return IOUtils.toString(conn.getInputStream());
-    } finally {
-      if(conn != null) {
-        conn.disconnect();
-      }
-    }
-  }
-
   private WebHdfsFileSystem createWebHDFSAsTestUser(final Configuration conf,
       final URI uri, final String userName) throws Exception {
 
@@ -1467,131 +1318,4 @@ public class TestWebHDFS {
       }
     }
   }
-
-  /**
-   * A mock class to handle the {@link WebHdfsFileSystem} client
-   * request. The format of the response depends on how many of
-   * times it gets called (1 to 3 times).
-   * <p>
-   * First time call it return a wrapped json response with a
-   * IllegalArgumentException
-   * <p>
-   * Second time call it return a valid GET_BLOCK_LOCATIONS
-   * json response
-   * <p>
-   * Third time call it return a wrapped json response with
-   * a random IOException
-   *
-   */
-  public static class MockWebHdfsServlet extends HttpServlet {
-
-    private static final long serialVersionUID = 1L;
-    private static int respondTimes = 0;
-    private static final String RANDOM_EXCEPTION_MSG =
-        "This is a random exception";
-
-    @Override
-    public void doGet(HttpServletRequest request,
-        HttpServletResponse response) throws ServletException, IOException {
-      response.setHeader("Content-Type",
-          MediaType.APPLICATION_JSON);
-      String param = request.getParameter("op");
-      if(respondTimes == 0) {
-        Exception mockException = new IllegalArgumentException(
-            "Invalid value for webhdfs parameter \"op\". "
-                + "" + "No enum constant " + param);
-        sendException(request, response, mockException);
-      } else if (respondTimes == 1) {
-        sendResponse(request, response);
-      } else if (respondTimes == 2) {
-        Exception mockException = new IOException(RANDOM_EXCEPTION_MSG);
-        sendException(request, response, mockException);
-      }
-      respondTimes++;
-    }
-
-    private void sendResponse(HttpServletRequest request,
-        HttpServletResponse response) throws IOException {
-      response.setStatus(HttpServletResponse.SC_OK);
-      // Construct a LocatedBlock for testing
-      DatanodeInfo d = DFSTestUtil.getLocalDatanodeInfo();
-      DatanodeInfo[] ds = new DatanodeInfo[1];
-      ds[0] = d;
-      ExtendedBlock b1 = new ExtendedBlock("bpid", 1, 121, 1);
-      LocatedBlock l1 = new LocatedBlock(b1, ds);
-      l1.setStartOffset(0);
-      l1.setCorrupt(false);
-      List<LocatedBlock> ls = Arrays.asList(l1);
-      LocatedBlocks locatedblocks =
-          new LocatedBlocks(10, false, ls, l1,
-              true, null, null);
-
-      try (PrintWriter pw = response.getWriter()) {
-        pw.write(JsonUtil.toJsonString(locatedblocks));
-      }
-    }
-
-    private void sendException(HttpServletRequest request,
-        HttpServletResponse response,
-        Exception mockException) throws IOException {
-      response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
-      String errJs = JsonUtil.toJsonString(mockException);
-      try (PrintWriter pw = response.getWriter()) {
-        pw.write(errJs);
-      }
-    }
-  }
-
-  @Test
-  public void testGetFileBlockLocationsBackwardsCompatibility()
-      throws Exception {
-    final Configuration conf = WebHdfsTestUtil.createConf();
-    final String pathSpec = WebHdfsFileSystem.PATH_PREFIX + "/*";
-    HttpServer2 http = null;
-    try {
-      http = HttpServerFunctionalTest.createTestServer(conf);
-      http.addServlet("test", pathSpec, MockWebHdfsServlet.class);
-      http.start();
-
-      // Write the address back to configuration so
-      // WebHdfsFileSystem could connect to the mock server
-      conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY,
-          "localhost:" + http.getConnectorAddress(0).getPort());
-
-      final WebHdfsFileSystem webFS = WebHdfsTestUtil.getWebHdfsFileSystem(
-          conf, WebHdfsConstants.WEBHDFS_SCHEME);
-
-      WebHdfsFileSystem spyFs = spy(webFS);
-      BlockLocation[] locations = spyFs
-          .getFileBlockLocations(new Path("p"), 0, 100);
-
-      // Verify result
-      assertEquals(1, locations.length);
-      assertEquals(121, locations[0].getLength());
-
-      // Verify the fall back
-      // The function should be called exactly 2 times
-      // 1st time handles GETFILEBLOCKLOCATIONS and found it is not supported
-      // 2nd time fall back to handle GET_FILE_BLOCK_LOCATIONS
-      verify(spyFs, times(2)).getFileBlockLocations(any(),
-          any(), anyLong(), anyLong());
-
-      // Verify it doesn't erroneously fall back
-      // When server returns a different error, it should directly
-      // throw an exception.
-      try {
-        spyFs.getFileBlockLocations(new Path("p"), 0, 100);
-      } catch (Exception e) {
-        assertTrue(e instanceof IOException);
-        assertEquals(e.getMessage(), MockWebHdfsServlet.RANDOM_EXCEPTION_MSG);
-        // Totally this function has been called 3 times
-        verify(spyFs, times(3)).getFileBlockLocations(any(),
-            any(), anyLong(), anyLong());
-      }
-    } finally {
-      if(http != null) {
-        http.stop();
-      }
-    }
-  }
 }


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


[16/50] [abbrv] hadoop git commit: HDFS-12456. TestNamenodeMetrics.testSyncAndBlockReportMetric fails. Contributed by Bharat Viswanadham.

Posted by ae...@apache.org.
HDFS-12456. TestNamenodeMetrics.testSyncAndBlockReportMetric fails. Contributed by Bharat Viswanadham.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/50764ef3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/50764ef3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/50764ef3

Branch: refs/heads/HDFS-7240
Commit: 50764ef3e45b5dac1a8fa3af4ed04b008cbd73bd
Parents: 08d996d
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Sep 14 22:23:36 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Sep 14 22:23:36 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/50764ef3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
index d716f04..077a5f8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
@@ -889,7 +889,7 @@ public class TestNameNodeMetrics {
   public void testSyncAndBlockReportMetric() throws Exception {
     MetricsRecordBuilder rb = getMetrics(NN_METRICS);
     // We have one sync when the cluster starts up, just opening the journal
-    assertCounter("SyncsNumOps", 3L, rb);
+    assertCounter("SyncsNumOps", 4L, rb);
     // Each datanode reports in when the cluster comes up
     assertCounter("StorageBlockReportNumOps",
                   (long) DATANODE_COUNT * cluster.getStoragesPerDatanode(), rb);


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


[39/50] [abbrv] hadoop git commit: HADOOP-13714. Tighten up our compatibility guidelines for Hadoop 3

Posted by ae...@apache.org.
HADOOP-13714. Tighten up our compatibility guidelines for Hadoop 3


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7618fa91
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7618fa91
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7618fa91

Branch: refs/heads/HDFS-7240
Commit: 7618fa9194b40454405f11a25bec4e2d79506912
Parents: 38c14ef
Author: Daniel Templeton <te...@apache.org>
Authored: Sat Sep 16 09:20:33 2017 +0200
Committer: Daniel Templeton <te...@apache.org>
Committed: Sat Sep 16 09:20:33 2017 +0200

----------------------------------------------------------------------
 .../src/site/markdown/Compatibility.md          | 645 +++++++++++++++----
 .../site/markdown/InterfaceClassification.md    | 227 ++++---
 2 files changed, 675 insertions(+), 197 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7618fa91/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md b/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
index 05b18b5..4fa8c02 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
@@ -20,109 +20,276 @@ Apache Hadoop Compatibility
 Purpose
 -------
 
-This document captures the compatibility goals of the Apache Hadoop project. The different types of compatibility between Hadoop releases that affects Hadoop developers, downstream projects, and end-users are enumerated. For each type of compatibility we:
+This document captures the compatibility goals of the Apache Hadoop project.
+The different types of compatibility between Hadoop releases that affect
+Hadoop developers, downstream projects, and end-users are enumerated. For each
+type of compatibility this document will:
 
 * describe the impact on downstream projects or end-users
 * where applicable, call out the policy adopted by the Hadoop developers when incompatible changes are permitted.
 
+All Hadoop interfaces are classified according to the intended audience and
+stability in order to maintain compatibility with previous releases. See the
+[Hadoop Interface Taxonomy](./InterfaceClassification.html) for details
+about the classifications.
+
+### Target Audience
+
+This document is intended for consumption by the Hadoop developer community.
+This document describes the lens through which changes to the Hadoop project
+should be viewed. In order for end users and third party developers to have
+confidence about cross-release compatibility, the developer community must
+ensure that development efforts adhere to these policies. It is the
+responsibility of the project committers to validate that all changes either
+maintain compatibility or are explicitly marked as incompatible.
+
+Within a component Hadoop developers are free to use Private and Limited Private
+APIs, but when using components from a different module Hadoop developers
+should follow the same guidelines as third-party developers: do not
+use Private or Limited Private (unless explicitly allowed) interfaces and
+prefer instead Stable interfaces to Evolving or Unstable interfaces where
+possible. Where not possible, the preferred solution is to expand the audience
+of the API rather than introducing or perpetuating an exception to these
+compatibility guidelines. When working within a Maven module Hadoop developers
+should observe where possible the same level of restraint with regard to
+using components located in other Maven modules.
+
+Above all, Hadoop developers must be mindful of the impact of their changes.
+Stable interfaces must not change between major releases. Evolving interfaces
+must not change between minor releases. New classes and components must be
+labeled appropriately for audience and stability. See the
+[Hadoop Interface Taxonomy](./InterfaceClassification.html) for details about
+when the various labels are appropriate. As a general rule, all new interfaces
+and APIs should have the most limited labels (e.g. Private Unstable) that will
+not inhibit the intent of the interface or API.
+
+### Notational Conventions
+
+The key words "MUST" "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" are to be interpreted as
+described in [RFC 2119](http://tools.ietf.org/html/rfc2119).
+
+Deprecation
+-----------
+
+The Java API provides a @Deprecated annotation to mark an API element as
+flagged for removal. The standard meaning of the annotation is that the
+API element should not be used and may be removed in a later version.
+
+In all cases removing an element from an API is an incompatible
+change. In the case of [Stable](./InterfaceClassification.html#Stable) APIs,
+the change cannot be made between minor releases within the same major
+version. In addition, to allow consumers of the API time to adapt to the change,
+the API element to be removed should be marked as deprecated for a full major
+release before it is removed. For example, if a method is marked as deprecated
+in Hadoop 2.8, it cannot be removed until Hadoop 4.0.
+
+### Policy
+
+[Stable](./InterfaceClassification.html#Stable) API elements MUST NOT be removed
+until they have been marked as deprecated (through the @Deprecated annotation or
+other appropriate documentation) for a full major release. In the case that an
+API element was introduced as deprecated (to indicate that it is a temporary
+measure that is intended to be removed) the API element MAY be removed in the
+following major release. When modifying a
+[Stable](./InterfaceClassification.html#Stable) API, developers SHOULD prefer
+introducing a new method or endpoint and deprecating the existing one to making
+incompatible changes to the method or endpoint.
+
 Compatibility types
 -------------------
 
 ### Java API
 
-Hadoop interfaces and classes are annotated to describe the intended audience and stability in order to maintain compatibility with previous releases. See [Hadoop Interface Classification](./InterfaceClassification.html) for details.
-
-* InterfaceAudience: captures the intended audience, possible values are Public (for end users and external projects), LimitedPrivate (for other Hadoop components, and closely related projects like YARN, MapReduce, HBase etc.), and Private (for intra component use).
-* InterfaceStability: describes what types of interface changes are permitted. Possible values are Stable, Evolving, Unstable, and Deprecated.
+Developers SHOULD annotate all Hadoop interfaces and classes with the
+@InterfaceAudience and @InterfaceStability annotations to describe the
+intended audience and stability. Annotations may be at the package, class, or
+member variable or method level. Member variable and method annotations SHALL
+override class annotations, and class annotations SHALL override package
+annotations. A package, class, or member variable or method that is not
+annotated SHALL be interpreted as implicitly
+[Private](./InterfaceClassification.html#Private) and
+[Unstable](./InterfaceClassification.html#Unstable).
+
+* @InterfaceAudience captures the intended audience. Possible values are
+[Public](./InterfaceClassification.html#Public) (for end users and external
+projects), Limited[Private](./InterfaceClassification.html#Private) (for other
+Hadoop components, and closely related projects like YARN, MapReduce, HBase
+etc.), and [Private](./InterfaceClassification.html#Private)
+(for intra component use).
+* @InterfaceStability describes what types of interface changes are permitted. Possible values are [Stable](./InterfaceClassification.html#Stable), [Evolving](./InterfaceClassification.html#Evolving), and [Unstable](./InterfaceClassification.html#Unstable).
+* @Deprecated notes that the package, class, or member variable or method could potentially be removed in the future and should not be used.
 
 #### Use Cases
 
-* Public-Stable API compatibility is required to ensure end-user programs and downstream projects continue to work without modification.
-* LimitedPrivate-Stable API compatibility is required to allow upgrade of individual components across minor releases.
-* Private-Stable API compatibility is required for rolling upgrades.
+* [Public](./InterfaceClassification.html#Public)-[Stable](./InterfaceClassification.html#Stable) API compatibility is required to ensure end-user programs and downstream projects continue to work without modification.
+* [Public](./InterfaceClassification.html#Public)-[Evolving](./InterfaceClassification.html#Evolving) API compatibility is useful to make functionality available for consumption before it is fully baked.
+* Limited Private-[Stable](./InterfaceClassification.html#Stable) API compatibility is required to allow upgrade of individual components across minor releases.
+* [Private](./InterfaceClassification.html#Private)-[Stable](./InterfaceClassification.html#Stable) API compatibility is required for rolling upgrades.
+* [Private](./InterfaceClassification.html#Private)-[Unstable](./InterfaceClassification.html#Unstable) API compatibility allows internal components to evolve rapidly without concern for downstream consumers, and is how most interfaces should be labeled.
 
 #### Policy
 
-* Public-Stable APIs must be deprecated for at least one major release prior to their removal in a major release.
-* LimitedPrivate-Stable APIs can change across major releases, but not within a major release.
-* Private-Stable APIs can change across major releases, but not within a major release.
-* Classes not annotated are implicitly "Private". Class members not annotated inherit the annotations of the enclosing class.
-* Note: APIs generated from the proto files need to be compatible for rolling-upgrades. See the section on wire-compatibility for more details. The compatibility policies for APIs and wire-communication need to go hand-in-hand to address this.
-
-### Semantic compatibility
-
-Apache Hadoop strives to ensure that the behavior of APIs remains consistent over versions, though changes for correctness may result in changes in behavior. Tests and javadocs specify the API's behavior. The community is in the process of specifying some APIs more rigorously, and enhancing test suites to verify compliance with the specification, effectively creating a formal specification for the subset of behaviors that can be easily tested.
+The compatibility policy SHALL be determined by the relevant package, class, or
+member variable or method annotations.
+
+Note: APIs generated from the proto files MUST be compatible for rolling
+upgrades. See the section on wire protocol compatibility for more details. The
+compatibility policies for APIs and wire protocols must therefore go hand
+in hand.
+
+#### Semantic compatibility
+
+Apache Hadoop strives to ensure that the behavior of APIs remains consistent
+over versions, though changes for correctness may result in changes in
+behavior. API behavior SHALL be specified by the JavaDoc API documentation
+where present and complete. When JavaDoc API documentation is not available,
+behavior SHALL be specified by the behavior expected by the related unit tests.
+In cases with no JavaDoc API documentation or unit test coverage, the expected
+behavior is presumed to be obvious and SHOULD be assumed to be the minimum
+functionality implied by the interface naming. The community is in the process
+of specifying some APIs more rigorously and enhancing test suites to verify
+compliance with the specification, effectively creating a formal specification
+for the subset of behaviors that can be easily tested.
+
+The behavior of any API MAY be changed to fix incorrect behavior according to
+the stability of the API, with such a change to be accompanied by updating
+existing documentation and tests and/or adding new documentation or tests.
+
+#### Java Binary compatibility for end-user applications i.e. Apache Hadoop ABI
+
+Apache Hadoop revisions SHOUD retain binary compatability such that end-user
+applications continue to work without any modifications. Minor Apache Hadoop
+revisions within the same major revision MUST retain compatibility such that
+existing MapReduce applications (e.g. end-user applications and projects such
+as Apache Pig, Apache Hive, et al), existing YARN applications (e.g.
+end-user applications and projects such as Apache Spark, Apache Tez et al),
+and applications that accesses HDFS directly (e.g. end-user applications and
+projects such as Apache HBase, Apache Flume, et al) work unmodified and without
+recompilation when used with any Apache Hadoop cluster within the same major
+release as the original build target.
+
+For MapReduce applications in particular, i.e. applications using the
+org.apache.hadoop.mapred and/or org.apache.hadoop.mapreduce APIs, the developer
+community SHALL support binary compatibility across major releases. The
+MapReduce APIs SHALL be supported compatibly across major releases. See
+[Compatibility for MapReduce applications between hadoop-1.x and hadoop-2.x](../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduce_Compatibility_Hadoop1_Hadoop2.html) for more details.
+
+Some applications may be affected by changes to disk layouts or other internal
+changes. See the sections that follow for policies on how incompatible
+changes to non-API interfaces are handled.
+
+### Native Dependencies
+
+Hadoop includes several native components, including compression, the
+container executor binary, and various native integrations. These native
+components introduce a set of native dependencies for Hadoop, both at compile
+time and at runtime, such as cmake, gcc, zlib, etc. This set of native
+dependencies is part of the Hadoop ABI.
 
 #### Policy
 
-The behavior of API may be changed to fix incorrect behavior, such a change to be accompanied by updating existing buggy tests or adding tests in cases there were none prior to the change.
+The minimum required versions of the native components on which Hadoop depends
+at compile time and/or runtime SHALL be considered
+[Stable](./InterfaceClassification.html#Stable). Changes to the minimum
+required versions MUST NOT increase between minor releases within a major
+version.
 
-### Wire compatibility
+### Wire Protocols
 
-Wire compatibility concerns data being transmitted over the wire between Hadoop processes. Hadoop uses Protocol Buffers for most RPC communication. Preserving compatibility requires prohibiting modification as described below. Non-RPC communication should be considered as well, for example using HTTP to transfer an HDFS image as part of snapshotting or transferring MapTask output. The potential communications can be categorized as follows:
+Wire compatibility concerns data being transmitted "over the wire" between
+Hadoop processes. Hadoop uses
+[Protocol Buffers](https://developers.google.com/protocol-buffers/) for most
+RPC communication. Preserving compatibility requires prohibiting modification
+as described below. Non-RPC communication should be considered as well, for
+example using HTTP to transfer an HDFS image as part of snapshotting or
+transferring MapReduce map task output. The communications can be categorized as
+follows:
 
 * Client-Server: communication between Hadoop clients and servers (e.g., the HDFS client to NameNode protocol, or the YARN client to ResourceManager protocol).
-* Client-Server (Admin): It is worth distinguishing a subset of the Client-Server protocols used solely by administrative commands (e.g., the HAAdmin protocol) as these protocols only impact administrators who can tolerate changes that end users (which use general Client-Server protocols) can not.
+* Client-Server (Admin): It is worth distinguishing a subset of the Client-Server protocols used solely by administrative commands (e.g., the HAAdmin protocol) as these protocols only impact administrators who can tolerate changes that end users (which use general Client-Server protocols) cannot.
 * Server-Server: communication between servers (e.g., the protocol between the DataNode and NameNode, or NodeManager and ResourceManager)
 
-#### Use Cases
+#### Protocol Dependencies
 
-* Client-Server compatibility is required to allow users to continue using the old clients even after upgrading the server (cluster) to a later version (or vice versa). For example, a Hadoop 2.1.0 client talking to a Hadoop 2.3.0 cluster.
-* Client-Server compatibility is also required to allow users to upgrade the client before upgrading the server (cluster). For example, a Hadoop 2.4.0 client talking to a Hadoop 2.3.0 cluster. This allows deployment of client-side bug fixes ahead of full cluster upgrades. Note that new cluster features invoked by new client APIs or shell commands will not be usable. YARN applications that attempt to use new APIs (including new fields in data structures) that have not yet been deployed to the cluster can expect link exceptions.
-* Client-Server compatibility is also required to allow upgrading individual components without upgrading others. For example, upgrade HDFS from version 2.1.0 to 2.2.0 without upgrading MapReduce.
-* Server-Server compatibility is required to allow mixed versions within an active cluster so the cluster may be upgraded without downtime in a rolling fashion.
+The components of Apache Hadoop may have dependencies that include their own
+protocols, such as Zookeeper, S3, Kerberos, etc. These protocol dependencies
+SHALL be treated as internal protocols and governed by the same policy.
 
-#### Policy
+#### Transports
+
+In addition to compatibility of the protocols themselves, maintaining
+cross-version communications requires that the transports supported also be
+stable. The most likely source of transport changes stems from secure
+transports, such as SSL. Upgrading a service from SSLv2 to SSLv3 may break
+existing SSLv2 clients. The minimum supported major version of any transports
+MUST not increase across minor releases within a major version.
 
-* Both Client-Server and Server-Server compatibility is preserved within a major release. (Different policies for different categories are yet to be considered.)
-* Compatibility can be broken only at a major release, though breaking compatibility even at major releases has grave consequences and should be discussed in the Hadoop community.
-* Hadoop protocols are defined in .proto (ProtocolBuffers) files. Client-Server protocols and Server-Server protocol .proto files are marked as stable. When a .proto file is marked as stable it means that changes should be made in a compatible fashion as described below:
-    * The following changes are compatible and are allowed at any time:
-        * Add an optional field, with the expectation that the code deals with the field missing due to communication with an older version of the code.
-        * Add a new rpc/method to the service
-        * Add a new optional request to a Message
-        * Rename a field
-        * Rename a .proto file
-        * Change .proto annotations that effect code generation (e.g. name of java package)
-    * The following changes are incompatible but can be considered only at a major release
-        * Change the rpc/method name
-        * Change the rpc/method parameter type or return type
-        * Remove an rpc/method
-        * Change the service name
-        * Change the name of a Message
-        * Modify a field type in an incompatible way (as defined recursively)
-        * Change an optional field to required
-        * Add or delete a required field
-        * Delete an optional field as long as the optional field has reasonable defaults to allow deletions
-    * The following changes are incompatible and hence never allowed
-        * Change a field id
-        * Reuse an old field that was previously deleted.
-        * Field numbers are cheap and changing and reusing is not a good idea.
-
-### Java Binary compatibility for end-user applications i.e. Apache Hadoop ABI
-
-As Apache Hadoop revisions are upgraded end-users reasonably expect that their applications should continue to work without any modifications. This is fulfilled as a result of supporting API compatibility, Semantic compatibility and Wire compatibility.
-
-However, Apache Hadoop is a very complex, distributed system and services a very wide variety of use-cases. In particular, Apache Hadoop MapReduce is a very, very wide API; in the sense that end-users may make wide-ranging assumptions such as layout of the local disk when their map/reduce tasks are executing, environment variables for their tasks etc. In such cases, it becomes very hard to fully specify, and support, absolute compatibility.
-
-#### Use cases
-
-* Existing MapReduce applications, including jars of existing packaged end-user applications and projects such as Apache Pig, Apache Hive, Cascading etc. should work unmodified when pointed to an upgraded Apache Hadoop cluster within a major release.
-* Existing YARN applications, including jars of existing packaged end-user applications and projects such as Apache Tez etc. should work unmodified when pointed to an upgraded Apache Hadoop cluster within a major release.
-* Existing applications which transfer data in/out of HDFS, including jars of existing packaged end-user applications and frameworks such as Apache Flume, should work unmodified when pointed to an upgraded Apache Hadoop cluster within a major release.
+Service ports are considered as part of the transport mechanism. Fixed
+service port numbers MUST be kept consistent to prevent breaking clients.
 
 #### Policy
 
-* Existing MapReduce, YARN & HDFS applications and frameworks should work unmodified within a major release i.e. Apache Hadoop ABI is supported.
-* A very minor fraction of applications maybe affected by changes to disk layouts etc., the developer community will strive to minimize these changes and will not make them within a minor version. In more egregious cases, we will consider strongly reverting these breaking changes and invalidating offending releases if necessary.
-* In particular for MapReduce applications, the developer community will try our best to support providing binary compatibility across major releases e.g. applications using org.apache.hadoop.mapred.
-* APIs are supported compatibly across hadoop-1.x and hadoop-2.x. See [Compatibility for MapReduce applications between hadoop-1.x and hadoop-2.x](../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduce_Compatibility_Hadoop1_Hadoop2.html) for more details.
+Hadoop wire protocols are defined in .proto (ProtocolBuffers) files.
+Client-Server and Server-Server protocols SHALL be classified according to the
+audience and stability classifications noted in their .proto files. In cases
+where no classifications are present, the protocols SHOULD be assumed to be
+[Private](./InterfaceClassification.html#Private) and
+[Stable](./InterfaceClassification.html#Stable).
+
+The following changes to a .proto file SHALL be considered compatible:
+
+* Add an optional field, with the expectation that the code deals with the field missing due to communication with an older version of the code
+* Add a new rpc/method to the service
+* Add a new optional request to a Message
+* Rename a field
+* Rename a .proto file
+* Change .proto annotations that effect code generation (e.g. name of java package)
+
+The following changes to a .proto file SHALL be considered incompatible:
+
+* Change an rpc/method name
+* Change an rpc/method parameter type or return type
+* Remove an rpc/method
+* Change the service name
+* Change the name of a Message
+* Modify a field type in an incompatible way (as defined recursively)
+* Change an optional field to required
+* Add or delete a required field
+* Delete an optional field as long as the optional field has reasonable defaults to allow deletions
+
+The following changes to a .proto file SHALL be considered incompatible:
+
+* Change a field id
+* Reuse an old field that was previously deleted.
+
+Hadoop wire protocols that are not defined via .proto files SHOULD be considered
+to be [Private](./InterfaceClassification.html#Private) and
+[Stable](./InterfaceClassification.html#Stable).
+
+In addition to the limitations imposed by being
+[Stable](./InterfaceClassification.html#Stable), Hadoop's wire protocols
+MUST also be forward compatible across minor releases within a major version
+according to the following:
+
+* Client-Server compatibility MUST be maintained so as to allow users to continue using older clients even after upgrading the server (cluster) to a later version (or vice versa). For example, a Hadoop 2.1.0 client talking to a Hadoop 2.3.0 cluster.
+* Client-Server compatibility MUST be maintained so as to allow users to upgrade the client before upgrading the server (cluster). For example, a Hadoop 2.4.0 client talking to a Hadoop 2.3.0 cluster. This allows deployment of client-side bug fixes ahead of full cluster upgrades. Note that new cluster features invoked by new client APIs or shell commands will not be usable. YARN applications that attempt to use new APIs (including new fields in data structures) that have not yet been deployed to the cluster can expect link exceptions.
+* Client-Server compatibility MUST be maintained so as to allow upgrading individual components without upgrading others. For example, upgrade HDFS from version 2.1.0 to 2.2.0 without upgrading MapReduce.
+* Server-Server compatibility MUST be maintained so as to allow mixed versions within an active cluster so the cluster may be upgraded without downtime in a rolling fashion.
+
+New transport mechanisms MUST only be introduced with minor or major version
+changes. Existing transport mechanisms MUST continue to be supported across
+minor versions within a major version. Service port numbers MUST remain
+consistent across minor version numbers within a major version.
 
 ### REST APIs
 
-REST API compatibility corresponds to both the requests (URLs) and responses to each request (content, which may contain other URLs). Hadoop REST APIs are specifically meant for stable use by clients across releases, even major ones. The following are the exposed REST APIs:
+REST API compatibility applies to the REST endpoints (URLs) and response data
+format. Hadoop REST APIs are specifically meant for stable use by clients across
+releases, even major ones. The following is a non-exhaustive list of the
+exposed REST APIs:
 
-* [WebHDFS](../hadoop-hdfs/WebHDFS.html) - Stable
+* [WebHDFS](../hadoop-hdfs/WebHDFS.html)
 * [ResourceManager](../../hadoop-yarn/hadoop-yarn-site/ResourceManagerRest.html)
 * [NodeManager](../../hadoop-yarn/hadoop-yarn-site/NodeManagerRest.html)
 * [MR Application Master](../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapredAppMasterRest.html)
@@ -130,134 +297,390 @@ REST API compatibility corresponds to both the requests (URLs) and responses to
 * [Timeline Server v1 REST API](../../hadoop-yarn/hadoop-yarn-site/TimelineServer.html)
 * [Timeline Service v2 REST API](../../hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html)
 
+Each API has an API-specific version number. Any incompatible changes MUST
+increment the API version number.
+
+#### Policy
+
+The Hadoop REST APIs SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Evolving](./InterfaceClassification.html#Evolving). With respect to API version
+numbers, the Hadoop REST APIs SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Stable](./InterfaceClassification.html#Stable), i.e. no incompatible changes
+are allowed to within an API version number.
+
+### Log Output
+
+The Hadoop daemons and CLIs produce log output via Log4j that is intended to
+aid administrators and developers in understanding and troubleshooting cluster
+behavior. Log messages are intended for human consumption, though automation
+use cases are also supported.
+
 #### Policy
 
-The APIs annotated stable in the text above preserve compatibility across at least one major release, and maybe deprecated by a newer version of the REST API in a major release.
+All log output SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Evolving](./InterfaceClassification.html#Evolving).
+
+### Audit Log Output
+
+Several components have audit logging systems that record system information in
+a machine readable format. Incompatible changes to that data format may break
+existing automation utilities. For the audit log, an incompatible change is any
+change that changes the format such that existing parsers no longer can parse
+the logs.
+
+#### Policy
+
+All audit log output SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Stable](./InterfaceClassification.html#Stable). Any change to the
+data format SHALL be considered an incompatible change.
 
 ### Metrics/JMX
 
-While the Metrics API compatibility is governed by Java API compatibility, the actual metrics exposed by Hadoop need to be compatible for users to be able to automate using them (scripts etc.). Adding additional metrics is compatible. Modifying (e.g. changing the unit or measurement) or removing existing metrics breaks compatibility. Similarly, changes to JMX MBean object names also break compatibility.
+While the Metrics API compatibility is governed by Java API compatibility, the
+Metrics data format exposed by Hadoop MUST be maintained as compatible for
+consumers of the data, e.g. for automation tasks.
 
 #### Policy
 
-Metrics should preserve compatibility within the major release.
+The data format exposed via Metrics SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Stable](./InterfaceClassification.html#Stable).
 
 ### File formats & Metadata
 
-User and system level data (including metadata) is stored in files of different formats. Changes to the metadata or the file formats used to store data/metadata can lead to incompatibilities between versions.
+User and system level data (including metadata) is stored in files of various
+formats. Changes to the metadata or the file formats used to store
+data/metadata can lead to incompatibilities between versions. Each class of file
+format is addressed below.
 
 #### User-level file formats
 
-Changes to formats that end-users use to store their data can prevent them from accessing the data in later releases, and hence it is highly important to keep those file-formats compatible. One can always add a "new" format improving upon an existing format. Examples of these formats include har, war, SequenceFileFormat etc.
+Changes to formats that end users use to store their data can prevent them from
+accessing the data in later releases, and hence are important to be compatible.
+Examples of these formats include har, war, SequenceFileFormat, etc.
 
 ##### Policy
 
-* Non-forward-compatible user-file format changes are restricted to major releases. When user-file formats change, new releases are expected to read existing formats, but may write data in formats incompatible with prior releases. Also, the community shall prefer to create a new format that programs must opt in to instead of making incompatible changes to existing formats.
+User-level file formats SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Stable](./InterfaceClassification.html#Stable). User-lever file
+format changes SHOULD be made forward compatible across major releases and MUST
+be made forward compatible within a major release. The developer community
+SHOULD prefer the creation of a new derivative file format to making
+incompatible changes to an existing file format. Such new file formats MUST be
+created as opt-in, meaning that users must be able to continue using the
+existing compatible format until and unless they explicitly opt in to using
+the new file format.
 
-#### System-internal file formats
+#### System-internal data schemas
 
-Hadoop internal data is also stored in files and again changing these formats can lead to incompatibilities. While such changes are not as devastating as the user-level file formats, a policy on when the compatibility can be broken is important.
+Hadoop internal data may also be stored in files or other data stores. Changing
+the schemas of these data stores can lead to incompatibilities.
 
 ##### MapReduce
 
 MapReduce uses formats like I-File to store MapReduce-specific data.
 
-##### Policy
+###### Policy
 
-MapReduce-internal formats like IFile maintain compatibility within a major release. Changes to these formats can cause in-flight jobs to fail and hence we should ensure newer clients can fetch shuffle-data from old servers in a compatible manner.
+All MapReduce-internal file formats, such as I-File format or the job history
+server's jhist file format, SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Stable](./InterfaceClassification.html#Stable).
 
 ##### HDFS Metadata
 
-HDFS persists metadata (the image and edit logs) in a particular format. Incompatible changes to either the format or the metadata prevent subsequent releases from reading older metadata. Such incompatible changes might require an HDFS "upgrade" to convert the metadata to make it accessible. Some changes can require more than one such "upgrades".
+HDFS persists metadata (the image and edit logs) in a private file format.
+Incompatible changes to either the format or the metadata prevent subsequent
+releases from reading older metadata. Incompatible changes MUST include a
+process by which existing metadata may be upgraded. Changes SHALL be
+allowed to require more than one upgrade. Incompatible changes MUST result in
+the metadata version number being incremented.
 
-Depending on the degree of incompatibility in the changes, the following potential scenarios can arise:
+Depending on the degree of incompatibility in the changes, the following
+potential scenarios can arise:
 
 * Automatic: The image upgrades automatically, no need for an explicit "upgrade".
 * Direct: The image is upgradable, but might require one explicit release "upgrade".
 * Indirect: The image is upgradable, but might require upgrading to intermediate release(s) first.
 * Not upgradeable: The image is not upgradeable.
 
-##### Policy
-
-* A release upgrade must allow a cluster to roll-back to the older version and its older disk format. The rollback needs to restore the original data, but not required to restore the updated data.
-* HDFS metadata changes must be upgradeable via any of the upgrade paths - automatic, direct or indirect.
-* More detailed policies based on the kind of upgrade are yet to be considered.
+HDFS data nodes store data in a private directory structure. The schema of that
+directory structure must remain stable to retain compatibility.
+
+###### Policy
+
+The HDFS metadata format SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Evolving](./InterfaceClassification.html#Evolving). Incompatible
+changes MUST include a process by which existing metada may be upgraded. The
+upgrade process MUST allow the cluster metadata to be rolled back to the older
+version and its older disk format. The rollback MUST restore the original data
+but is not REQUIRED to restore the updated data. Any incompatible change
+to the format MUST result in the major version number of the schema being
+incremented.
+
+The data node directory format SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Evolving](./InterfaceClassification.html#Evolving).
+
+##### AWS S3A Guard Metadata
+
+For each operation in the Hadoop S3 client (s3a) that reads or modifies
+file metadata, a shadow copy of that file metadata is stored in a separate
+metadata store, which offers HDFS-like consistency for the metadata, and may
+also provide faster lookups for things like file status or directory listings.
+S3A guard tables are created with a version marker which indicates
+compatibility.
+
+###### Policy
+
+The S3A guard metadata schema SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Unstable](./InterfaceClassification.html#Unstable). Any incompatible change
+to the schema MUST result in the version number of the schema being incremented.
+
+##### YARN Resource Manager State Store
+
+The YARN resource manager stores information about the cluster state in an
+external state store for use in fail over and recovery. If the schema used for
+the state store data does not remain compatible, the resource manager will not
+be able to recover its state and will fail to start. The state store data
+schema includes a version number that indicates compatibility.
+
+###### Policy
+
+The YARN resource manager state store data schema SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Evolving](./InterfaceClassification.html#Evolving). Any incompatible change
+to the schema MUST result in the major version number of the schema being
+incremented. Any compatible change to the schema MUST result in the minor
+version number being incremented.
+
+##### YARN Node Manager State Store
+
+The YARN node manager stores information about the node state in an
+external state store for use in recovery. If the schema used for the state
+store data does not remain compatible, the node manager will not
+be able to recover its state and will fail to start. The state store data
+schema includes a version number that indicates compatibility.
+
+###### Policy
+
+The YARN node manager state store data schema SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Evolving](./InterfaceClassification.html#Evolving). Any incompatible change
+to the schema MUST result in the major version number of the schema being
+incremented. Any compatible change to the schema MUST result in the minor
+version number being incremented.
+
+##### YARN Federation State Store
+
+The YARN resource manager federation service stores information about the
+federated clusters, running applications, and routing policies in an
+external state store for use in replication and recovery. If the schema used
+for the state store data does not remain compatible, the federation service
+will fail to initialize. The state store data schema includes a version number
+that indicates compatibility.
+
+###### Policy
+
+The YARN federation service state store data schema SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Evolving](./InterfaceClassification.html#Evolving). Any incompatible change
+to the schema MUST result in the major version number of the schema being
+incremented. Any compatible change to the schema MUST result in the minor
+version number being incremented.
 
 ### Command Line Interface (CLI)
 
-The Hadoop command line programs may be used either directly via the system shell or via shell scripts. Changing the path of a command, removing or renaming command line options, the order of arguments, or the command return code and output break compatibility and may adversely affect users.
+The Hadoop command line programs may be used either directly via the system
+shell or via shell scripts. The CLIs include both the user-facing commands, such
+as the hdfs command or the yarn command, and the admin-facing commands, such as
+the scripts used to start and stop daemons.  Changing the path of a command,
+removing or renaming command line options, the order of arguments, or the
+command return codes and output break compatibility and adversely affect users.
 
 #### Policy
 
-CLI commands are to be deprecated (warning when used) for one major release before they are removed or incompatibly modified in a subsequent major release.
+All Hadoop CLI paths, usage, and output SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Stable](./InterfaceClassification.html#Stable).
+Note that the CLI output SHALL be considered distinct from the log output
+generated by the Hadoop CLIs. The latter SHALL be governed by the policy on log
+output. Note also that for CLI output, all changes SHALL be considered
+incompatible changes.
 
 ### Web UI
 
-Web UI, particularly the content and layout of web pages, changes could potentially interfere with attempts to screen scrape the web pages for information.
+Web UI, particularly the content and layout of web pages, changes could
+potentially interfere with attempts to screen scrape the web pages for
+information. The Hadoop Web UI pages, however, are not meant to be scraped, e.g.
+for automation purposes. Users are expected to use REST APIs to programmatically
+access cluster information.
 
 #### Policy
 
-Web pages are not meant to be scraped and hence incompatible changes to them are allowed at any time. Users are expected to use REST APIs to get any information.
+The Hadoop Web UI SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Unstable](./InterfaceClassification.html#Unstable).
 
 ### Hadoop Configuration Files
 
-Users use (1) Hadoop-defined properties to configure and provide hints to Hadoop and (2) custom properties to pass information to jobs. Hence, compatibility of config properties is two-fold:
+Users use Hadoop-defined properties to configure and provide hints to Hadoop and
+custom properties to pass information to jobs. Users are encouraged to avoid
+using custom configuration property names that conflict with the namespace of
+Hadoop-defined properties and should avoid using any prefixes used by Hadoop,
+e.g. hadoop, io, ipc, fs, net, file, ftp, s3, kfs, ha, file, dfs, mapred,
+mapreduce, and yarn.
+
+#### Policy
+
+Hadoop-defined properties (names and meanings) SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Stable](./InterfaceClassification.html#Stable). The units implied by a
+Hadoop-defined property MUST NOT change, even
+across major versions. Default values of Hadoop-defined properties SHALL be
+considered [Public](./InterfaceClassification.html#Public) and
+[Evolving](./InterfaceClassification.html#Evolving).
+
+### Log4j Configuration Files
 
-* Modifying key-names, units of values, and default values of Hadoop-defined properties.
-* Custom configuration property keys should not conflict with the namespace of Hadoop-defined properties. Typically, users should avoid using prefixes used by Hadoop: hadoop, io, ipc, fs, net, file, ftp, s3, kfs, ha, file, dfs, mapred, mapreduce, yarn.
+The log output produced by Hadoop daemons and CLIs is governed by a set of
+configuration files. These files control the minimum level of log message that
+will be output by the various components of Hadoop, as well as where and how
+those messages are stored.
 
 #### Policy
 
-* Hadoop-defined properties are to be deprecated at least for one major release before being removed. Modifying units for existing properties is not allowed.
-* The default values of Hadoop-defined properties can be changed across minor/major releases, but will remain the same across point releases within a minor release.
-* Currently, there is NO explicit policy regarding when new prefixes can be added/removed, and the list of prefixes to be avoided for custom configuration properties. However, as noted above, users should avoid using prefixes used by Hadoop: hadoop, io, ipc, fs, net, file, ftp, s3, kfs, ha, file, dfs, mapred, mapreduce, yarn.
+All Log4j configurations SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Evolving](./InterfaceClassification.html#Evolving).
 
 ### Directory Structure
 
-Source code, artifacts (source and tests), user logs, configuration files, output and job history are all stored on disk either local file system or HDFS. Changing the directory structure of these user-accessible files break compatibility, even in cases where the original path is preserved via symbolic links (if, for example, the path is accessed by a servlet that is configured to not follow symbolic links).
+Source code, artifacts (source and tests), user logs, configuration files,
+output, and job history are all stored on disk either local file system or HDFS.
+Changing the directory structure of these user-accessible files can break
+compatibility, even in cases where the original path is preserved via symbolic
+links (such as when the path is accessed by a servlet that is configured to
+not follow symbolic links).
 
 #### Policy
 
-* The layout of source code and build artifacts can change anytime, particularly so across major versions. Within a major version, the developers will attempt (no guarantees) to preserve the directory structure; however, individual files can be added/moved/deleted. The best way to ensure patches stay in sync with the code is to get them committed to the Apache source tree.
-* The directory structure of configuration files, user logs, and job history will be preserved across minor and point releases within a major release.
+The layout of source code and build artifacts SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Unstable](./InterfaceClassification.html#Unstable). Within a major version,
+the developer community SHOULD preserve the
+overall directory structure, though individual files MAY be added, moved, or
+deleted with no warning.
+
+The directory structure of configuration files, user logs, and job history SHALL
+be considered [Public](./InterfaceClassification.html#Public) and
+[Evolving](./InterfaceClassification.html#Evolving).
 
 ### Java Classpath
 
-User applications built against Hadoop might add all Hadoop jars (including Hadoop's library dependencies) to the application's classpath. Adding new dependencies or updating the version of existing dependencies may interfere with those in applications' classpaths.
+Hadoop provides several client artifacts that applications use to interact
+with the system. These artifacts typically have their own dependencies on
+common libraries. In the cases where these dependencies are exposed to
+end user applications or downstream consumers (i.e. not
+[shaded](https://stackoverflow.com/questions/13620281/what-is-the-maven-shade-plugin-used-for-and-why-would-you-want-to-relocate-java))
+changes to these dependencies can be disruptive. Developers are strongly
+encouraged to avoid exposing dependencies to clients by using techniques
+such as
+[shading](https://stackoverflow.com/questions/13620281/what-is-the-maven-shade-plugin-used-for-and-why-would-you-want-to-relocate-java).
+
+With regard to dependencies, adding a dependency is an incompatible change,
+whereas removing a dependency is a compatible change.
+
+Some user applications built against Hadoop may add all Hadoop JAR files
+(including Hadoop's library dependencies) to the application's classpath.
+Adding new dependencies or updating the versions of existing dependencies may
+interfere with those in applications' classpaths and hence their correct
+operation. Users are therefore discouraged from adopting this practice.
 
 #### Policy
 
-Currently, there is NO policy on when Hadoop's dependencies can change.
+The set of dependencies exposed by the Hadoop client artifacts SHALL be
+considered [Public](./InterfaceClassification.html#Public) and
+[Stable](./InterfaceClassification.html#Stable). Any dependencies that are not
+exposed to clients (either because they are shaded or only exist in non-client
+artifacts) SHALL be considered [Private](./InterfaceClassification.html#Private)
+and [Unstable](./InterfaceClassification.html#Unstable)
 
 ### Environment variables
 
-Users and related projects often utilize the exported environment variables (eg HADOOP\_CONF\_DIR), therefore removing or renaming environment variables is an incompatible change.
+Users and related projects often utilize the environment variables exported by
+Hadoop (e.g. HADOOP\_CONF\_DIR). Removing or renaming environment variables can
+therefore impact end user applications.
 
 #### Policy
 
-Currently, there is NO policy on when the environment variables can change. Developers try to limit changes to major releases.
+The environment variables consumed by Hadoop and the environment variables made
+accessible to applications through YARN SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Evolving](./InterfaceClassification.html#Evolving).
+The developer community SHOULD limit changes to major releases.
 
 ### Build artifacts
 
-Hadoop uses maven for project management and changing the artifacts can affect existing user workflows.
+Hadoop uses Maven for project management. Changes to the contents of
+generated artifacts can impact existing user applications.
 
 #### Policy
 
-* Test artifacts: The test jars generated are strictly for internal use and are not expected to be used outside of Hadoop, similar to APIs annotated @Private, @Unstable.
-* Built artifacts: The hadoop-client artifact (maven groupId:artifactId) stays compatible within a major release, while the other artifacts can change in incompatible ways.
+The contents of Hadoop test artifacts SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Unstable](./InterfaceClassification.html#Unstable). Test artifacts include
+all JAR files generated from test source code and all JAR files that include
+"tests" in the file name.
+
+The Hadoop client artifacts SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Stable](./InterfaceClassification.html#Stable). Client artifacts are the
+following:
+
+* hadoop-client
+* hadoop-client-api
+* hadoop-client-minicluster
+* hadoop-client-runtime
+* hadoop-hdfs-client
+* hadoop-hdfs-native-client
+* hadoop-mapreduce-client-app
+* hadoop-mapreduce-client-common
+* hadoop-mapreduce-client-core
+* hadoop-mapreduce-client-hs
+* hadoop-mapreduce-client-hs-plugins
+* hadoop-mapreduce-client-jobclient
+* hadoop-mapreduce-client-nativetask
+* hadoop-mapreduce-client-shuffle
+* hadoop-yarn-client
+
+All other build artifacts SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Unstable](./InterfaceClassification.html#Unstable).
 
 ### Hardware/Software Requirements
 
-To keep up with the latest advances in hardware, operating systems, JVMs, and other software, new Hadoop releases or some of their features might require higher versions of the same. For a specific environment, upgrading Hadoop might require upgrading other dependent software components.
+To keep up with the latest advances in hardware, operating systems, JVMs, and
+other software, new Hadoop releases may include features that require
+newer hardware, operating systems releases, or JVM versions than previous
+Hadoop releases. For a specific environment, upgrading Hadoop might require
+upgrading other dependent software components.
 
 #### Policies
 
 * Hardware
     * Architecture: The community has no plans to restrict Hadoop to specific architectures, but can have family-specific optimizations.
-    * Minimum resources: While there are no guarantees on the minimum resources required by Hadoop daemons, the community attempts to not increase requirements within a minor release.
-* Operating Systems: The community will attempt to maintain the same OS requirements (OS kernel versions) within a minor release. Currently GNU/Linux and Microsoft Windows are the OSes officially supported by the community while Apache Hadoop is known to work reasonably well on other OSes such as Apple MacOSX, Solaris etc.
-* The JVM requirements will not change across point releases within the same minor release except if the JVM version under question becomes unsupported. Minor/major releases might require later versions of JVM for some/all of the supported operating systems.
-* Other software: The community tries to maintain the minimum versions of additional software required by Hadoop. For example, ssh, kerberos etc.
+    * Minimum resources: While there are no guarantees on the minimum resources required by Hadoop daemons, the developer community SHOULD avoid increasing requirements within a minor release.
+* Operating Systems: The community SHOULD maintain the same minimum OS requirements (OS kernel versions) within a minor release. Currently GNU/Linux and Microsoft Windows are the OSes officially supported by the community, while Apache Hadoop is known to work reasonably well on other OSes such as Apple MacOSX, Solaris, etc.
+* The JVM requirements SHALL NOT change across minor releases within the same major release unless the JVM version in question becomes unsupported. The JVM version requirement MAY be different for different operating systems or even operating system releases.
+* File systems supported by Hadoop, e.g. through the HDFS FileSystem API, SHOULD not become unsupported between minor releases within a major version unless a migration path to an alternate client implementation is available.
 
 References
 ----------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7618fa91/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md b/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
index c7309ab..451f9be 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
@@ -66,54 +66,103 @@ Hadoop uses the following kinds of audience in order of increasing/wider visibil
 
 #### Private
 
-The interface is for internal use within the project (such as HDFS or MapReduce)
-and should not be used by applications or by other projects. It is subject to
-change at anytime without notice. Most interfaces of a project are Private (also
-referred to as project-private).
+A Private interface is for internal use within the project (such as HDFS or
+MapReduce) and should not be used by applications or by other projects. Most
+interfaces of a project are Private (also referred to as project-private).
+Unless an interface is intentionally exposed for external consumption, it should
+be marked Private.
 
 #### Limited-Private
 
-The interface is used by a specified set of projects or systems (typically
-closely related projects). Other projects or systems should not use the
-interface. Changes to the interface will be communicated/negotiated with the
+A Limited-Private interface is used by a specified set of projects or systems
+(typically closely related projects). Other projects or systems should not use
+the interface. Changes to the interface will be communicated/negotiated with the
 specified projects. For example, in the Hadoop project, some interfaces are
 LimitedPrivate{HDFS, MapReduce} in that they are private to the HDFS and
 MapReduce projects.
 
 #### Public
 
-The interface is for general use by any application.
+A Public interface is for general use by any application.
+
+### Change Compatibility
+
+Changes to an API fall into two broad categories: compatible and incompatible.
+A compatible change is a change that meets the following criteria:
+
+* no existing capabilities are removed,
+* no existing capabilities are modified in a way that prevents their use by clients that were constructed to use the interface prior to the change, and
+* no capabilities are added that require changes to clients that were constructed to use the interface prior to the change.
+
+Any change that does not meet these three criteria is an incompatible change.
+Stated simply a compatible change will not break existing clients.  These
+examples are compatible changes:
+
+* adding a method to a Java class,
+* adding an optional parameter to a RESTful web service, or
+* adding a tag to an XML document.
+* making the audience annotation of an interface more broad (e.g. from Private to Public) or the change compatibility annotation more restrictive (e.g. from Evolving to Stable)
+
+These examples are incompatible changes:
+
+* removing a method from a Java class,
+* adding a method to a Java interface,
+* adding a required parameter to a RESTful web service, or
+* renaming a field in a JSON document.
+* making the audience annotation of an interface less broad (e.g. from Public to Limited Private) or the change compatibility annotation more restrictive (e.g. from Evolving to Unstable)
 
 ### Stability
 
-Stability denotes how stable an interface is, as in when incompatible changes to
-the interface are allowed. Hadoop APIs have the following levels of stability.
+Stability denotes how stable an interface is and when compatible and
+incompatible changes to the interface are allowed. Hadoop APIs have the
+following levels of stability.
 
 #### Stable
 
-Can evolve while retaining compatibility for minor release boundaries; in other
-words, incompatible changes to APIs marked as Stable are allowed only at major
-releases (i.e. at m.0).
+A Stable interface is exposed as a preferred means of communication. A Stable
+interface is expected not to change incompatibly within a major release and
+hence serves as a safe development target. A Stable interface may evolve
+compatibly between minor releases.
+
+Incompatible changes allowed: major (X.0.0)
+Compatible changes allowed: maintenance (x.Y.0)
 
 #### Evolving
 
-Evolving, but incompatible changes are allowed at minor releases (i.e. m .x)
+An Evolving interface is typically exposed so that users or external code can
+make use of a feature before it has stabilized. The expectation that an
+interface should "eventually" stabilize and be promoted to Stable, however,
+is not a requirement for the interface to be labeled as Evolving.
+
+Incompatible changes are allowed for Evolving interface only at minor releases.
+
+Incompatible changes allowed: minor (x.Y.0)
+Compatible changes allowed: maintenance (x.y.Z)
 
 #### Unstable
 
-Incompatible changes to Unstable APIs are allowed at any time. This usually makes
-sense for only private interfaces.
+An Unstable interface is one for which no compatibility guarantees are made. An
+Unstable interface is not necessarily unstable. An unstable interface is
+typically exposed because a user or external code needs to access an interface
+that is not intended for consumption. The interface is exposed as an Unstable
+interface to state clearly that even though the interface is exposed, it is not
+the preferred access path, and no compatibility guarantees are made for it.
 
-However one may call this out for a supposedly public interface to highlight
-that it should not be used as an interface; for public interfaces, labeling it
-as Not-an-interface is probably more appropriate than "Unstable".
+Unless there is a reason to offer a compatibility guarantee on an interface,
+whether it is exposed or not, it should be labeled as Unstable.  Private
+interfaces also should be Unstable in most cases.
 
-Examples of publicly visible interfaces that are unstable
-(i.e. not-an-interface): GUI, CLIs whose output format will change.
+Incompatible changes to Unstable interfaces are allowed at any time.
+
+Incompatible changes allowed: maintenance (x.y.Z)
+Compatible changes allowed: maintenance (x.y.Z)
 
 #### Deprecated
 
-APIs that could potentially be removed in the future and should not be used.
+A Deprecated interface could potentially be removed in the future and should
+not be used.  Even so, a Deprecated interface will continue to function until
+it is removed.  When a Deprecated interface can be removed depends on whether
+it is also Stable, Evolving, or Unstable.
 
 How are the Classifications Recorded?
 -------------------------------------
@@ -121,95 +170,101 @@ How are the Classifications Recorded?
 How will the classification be recorded for Hadoop APIs?
 
 * Each interface or class will have the audience and stability recorded using
-  annotations in org.apache.hadoop.classification package.
+  annotations in the org.apache.hadoop.classification package.
 
-* The javadoc generated by the maven target javadoc:javadoc lists only the public API.
+* The javadoc generated by the maven target javadoc:javadoc lists only the
+  public API.
 
 * One can derive the audience of java classes and java interfaces by the
   audience of the package in which they are contained. Hence it is useful to
   declare the audience of each java package as public or private (along with the
   private audience variations).
 
+How will the classification be recorded for other interfaces, such as CLIs?
+
+* See the [Hadoop Compatibility](Compatibility.html) page for details.
+
 FAQ
 ---
 
 * Why aren’t the java scopes (private, package private and public) good enough?
     * Java’s scoping is not very complete. One is often forced to make a class
-      public in order for other internal components to use it. It does not have
-      friends or sub-package-private like C++.
-
-* But I can easily access a private implementation interface if it is Java public.
-  Where is the protection and control?
-    * The purpose of this is not providing absolute access control. Its purpose
-      is to communicate to users and developers. One can access private
-      implementation functions in libc; however if they change the internal
-      implementation details, your application will break and you will have
-      little sympathy from the folks who are supplying libc. If you use a
-      non-public interface you understand the risks.
-
-* Why bother declaring the stability of a private interface?
-  Aren’t private interfaces always unstable?
-    * Private interfaces are not always unstable. In the cases where they are
-      stable they capture internal properties of the system and can communicate
+      public in order for other internal components to use it. It also does not
+      have friends or sub-package-private like C++.
+
+* But I can easily access a Private interface if it is Java public. Where is the
+  protection and control?
+    * The purpose of this classification scheme is not providing absolute
+      access control. Its purpose is to communicate to users and developers.
+      One can access private implementation functions in libc; however if
+      they change the internal implementation details, the application will
+      break and one will receive little sympathy from the folks who are
+      supplying libc. When using a non-public interface, the risks are
+      understood.
+
+* Why bother declaring the stability of a Private interface? Aren’t Private
+  interfaces always Unstable?
+    * Private interfaces are not always Unstable. In the cases where they are
+      Stable they capture internal properties of the system and can communicate
       these properties to its internal users and to developers of the interface.
-        * e.g. In HDFS, NN-DN protocol is private but stable and can help
-          implement rolling upgrades. It communicates that this interface should
-          not be changed in incompatible ways even though it is private.
-        * e.g. In HDFS, FSImage stability provides more flexible rollback.
-
-* What is the harm in applications using a private interface that is stable? How
-  is it different than a public stable interface?
-    * While a private interface marked as stable is targeted to change only at
+        * e.g. In HDFS, NN-DN protocol is Private but Stable and can help
+          implement rolling upgrades. The stability annotation communicates that
+          this interface should not be changed in incompatible ways even though
+          it is Private.
+        * e.g. In HDFS, FSImage the Stabile designation provides more flexible
+          rollback.
+
+* What is the harm in applications using a Private interface that is Stable?
+  How is it different from a Public Stable interface?
+    * While a Private interface marked as Stable is targeted to change only at
       major releases, it may break at other times if the providers of that
-      interface are willing to change the internal users of that
-      interface. Further, a public stable interface is less likely to break even
+      interface also are willing to change the internal consumers of that
+      interface. Further, a Public Stable interface is less likely to break even
       at major releases (even though it is allowed to break compatibility)
-      because the impact of the change is larger. If you use a private interface
+      because the impact of the change is larger. If you use a Private interface
       (regardless of its stability) you run the risk of incompatibility.
 
-* Why bother with Limited-private? Isn’t it giving special treatment to some projects?
-  That is not fair.
-    * First, most interfaces should be public or private; actually let us state
-      it even stronger: make it private unless you really want to expose it to
-      public for general use.
-    * Limited-private is for interfaces that are not intended for general
+* Why bother with Limited-Private? Isn’t it giving special treatment to some
+  projects? That is not fair.
+    * Most interfaces should be Public or Private. An interface should be
+      Private unless it is explicitly intended for general use.
+    * Limited-Private is for interfaces that are not intended for general
       use. They are exposed to related projects that need special hooks. Such a
-      classification has a cost to both the supplier and consumer of the limited
+      classification has a cost to both the supplier and consumer of the
       interface. Both will have to work together if ever there is a need to
       break the interface in the future; for example the supplier and the
       consumers will have to work together to get coordinated releases of their
-      respective projects. This should not be taken lightly – if you can get
-      away with private then do so; if the interface is really for general use
-      for all applications then do so. But remember that making an interface
-      public has huge responsibility. Sometimes Limited-private is just right.
-    * A good example of a limited-private interface is BlockLocations, This is a
-      fairly low-level interface that we are willing to expose to MR and perhaps
-      HBase. We are likely to change it down the road and at that time we will
-      coordinate release effort with the MR team.
-      While MR and HDFS are always released in sync today, they may
-      change down the road.
-    * If you have a limited-private interface with many projects listed then you
-      are fooling yourself. It is practically public.
-    * It might be worth declaring a special audience classification called
-      Hadoop-Private for the Hadoop family.
-
-* Lets treat all private interfaces as Hadoop-private. What is the harm in
-  projects in the Hadoop family have access to private classes?
-    * Do we want MR accessing class files that are implementation details inside
-      HDFS. There used to be many such layer violations in the code that we have
-      been cleaning up over the last few years. We don’t want such layer
-      violations to creep back in by no separating between the major components
-      like HDFS and MR.
-
-* Aren't all public interfaces stable?
-    * One may mark a public interface as evolving in its early days. Here one is
+      respective projects. This contract should not be taken lightly–use
+      Private if possible; if the interface is really for general use
+      for all applications then use Public. Always remember that making an
+      interface Public comes with large burden of responsibility. Sometimes
+      Limited-Private is just right.
+    * A good example of a Limited-Private interface is BlockLocations. This
+      interface is a fairly low-level interface that is exposed to MapReduce
+      and HBase. The interface is likely to change down the road, and at that
+      time the release effort will have to be coordinated with the
+      MapReduce development team. While MapReduce and HDFS are always released
+      in sync today, that policy may change down the road.
+    * If you have a Limited-Private interface with many projects listed then
+      the interface is probably a good candidate to be made Public.
+
+* Let's treat all Private interfaces as Limited-Private for all of Hadoop. What
+  is the harm if projects in the Hadoop family have access to private classes?
+    * There used to be many cases in the code where one project depended on the
+      internal implementation details of another. A significant effort went
+      into cleaning up those issues. Opening up all interfaces as
+      Limited-Private for all of Hadoop would open the door to reintroducing
+      such coupling issues.
+
+* Aren't all Public interfaces Stable?
+    * One may mark a Public interface as Evolving in its early days. Here one is
       promising to make an effort to make compatible changes but may need to
       break it at minor releases.
-    * One example of a public interface that is unstable is where one is
+    * One example of a Public interface that is Unstable is where one is
       providing an implementation of a standards-body based interface that is
       still under development. For example, many companies, in an attempt to be
       first to market, have provided implementations of a new NFS protocol even
       when the protocol was not fully completed by IETF. The implementor cannot
-      evolve the interface in a fashion that causes least distruption because
+      evolve the interface in a fashion that causes least disruption because
       the stability is controlled by the standards body. Hence it is appropriate
-      to label the interface as unstable.
+      to label the interface as Unstable.


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


[02/50] [abbrv] hadoop git commit: Revert "HDFS-10391. Always enable NameNode service RPC port. Contributed by Gergely Novak."

Posted by ae...@apache.org.
Revert "HDFS-10391. Always enable NameNode service RPC port. Contributed by Gergely Novak."

This reverts commit aa4b6fbe754ab7e3cf8ee106598d550f6e14783e.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/65a94100
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/65a94100
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/65a94100

Branch: refs/heads/HDFS-7240
Commit: 65a941008d4bbf906772399d3f035f2a0da5abfa
Parents: 66ca0a6
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Sep 14 11:17:08 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Sep 14 11:17:08 2017 -0700

----------------------------------------------------------------------
 .../hdfs/client/HdfsClientConfigKeys.java       |   1 -
 .../java/org/apache/hadoop/hdfs/DFSUtil.java    | 106 +++++++--------
 .../hdfs/server/datanode/BlockPoolManager.java  |   2 +-
 .../hadoop/hdfs/server/namenode/BackupNode.java |   2 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   5 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |  29 ++--
 .../hdfs/server/namenode/NameNodeRpcServer.java | 129 ++++++++++--------
 .../hdfs/server/namenode/SecondaryNameNode.java |   2 +-
 .../hdfs/server/namenode/ha/EditLogTailer.java  |   3 +-
 .../server/namenode/ha/RemoteNameNodeInfo.java  |   2 +-
 .../server/namenode/ha/StandbyCheckpointer.java |   2 +-
 .../org/apache/hadoop/hdfs/tools/GetConf.java   |   4 +-
 .../src/main/resources/hdfs-default.xml         |   3 +-
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |  66 ++-------
 .../apache/hadoop/hdfs/MiniDFSNNTopology.java   |  17 +--
 .../org/apache/hadoop/hdfs/TestDFSUtil.java     | 133 +++----------------
 .../apache/hadoop/hdfs/TestHDFSServerPorts.java |  85 ++++++------
 .../org/apache/hadoop/hdfs/TestSafeMode.java    |   2 +-
 .../hadoop/hdfs/qjournal/MiniQJMHACluster.java  |   9 +-
 .../balancer/TestBalancerWithHANameNodes.java   |   5 +-
 .../datanode/InternalDataNodeTestUtils.java     |   9 +-
 .../hdfs/server/datanode/TestBlockRecovery.java |   6 +-
 .../datanode/TestDataNodeMetricsLogger.java     |  10 +-
 .../TestDataNodeMultipleRegistrations.java      |   8 +-
 .../datanode/TestDataNodeReconfiguration.java   |  11 +-
 .../TestDatanodeProtocolRetryPolicy.java        |   6 +-
 .../server/datanode/TestRefreshNamenodes.java   |  17 +--
 .../hdfs/server/namenode/TestBackupNode.java    |   2 -
 .../hdfs/server/namenode/TestCheckpoint.java    |   6 +-
 .../server/namenode/TestNameNodeMXBean.java     |  11 +-
 .../namenode/TestNameNodeMetricsLogger.java     |   1 -
 .../TestValidateConfigurationSettings.java      |   2 -
 .../hdfs/server/namenode/ha/HATestUtil.java     |   1 +
 .../server/namenode/ha/TestEditLogTailer.java   |  20 +--
 .../hadoop/hdfs/tools/TestDFSHAAdmin.java       |   7 +-
 .../hdfs/tools/TestDFSHAAdminMiniCluster.java   |   2 +-
 .../hdfs/tools/TestDFSZKFailoverController.java |   6 +-
 .../apache/hadoop/hdfs/tools/TestGetConf.java   |  40 ++++--
 38 files changed, 297 insertions(+), 475 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index d6efb5c..e99b099 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -74,7 +74,6 @@ public interface HdfsClientConfigKeys {
   String  DFS_NAMENODE_HTTPS_ADDRESS_KEY = "dfs.namenode.https-address";
   String DFS_HA_NAMENODES_KEY_PREFIX = "dfs.ha.namenodes";
   int DFS_NAMENODE_RPC_PORT_DEFAULT = 9820;
-  int DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT = 9840;
   String DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY =
       "dfs.namenode.kerberos.principal";
   String  DFS_CLIENT_WRITE_PACKET_SIZE_KEY = "dfs.client-write-packet-size";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 3c71e76..7776dc2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -35,7 +35,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICE_ID;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYPASSWORD_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_PASSWORD_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY;
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
@@ -493,25 +492,61 @@ public class DFSUtil {
   }
 
   /**
+   * Returns list of InetSocketAddresses corresponding to namenodes from the
+   * configuration.
+   * 
+   * Returns namenode address specifically configured for datanodes (using
+   * service ports), if found. If not, regular RPC address configured for other
+   * clients is returned.
+   * 
+   * @param conf configuration
+   * @return list of InetSocketAddress
+   * @throws IOException on error
+   */
+  public static Map<String, Map<String, InetSocketAddress>> getNNServiceRpcAddresses(
+      Configuration conf) throws IOException {
+    // Use default address as fall back
+    String defaultAddress;
+    try {
+      defaultAddress = NetUtils.getHostPortString(
+          DFSUtilClient.getNNAddress(conf));
+    } catch (IllegalArgumentException e) {
+      defaultAddress = null;
+    }
+    
+    Map<String, Map<String, InetSocketAddress>> addressList =
+      DFSUtilClient.getAddresses(conf, defaultAddress,
+                                 DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
+                                 DFS_NAMENODE_RPC_ADDRESS_KEY);
+    if (addressList.isEmpty()) {
+      throw new IOException("Incorrect configuration: namenode address "
+          + DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY + " or "  
+          + DFS_NAMENODE_RPC_ADDRESS_KEY
+          + " is not configured.");
+    }
+    return addressList;
+  }
+
+  /**
    * Returns list of InetSocketAddresses corresponding to the namenode
    * that manages this cluster. Note this is to be used by datanodes to get
    * the list of namenode addresses to talk to.
    *
-   * Returns namenode address specifically configured for datanodes
+   * Returns namenode address specifically configured for datanodes (using
+   * service ports), if found. If not, regular RPC address configured for other
+   * clients is returned.
    *
    * @param conf configuration
    * @return list of InetSocketAddress
    * @throws IOException on error
    */
   public static Map<String, Map<String, InetSocketAddress>>
-      getNNServiceRpcAddresses(Configuration conf) throws IOException {
+    getNNServiceRpcAddressesForCluster(Configuration conf) throws IOException {
     // Use default address as fall back
     String defaultAddress;
     try {
-      InetSocketAddress rpcAddress = DFSUtilClient.getNNAddress(conf);
-      InetSocketAddress serviceAddress = InetSocketAddress.createUnresolved(
-          rpcAddress.getHostName(), DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
-      defaultAddress = NetUtils.getHostPortString(serviceAddress);
+      defaultAddress = NetUtils.getHostPortString(
+          DFSUtilClient.getNNAddress(conf));
     } catch (IllegalArgumentException e) {
       defaultAddress = null;
     }
@@ -534,46 +569,16 @@ public class DFSUtil {
       }
     }
 
-    // If true, then replace the port numbers in the final address list
-    // with the default service RPC port.
-    boolean replacePortNumbers = false;
-
-    // First try to lookup using the service RPC address keys.
     Map<String, Map<String, InetSocketAddress>> addressList =
-            DFSUtilClient.getAddressesForNsIds(
-                conf, parentNameServices, null,
-                DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY);
-
-    // Next try to lookup using the RPC address key.
-    if (addressList.isEmpty()) {
-      replacePortNumbers = true;
-      addressList = DFSUtilClient.getAddressesForNsIds(
-          conf, parentNameServices, null, DFS_NAMENODE_RPC_ADDRESS_KEY);
-    }
-
-    // Finally, fallback to the default address.
-    // This will not yield the correct address in a federated/HA setup.
-    if (addressList.isEmpty()) {
-      addressList = DFSUtilClient.getAddressesForNsIds(
-          conf, parentNameServices, defaultAddress);
-    }
-
+            DFSUtilClient.getAddressesForNsIds(conf, parentNameServices,
+                                               defaultAddress,
+                                               DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
+                                               DFS_NAMENODE_RPC_ADDRESS_KEY);
     if (addressList.isEmpty()) {
       throw new IOException("Incorrect configuration: namenode address "
-          + DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY + " or "
-          + DFS_NAMENODE_RPC_ADDRESS_KEY
-          + " is not configured.");
-    }
-
-    if (replacePortNumbers) {
-      // Replace the RPC port(s) with the default service RPC port(s)
-      addressList.forEach((nsId, addresses) -> {
-        addresses.forEach((nnId, address) -> {
-          InetSocketAddress serviceAddress = InetSocketAddress.createUnresolved(
-              address.getHostName(), DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
-          addresses.put(nnId, serviceAddress);
-        });
-      });
+              + DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY + " or "
+              + DFS_NAMENODE_RPC_ADDRESS_KEY
+              + " is not configured.");
     }
     return addressList;
   }
@@ -1225,17 +1230,12 @@ public class DFSUtil {
     String serviceAddrKey = DFSUtilClient.concatSuffixes(
         DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, nsId, nnId);
 
+    String addrKey = DFSUtilClient.concatSuffixes(
+        DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nsId, nnId);
+
     String serviceRpcAddr = conf.get(serviceAddrKey);
     if (serviceRpcAddr == null) {
-      String addrKey = DFSUtilClient.concatSuffixes(
-          DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nsId, nnId);
-      String rpcAddress = conf.get(addrKey);
-      if (rpcAddress != null) {
-        InetSocketAddress rpcAddr = NetUtils.createSocketAddr(rpcAddress);
-        InetSocketAddress serviceAddr = InetSocketAddress.createUnresolved(
-            rpcAddr.getHostName(), DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
-        serviceRpcAddr = NetUtils.getHostPortString(serviceAddr);
-      }
+      serviceRpcAddr = conf.get(addrKey);
     }
     return serviceRpcAddr;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java
index 677559c..f6a11c2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java
@@ -150,7 +150,7 @@ class BlockPoolManager {
             (DFSConfigKeys.DFS_NAMESERVICES));
 
     Map<String, Map<String, InetSocketAddress>> newAddressMap = DFSUtil
-            .getNNServiceRpcAddresses(conf);
+            .getNNServiceRpcAddressesForCluster(conf);
     Map<String, Map<String, InetSocketAddress>> newLifelineAddressMap = DFSUtil
             .getNNLifelineRpcAddressesForCluster(conf);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
index 5c2dcdc..318d8e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
@@ -318,7 +318,7 @@ public class BackupNode extends NameNode {
 
   private NamespaceInfo handshake(Configuration conf) throws IOException {
     // connect to name node
-    InetSocketAddress nnAddress = NameNode.getServiceAddress(conf);
+    InetSocketAddress nnAddress = NameNode.getServiceAddress(conf, true);
     this.namenode = NameNodeProxies.createNonHAProxy(conf, nnAddress,
         NamenodeProtocol.class, UserGroupInformation.getCurrentUser(),
         true).getProxy();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 55695b3..d9f3c0e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -1157,8 +1157,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       dir.setINodeAttributeProvider(inodeAttributeProvider);
     }
     snapshotManager.registerMXBean();
-    InetSocketAddress serviceAddress = NameNode.getServiceAddress(conf);
-    this.nameNodeHostName = serviceAddress.getHostName();
+    InetSocketAddress serviceAddress = NameNode.getServiceAddress(conf, true);
+    this.nameNodeHostName = (serviceAddress != null) ?
+        serviceAddress.getHostName() : "";
   }
   
   /** 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index d700439..79bbbc5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -505,17 +505,18 @@ public class NameNode extends ReconfigurableBase implements
   
   /**
    * Fetches the address for services to use when connecting to namenode
+   * based on the value of fallback returns null if the special
+   * address is not specified or returns the default namenode address
+   * to be used by both clients and services.
    * Services here are datanodes, backup node, any non client connection
    */
-  public static InetSocketAddress getServiceAddress(Configuration conf) {
-    String address = conf.getTrimmed(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY);
-    if (address == null || address.isEmpty()) {
-      InetSocketAddress rpcAddress = DFSUtilClient.getNNAddress(conf);
-      return NetUtils.createSocketAddr(rpcAddress.getHostName(),
-          HdfsClientConfigKeys.DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
+  public static InetSocketAddress getServiceAddress(Configuration conf,
+                                                        boolean fallback) {
+    String addr = conf.getTrimmed(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY);
+    if (addr == null || addr.isEmpty()) {
+      return fallback ? DFSUtilClient.getNNAddress(conf) : null;
     }
-    return NetUtils.createSocketAddr(address,
-        HdfsClientConfigKeys.DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT);
+    return DFSUtilClient.getNNAddress(addr);
   }
 
   //
@@ -553,7 +554,7 @@ public class NameNode extends ReconfigurableBase implements
    * If the service rpc is not configured returns null
    */
   protected InetSocketAddress getServiceRpcServerAddress(Configuration conf) {
-    return NameNode.getServiceAddress(conf);
+    return NameNode.getServiceAddress(conf, false);
   }
 
   protected InetSocketAddress getRpcServerAddress(Configuration conf) {
@@ -614,8 +615,7 @@ public class NameNode extends ReconfigurableBase implements
   }
 
   /**
-   * Modifies the configuration passed to contain the service rpc address
-   * setting.
+   * Modifies the configuration passed to contain the service rpc address setting
    */
   protected void setRpcServiceServerAddress(Configuration conf,
       InetSocketAddress serviceRPCAddress) {
@@ -1071,13 +1071,6 @@ public class NameNode extends ReconfigurableBase implements
   }
 
   /**
-   * @return NameNode service RPC address in "host:port" string form
-   */
-  public String getServiceRpcAddressHostPortString() {
-    return NetUtils.getHostPortString(getServiceRpcAddress());
-  }
-
-  /**
    * @return NameNode HTTP address, used by the Web UI, image transfer,
    *    and HTTP-based file system clients like WebHDFS
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 78790bd..1ef3f55 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -333,63 +333,66 @@ public class NameNodeRpcServer implements NamenodeProtocols {
         .newReflectiveBlockingService(traceAdminXlator);
 
     InetSocketAddress serviceRpcAddr = nn.getServiceRpcServerAddress(conf);
-    String bindHost = nn.getServiceRpcServerBindHost(conf);
-    if (bindHost == null) {
-      bindHost = serviceRpcAddr.getHostName();
-    }
+    if (serviceRpcAddr != null) {
+      String bindHost = nn.getServiceRpcServerBindHost(conf);
+      if (bindHost == null) {
+        bindHost = serviceRpcAddr.getHostName();
+      }
+      LOG.info("Service RPC server is binding to " + bindHost + ":" +
+          serviceRpcAddr.getPort());
+
+      int serviceHandlerCount =
+        conf.getInt(DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY,
+                    DFS_NAMENODE_SERVICE_HANDLER_COUNT_DEFAULT);
+      serviceRpcServer = new RPC.Builder(conf)
+          .setProtocol(
+              org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB.class)
+          .setInstance(clientNNPbService)
+          .setBindAddress(bindHost)
+          .setPort(serviceRpcAddr.getPort())
+          .setNumHandlers(serviceHandlerCount)
+          .setVerbose(false)
+          .setSecretManager(namesystem.getDelegationTokenSecretManager())
+          .build();
 
-    LOG.info("Service RPC server is binding to " + bindHost + ":" +
-        serviceRpcAddr.getPort());
+      // Add all the RPC protocols that the namenode implements
+      DFSUtil.addPBProtocol(conf, HAServiceProtocolPB.class, haPbService,
+          serviceRpcServer);
+      DFSUtil.addPBProtocol(conf, ReconfigurationProtocolPB.class,
+          reconfigurationPbService, serviceRpcServer);
+      DFSUtil.addPBProtocol(conf, NamenodeProtocolPB.class, NNPbService,
+          serviceRpcServer);
+      DFSUtil.addPBProtocol(conf, DatanodeProtocolPB.class, dnProtoPbService,
+          serviceRpcServer);
+      DFSUtil.addPBProtocol(conf, RefreshAuthorizationPolicyProtocolPB.class,
+          refreshAuthService, serviceRpcServer);
+      DFSUtil.addPBProtocol(conf, RefreshUserMappingsProtocolPB.class, 
+          refreshUserMappingService, serviceRpcServer);
+      // We support Refreshing call queue here in case the client RPC queue is full
+      DFSUtil.addPBProtocol(conf, RefreshCallQueueProtocolPB.class,
+          refreshCallQueueService, serviceRpcServer);
+      DFSUtil.addPBProtocol(conf, GenericRefreshProtocolPB.class,
+          genericRefreshService, serviceRpcServer);
+      DFSUtil.addPBProtocol(conf, GetUserMappingsProtocolPB.class, 
+          getUserMappingService, serviceRpcServer);
+      DFSUtil.addPBProtocol(conf, TraceAdminProtocolPB.class,
+          traceAdminService, serviceRpcServer);
 
-    int serviceHandlerCount = conf.getInt(
-        DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY,
-        DFS_NAMENODE_SERVICE_HANDLER_COUNT_DEFAULT);
-    serviceRpcServer = new RPC.Builder(conf)
-        .setProtocol(
-            org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB.class)
-        .setInstance(clientNNPbService)
-        .setBindAddress(bindHost)
-        .setPort(serviceRpcAddr.getPort())
-        .setNumHandlers(serviceHandlerCount)
-        .setVerbose(false)
-        .setSecretManager(namesystem.getDelegationTokenSecretManager())
-        .build();
-
-    // Add all the RPC protocols that the namenode implements
-    DFSUtil.addPBProtocol(conf, HAServiceProtocolPB.class, haPbService,
-        serviceRpcServer);
-    DFSUtil.addPBProtocol(conf, ReconfigurationProtocolPB.class,
-        reconfigurationPbService, serviceRpcServer);
-    DFSUtil.addPBProtocol(conf, NamenodeProtocolPB.class, NNPbService,
-        serviceRpcServer);
-    DFSUtil.addPBProtocol(conf, DatanodeProtocolPB.class, dnProtoPbService,
-        serviceRpcServer);
-    DFSUtil.addPBProtocol(conf, RefreshAuthorizationPolicyProtocolPB.class,
-        refreshAuthService, serviceRpcServer);
-    DFSUtil.addPBProtocol(conf, RefreshUserMappingsProtocolPB.class,
-        refreshUserMappingService, serviceRpcServer);
-    // We support Refreshing call queue here in case the client RPC queue
-    // is full.
-    DFSUtil.addPBProtocol(conf, RefreshCallQueueProtocolPB.class,
-        refreshCallQueueService, serviceRpcServer);
-    DFSUtil.addPBProtocol(conf, GenericRefreshProtocolPB.class,
-        genericRefreshService, serviceRpcServer);
-    DFSUtil.addPBProtocol(conf, GetUserMappingsProtocolPB.class,
-        getUserMappingService, serviceRpcServer);
-    DFSUtil.addPBProtocol(conf, TraceAdminProtocolPB.class,
-        traceAdminService, serviceRpcServer);
-
-    // Update the address with the correct port.
-    InetSocketAddress listenAddr = serviceRpcServer.getListenerAddress();
-    serviceRPCAddress = new InetSocketAddress(
-          serviceRpcAddr.getHostName(), listenAddr.getPort());
-    nn.setRpcServiceServerAddress(conf, serviceRPCAddress);
+      // Update the address with the correct port
+      InetSocketAddress listenAddr = serviceRpcServer.getListenerAddress();
+      serviceRPCAddress = new InetSocketAddress(
+            serviceRpcAddr.getHostName(), listenAddr.getPort());
+      nn.setRpcServiceServerAddress(conf, serviceRPCAddress);
+    } else {
+      serviceRpcServer = null;
+      serviceRPCAddress = null;
+    }
 
     InetSocketAddress lifelineRpcAddr = nn.getLifelineRpcServerAddress(conf);
     if (lifelineRpcAddr != null) {
       RPC.setProtocolEngine(conf, HAServiceProtocolPB.class,
           ProtobufRpcEngine.class);
-      bindHost = nn.getLifelineRpcServerBindHost(conf);
+      String bindHost = nn.getLifelineRpcServerBindHost(conf);
       if (bindHost == null) {
         bindHost = lifelineRpcAddr.getHostName();
       }
@@ -419,7 +422,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
           lifelineProtoPbService, lifelineRpcServer);
 
       // Update the address with the correct port
-      listenAddr = lifelineRpcServer.getListenerAddress();
+      InetSocketAddress listenAddr = lifelineRpcServer.getListenerAddress();
       lifelineRPCAddress = new InetSocketAddress(lifelineRpcAddr.getHostName(),
           listenAddr.getPort());
       nn.setRpcLifelineServerAddress(conf, lifelineRPCAddress);
@@ -429,7 +432,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     }
 
     InetSocketAddress rpcAddr = nn.getRpcServerAddress(conf);
-    bindHost = nn.getRpcServerBindHost(conf);
+    String bindHost = nn.getRpcServerBindHost(conf);
     if (bindHost == null) {
       bindHost = rpcAddr.getHostName();
     }
@@ -473,14 +476,16 @@ public class NameNodeRpcServer implements NamenodeProtocols {
           conf.getBoolean(
             CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {
       clientRpcServer.refreshServiceAcl(conf, new HDFSPolicyProvider());
-      serviceRpcServer.refreshServiceAcl(conf, new HDFSPolicyProvider());
+      if (serviceRpcServer != null) {
+        serviceRpcServer.refreshServiceAcl(conf, new HDFSPolicyProvider());
+      }
       if (lifelineRpcServer != null) {
         lifelineRpcServer.refreshServiceAcl(conf, new HDFSPolicyProvider());
       }
     }
 
     // The rpc-server port can be ephemeral... ensure we have the correct info
-    listenAddr = clientRpcServer.getListenerAddress();
+    InetSocketAddress listenAddr = clientRpcServer.getListenerAddress();
     clientRpcAddress = new InetSocketAddress(
         rpcAddr.getHostName(), listenAddr.getPort());
     nn.setRpcServerAddress(conf, clientRpcAddress);
@@ -518,7 +523,9 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     clientRpcServer.addSuppressedLoggingExceptions(StandbyException.class);
 
     clientRpcServer.setTracer(nn.tracer);
-    serviceRpcServer.setTracer(nn.tracer);
+    if (serviceRpcServer != null) {
+      serviceRpcServer.setTracer(nn.tracer);
+    }
     if (lifelineRpcServer != null) {
       lifelineRpcServer.setTracer(nn.tracer);
     }
@@ -547,7 +554,9 @@ public class NameNodeRpcServer implements NamenodeProtocols {
    */
   void start() {
     clientRpcServer.start();
-    serviceRpcServer.start();
+    if (serviceRpcServer != null) {
+      serviceRpcServer.start();      
+    }
     if (lifelineRpcServer != null) {
       lifelineRpcServer.start();
     }
@@ -558,7 +567,9 @@ public class NameNodeRpcServer implements NamenodeProtocols {
    */
   void join() throws InterruptedException {
     clientRpcServer.join();
-    serviceRpcServer.join();
+    if (serviceRpcServer != null) {
+      serviceRpcServer.join();      
+    }
     if (lifelineRpcServer != null) {
       lifelineRpcServer.join();
     }
@@ -571,7 +582,9 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     if (clientRpcServer != null) {
       clientRpcServer.stop();
     }
-    serviceRpcServer.stop();
+    if (serviceRpcServer != null) {
+      serviceRpcServer.stop();
+    }
     if (lifelineRpcServer != null) {
       lifelineRpcServer.stop();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
index e8dfb72..ff83e34 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
@@ -228,7 +228,7 @@ public class SecondaryNameNode implements Runnable,
 
     // Create connection to the namenode.
     shouldRun = true;
-    nameNodeAddr = NameNode.getServiceAddress(conf);
+    nameNodeAddr = NameNode.getServiceAddress(conf, true);
 
     this.conf = conf;
     this.namenode = NameNodeProxies.createNonHAProxy(conf, nameNodeAddr, 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
index fd5a70e..f57cb4b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
@@ -159,8 +159,7 @@ public class EditLogTailer {
 
       for (RemoteNameNodeInfo info : nns) {
         // overwrite the socket address, if we need to
-        InetSocketAddress ipc = NameNode.getServiceAddress(
-            info.getConfiguration());
+        InetSocketAddress ipc = NameNode.getServiceAddress(info.getConfiguration(), true);
         // sanity check the ipc address
         Preconditions.checkArgument(ipc.getPort() > 0,
             "Active NameNode must have an IPC port configured. " + "Got address '%s'", ipc);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java
index 248be55..9a51190 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java
@@ -54,7 +54,7 @@ public class RemoteNameNodeInfo {
     for (Configuration otherNode : otherNodes) {
       String otherNNId = HAUtil.getNameNodeId(otherNode, nsId);
       // don't do any validation here as in some cases, it can be overwritten later
-      InetSocketAddress otherIpcAddr = NameNode.getServiceAddress(otherNode);
+      InetSocketAddress otherIpcAddr = NameNode.getServiceAddress(otherNode, true);
 
 
       final String scheme = DFSUtil.getHttpClientScheme(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
index 3cbcd9c..789ed9c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
@@ -121,7 +121,7 @@ public class StandbyCheckpointer {
   
   private URL getHttpAddress(Configuration conf) throws IOException {
     final String scheme = DFSUtil.getHttpClientScheme(conf);
-    String defaultHost = NameNode.getServiceAddress(conf).getHostName();
+    String defaultHost = NameNode.getServiceAddress(conf, true).getHostName();
     URI addr = DFSUtil.getInfoServerWithDefaultHost(defaultHost, conf, scheme);
     return addr.toURL();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
index e780393..e6cf16c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
@@ -187,7 +187,7 @@ public class GetConf extends Configured implements Tool {
   static class NameNodesCommandHandler extends CommandHandler {
     @Override
     int doWorkInternal(GetConf tool, String []args) throws IOException {
-      tool.printMap(DFSUtil.getNNServiceRpcAddresses(tool.getConf()));
+      tool.printMap(DFSUtil.getNNServiceRpcAddressesForCluster(tool.getConf()));
       return 0;
     }
   }
@@ -224,7 +224,7 @@ public class GetConf extends Configured implements Tool {
     public int doWorkInternal(GetConf tool, String []args) throws IOException {
       Configuration config = tool.getConf();
       List<ConfiguredNNAddress> cnnlist = DFSUtil.flattenAddressMap(
-          DFSUtil.getNNServiceRpcAddresses(config));
+          DFSUtil.getNNServiceRpcAddressesForCluster(config));
       if (!cnnlist.isEmpty()) {
         for (ConfiguredNNAddress cnn : cnnlist) {
           InetSocketAddress rpc = cnn.getAddress();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 88c102a..af40a34 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -61,7 +61,8 @@
     connecting to this address if it is configured. In the case of HA/Federation where multiple namenodes exist,
     the name service id is added to the name e.g. dfs.namenode.servicerpc-address.ns1
     dfs.namenode.rpc-address.EXAMPLENAMESERVICE
-    The value of this property will take the form of nn-host1:rpc-port. The NameNode's default service RPC port is 9840.
+    The value of this property will take the form of nn-host1:rpc-port.
+    If the value of this property is unset the value of dfs.namenode.rpc-address will be used as the default.
   </description>
 </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index aa3ed30..0345cf5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@ -166,7 +166,6 @@ public class MiniDFSCluster implements AutoCloseable {
    */
   public static class Builder {
     private int nameNodePort = 0;
-    private int nameNodeServicePort = 0;
     private int nameNodeHttpPort = 0;
     private final Configuration conf;
     private int numDataNodes = 1;
@@ -209,14 +208,6 @@ public class MiniDFSCluster implements AutoCloseable {
       this.nameNodePort = val;
       return this;
     }
-
-    /**
-     * Default: 0
-     */
-    public Builder nameNodeServicePort(int val) {
-      this.nameNodeServicePort = val;
-      return this;
-    }
     
     /**
      * Default: 0
@@ -408,8 +399,8 @@ public class MiniDFSCluster implements AutoCloseable {
     }
 
     /**
-     * Default: false.
-     * When true the hosts file/include file for the cluster is setup.
+     * Default: false
+     * When true the hosts file/include file for the cluster is setup
      */
     public Builder setupHostsFile(boolean val) {
       this.setupHostsFile = val;
@@ -419,7 +410,7 @@ public class MiniDFSCluster implements AutoCloseable {
     /**
      * Default: a single namenode.
      * See {@link MiniDFSNNTopology#simpleFederatedTopology(int)} to set up
-     * federated nameservices.
+     * federated nameservices
      */
     public Builder nnTopology(MiniDFSNNTopology topology) {
       this.nnTopology = topology;
@@ -470,8 +461,7 @@ public class MiniDFSCluster implements AutoCloseable {
     if (builder.nnTopology == null) {
       // If no topology is specified, build a single NN. 
       builder.nnTopology = MiniDFSNNTopology.simpleSingleNN(
-          builder.nameNodePort, builder.nameNodeServicePort,
-          builder.nameNodeHttpPort);
+          builder.nameNodePort, builder.nameNodeHttpPort);
     }
     assert builder.storageTypes == null ||
            builder.storageTypes.length == builder.numDataNodes;
@@ -780,7 +770,7 @@ public class MiniDFSCluster implements AutoCloseable {
                        manageNameDfsDirs, true, manageDataDfsDirs, manageDataDfsDirs,
                        operation, null, racks, hosts,
                        null, simulatedCapacities, null, true, false,
-                       MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0, 0),
+                       MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0),
                        true, false, false, null, true, false);
   }
 
@@ -1259,11 +1249,6 @@ public class MiniDFSCluster implements AutoCloseable {
         DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId,
         nnConf.getNnId());
     conf.set(key, "127.0.0.1:" + nnConf.getIpcPort());
-
-    key = DFSUtil.addKeySuffixes(
-        DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, nameserviceId,
-        nnConf.getNnId());
-    conf.set(key, "127.0.0.1:" + nnConf.getServicePort());
   }
   
   private static String[] createArgs(StartupOption operation) {
@@ -1297,8 +1282,6 @@ public class MiniDFSCluster implements AutoCloseable {
     // the conf
     hdfsConf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
         nameserviceId, nnId), nn.getNameNodeAddressHostPortString());
-    hdfsConf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
-        nameserviceId, nnId), nn.getServiceRpcAddressHostPortString());
     if (nn.getHttpAddress() != null) {
       hdfsConf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_HTTP_ADDRESS_KEY,
           nameserviceId, nnId), NetUtils.getHostPortString(nn.getHttpAddress()));
@@ -1354,14 +1337,6 @@ public class MiniDFSCluster implements AutoCloseable {
     return getNN(nnIndex).conf;
   }
 
-  /**
-   * Return the cluster-wide configuration.
-   * @return
-   */
-  public Configuration getClusterConfiguration() {
-    return conf;
-  }
-
   private NameNodeInfo getNN(int nnIndex) {
     int count = 0;
     for (NameNodeInfo nn : namenodes.values()) {
@@ -1955,16 +1930,6 @@ public class MiniDFSCluster implements AutoCloseable {
   }
 
   /**
-   * Gets the service rpc port used by the NameNode, because the caller
-   * supplied port is not necessarily the actual port used.
-   * Assumption: cluster has a single namenode
-   */
-  public int getNameNodeServicePort() {
-    checkSingleNameNode();
-    return getNameNodeServicePort(0);
-  }
-
-  /**
    * @return the service rpc port used by the NameNode at the given index.
    */     
   public int getNameNodeServicePort(int nnIndex) {
@@ -2591,14 +2556,12 @@ public class MiniDFSCluster implements AutoCloseable {
     }
 
     NameNodeInfo info = getNN(nnIndex);
-    InetSocketAddress nameNodeAddress = info.nameNode.getNameNodeAddress();
-    assert nameNodeAddress.getPort() != 0;
-    DFSClient client = new DFSClient(nameNodeAddress, conf);
+    InetSocketAddress addr = info.nameNode.getServiceRpcAddress();
+    assert addr.getPort() != 0;
+    DFSClient client = new DFSClient(addr, conf);
 
     // ensure all datanodes have registered and sent heartbeat to the namenode
-    InetSocketAddress serviceAddress = info.nameNode.getServiceRpcAddress();
-    while (shouldWait(client.datanodeReport(DatanodeReportType.LIVE),
-        serviceAddress)) {
+    while (shouldWait(client.datanodeReport(DatanodeReportType.LIVE), addr)) {
       try {
         LOG.info("Waiting for cluster to become active");
         Thread.sleep(100);
@@ -3093,18 +3056,13 @@ public class MiniDFSCluster implements AutoCloseable {
     }
   }
 
-  public void addNameNode(Configuration conf, int namenodePort)
-      throws IOException{
-    addNameNode(conf, namenodePort, 0);
-  }
-
   /**
    * Add a namenode to a federated cluster and start it. Configuration of
    * datanodes in the cluster is refreshed to register with the new namenode.
    * 
    * @return newly started namenode
    */
-  public void addNameNode(Configuration conf, int namenodePort, int servicePort)
+  public void addNameNode(Configuration conf, int namenodePort)
       throws IOException {
     if(!federation)
       throw new IOException("cannot add namenode to non-federated cluster");
@@ -3118,9 +3076,7 @@ public class MiniDFSCluster implements AutoCloseable {
   
     String nnId = null;
     initNameNodeAddress(conf, nameserviceId,
-        new NNConf(nnId)
-            .setIpcPort(namenodePort)
-            .setServicePort(servicePort));
+        new NNConf(nnId).setIpcPort(namenodePort));
     // figure out the current number of NNs
     NameNodeInfo[] infos = this.getNameNodeInfos(nameserviceId);
     int nnIndex = infos == null ? 0 : infos.length;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java
index b1d609a..b9786a3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java
@@ -43,13 +43,12 @@ public class MiniDFSNNTopology {
    * Set up a simple non-federated non-HA NN.
    */
   public static MiniDFSNNTopology simpleSingleNN(
-      int rpcPort, int servicePort, int httpPort) {
+      int nameNodePort, int nameNodeHttpPort) {
     return new MiniDFSNNTopology()
       .addNameservice(new MiniDFSNNTopology.NSConf(null)
         .addNN(new MiniDFSNNTopology.NNConf(null)
-          .setIpcPort(rpcPort)
-          .setServicePort(servicePort)
-          .setHttpPort(httpPort)));
+          .setHttpPort(nameNodeHttpPort)
+          .setIpcPort(nameNodePort)));
   }
   
 
@@ -222,7 +221,6 @@ public class MiniDFSNNTopology {
     private final String nnId;
     private int httpPort;
     private int ipcPort;
-    private int servicePort;
     private String clusterId;
     
     public NNConf(String nnId) {
@@ -236,10 +234,6 @@ public class MiniDFSNNTopology {
     int getIpcPort() {
       return ipcPort;
     }
-
-    int getServicePort() {
-      return servicePort;
-    }
     
     int getHttpPort() {
       return httpPort;
@@ -259,11 +253,6 @@ public class MiniDFSNNTopology {
       return this;
     }
 
-    public NNConf setServicePort(int servicePort) {
-      this.servicePort = servicePort;
-      return this;
-    }
-
     public NNConf setClusterId(String clusterId) {
       this.clusterId = clusterId;
       return this;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
index 4ae2a77..f811d3d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
@@ -33,7 +33,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICE_ID;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYPASSWORD_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_PASSWORD_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY;
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT;
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows;
 import static org.hamcrest.CoreMatchers.not;
@@ -84,9 +83,9 @@ import com.google.common.collect.Sets;
 
 public class TestDFSUtil {
 
-  private static final String NS1_NN_ADDR    = "ns1-nn.example.com:9820";
-  private static final String NS1_NN1_ADDR   = "ns1-nn1.example.com:9820";
-  private static final String NS1_NN2_ADDR   = "ns1-nn2.example.com:9820";
+  static final String NS1_NN_ADDR    = "ns1-nn.example.com:9820";
+  static final String NS1_NN1_ADDR   = "ns1-nn1.example.com:9820";
+  static final String NS1_NN2_ADDR   = "ns1-nn2.example.com:9820";
 
   /**
    * Reset to default UGI settings since some tests change them.
@@ -274,13 +273,13 @@ public class TestDFSUtil {
     assertEquals(1, nn1Map.size());
     InetSocketAddress addr = nn1Map.get(null);
     assertEquals("localhost", addr.getHostName());
-    assertEquals(DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT, addr.getPort());
+    assertEquals(9000, addr.getPort());
     
     Map<String, InetSocketAddress> nn2Map = nnMap.get("nn2");
     assertEquals(1, nn2Map.size());
     addr = nn2Map.get(null);
     assertEquals("localhost", addr.getHostName());
-    assertEquals(DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT, addr.getPort());
+    assertEquals(9001, addr.getPort());
 
     // Test - can look up nameservice ID from service address
     checkNameServiceId(conf, NN1_ADDRESS, "nn1");
@@ -315,8 +314,7 @@ public class TestDFSUtil {
     Map<String, InetSocketAddress> defaultNsMap = addrMap.get(null);
     assertEquals(1, defaultNsMap.size());
     
-    assertEquals(DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT,
-        defaultNsMap.get(null).getPort());
+    assertEquals(9999, defaultNsMap.get(null).getPort());
   }
   
   /**
@@ -493,10 +491,6 @@ public class TestDFSUtil {
     final String NS1_NN2_HOST = "ns1-nn2.example.com:9820";
     final String NS2_NN1_HOST = "ns2-nn1.example.com:9820";
     final String NS2_NN2_HOST = "ns2-nn2.example.com:9820";
-    final String NS1_NN1_SERVICE_HOST = "ns1-nn1.example.com:9840";
-    final String NS1_NN2_SERVICE_HOST = "ns1-nn2.example.com:9840";
-    final String NS2_NN1_SERVICE_HOST = "ns2-nn1.example.com:9840";
-    final String NS2_NN2_SERVICE_HOST = "ns2-nn2.example.com:9840";
     conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://ns1");
     
     // Two nameservices, each with two NNs.
@@ -530,14 +524,12 @@ public class TestDFSUtil {
     assertEquals(NS2_NN1_HOST, map.get("ns2").get("ns2-nn1").toString());
     assertEquals(NS2_NN2_HOST, map.get("ns2").get("ns2-nn2").toString());
     
-    assertEquals(NS1_NN1_SERVICE_HOST,
+    assertEquals(NS1_NN1_HOST, 
         DFSUtil.getNamenodeServiceAddr(conf, "ns1", "ns1-nn1"));
-    assertEquals(NS1_NN2_SERVICE_HOST,
+    assertEquals(NS1_NN2_HOST, 
         DFSUtil.getNamenodeServiceAddr(conf, "ns1", "ns1-nn2"));
-    assertEquals(NS2_NN1_SERVICE_HOST,
+    assertEquals(NS2_NN1_HOST, 
         DFSUtil.getNamenodeServiceAddr(conf, "ns2", "ns2-nn1"));
-    assertEquals(NS2_NN2_SERVICE_HOST,
-        DFSUtil.getNamenodeServiceAddr(conf, "ns2", "ns2-nn2"));
 
     // No nameservice was given and we can't determine which service addr
     // to use as two nameservices could share a namenode ID.
@@ -563,11 +555,9 @@ public class TestDFSUtil {
     
     // One nameservice with two NNs
     final String NS1_NN1_HOST = "ns1-nn1.example.com:9820";
-    final String NS1_NN1_HOST_SVC = "ns1-nn1.example.com:9821";
-    final String NS1_NN1_HOST_DEFAULT_SVC = "ns1-nn1.example.com:9840";
-    final String NS1_NN2_HOST = "ns1-nn2.example.com:9820";
+    final String NS1_NN1_HOST_SVC = "ns1-nn2.example.com:9821";
+    final String NS1_NN2_HOST = "ns1-nn1.example.com:9820";
     final String NS1_NN2_HOST_SVC = "ns1-nn2.example.com:9821";
-    final String NS1_NN2_HOST_DEFAULT_SVC = "ns1-nn2.example.com:9840";
    
     conf.set(DFS_NAMESERVICES, "ns1");
     conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, "ns1"),"nn1,nn2"); 
@@ -577,15 +567,12 @@ public class TestDFSUtil {
     conf.set(DFSUtil.addKeySuffixes(
         DFS_NAMENODE_RPC_ADDRESS_KEY, "ns1", "nn2"), NS1_NN2_HOST);
 
-    // The default service rpc address is used if no service address is defined
-    assertEquals(NS1_NN1_HOST_DEFAULT_SVC,
-        DFSUtil.getNamenodeServiceAddr(conf, null, "nn1"));
-    assertEquals(NS1_NN2_HOST_DEFAULT_SVC,
-        DFSUtil.getNamenodeServiceAddr(conf, null, "nn2"));
+    // The rpc address is used if no service address is defined
+    assertEquals(NS1_NN1_HOST, DFSUtil.getNamenodeServiceAddr(conf, null, "nn1"));
+    assertEquals(NS1_NN2_HOST, DFSUtil.getNamenodeServiceAddr(conf, null, "nn2"));
 
     // A nameservice is specified explicitly
-    assertEquals(NS1_NN1_HOST_DEFAULT_SVC,
-        DFSUtil.getNamenodeServiceAddr(conf, "ns1", "nn1"));
+    assertEquals(NS1_NN1_HOST, DFSUtil.getNamenodeServiceAddr(conf, "ns1", "nn1"));
     assertEquals(null, DFSUtil.getNamenodeServiceAddr(conf, "invalid", "nn1"));
     
     // The service addrs are used when they are defined
@@ -1009,92 +996,6 @@ public class TestDFSUtil {
   }
 
   @Test
-  public void testGetNNServiceRpcAddresses() throws IOException {
-    Configuration conf = new HdfsConfiguration();
-    final String NN_HOST = "nn.example.com";
-    final String NN_ADDRESS = "hdfs://" + NN_HOST + ":9000/";
-    conf.set(FS_DEFAULT_NAME_KEY, NN_ADDRESS);
-
-    // No service RPC, no rpc
-    Map<String, Map<String, InetSocketAddress>> nsMap = DFSUtil
-        .getNNServiceRpcAddresses(conf);
-    assertEquals(1, nsMap.size());
-    InetSocketAddress address = nsMap.get(null).get(null);
-    assertEquals(DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT,
-        address.getPort());
-    assertEquals(NN_HOST, address.getHostName());
-
-    // No service RPC
-    final String RPC_ADDRESS = NN_HOST + ":9191";
-    conf.set(DFS_NAMENODE_RPC_ADDRESS_KEY, RPC_ADDRESS);
-    nsMap = DFSUtil.getNNServiceRpcAddresses(conf);
-    assertEquals(1, nsMap.size());
-    address = nsMap.get(null).get(null);
-    assertEquals(DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT,
-        address.getPort());
-    assertEquals(NN_HOST, address.getHostName());
-
-    // Service RPC present
-    final String SERVICE_RPC_ADDRESS = NN_HOST + ":9292";
-    conf.set(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, SERVICE_RPC_ADDRESS);
-    nsMap = DFSUtil.getNNServiceRpcAddresses(conf);
-    assertEquals(1, nsMap.size());
-    address = nsMap.get(null).get(null);
-    assertEquals(9292, address.getPort());
-    assertEquals(NN_HOST, address.getHostName());
-  }
-
-  @Test
-  public void testGetNNServiceRpcAddressesForHA() throws IOException {
-    Configuration conf = new HdfsConfiguration();
-
-    final String NS = "mycluster";
-    final String NN1_HOST = "nn1.example.com";
-    final String NN2_HOST = "nn2.example.com";
-    conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://mycluster");
-
-    conf.set(DFS_NAMESERVICES, NS);
-    conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, NS),
-        "nn1,nn2");
-    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, NS, "nn1"),
-        NN1_HOST + ":9820");
-    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, NS, "nn2"),
-        NN2_HOST + ":9820");
-
-    assertTrue(HAUtil.isHAEnabled(conf, NS));
-
-    // Without Service RPC keys
-    Map<String, Map<String, InetSocketAddress>> nsMap =
-        DFSUtil.getNNServiceRpcAddresses(conf);
-    assertEquals(1, nsMap.size());
-    Map<String, InetSocketAddress> nnMap = nsMap.get(NS);
-    assertEquals(2, nnMap.size());
-    InetSocketAddress nn1Address = nnMap.get("nn1");
-    assertEquals(NN1_HOST, nn1Address.getHostName());
-    assertEquals(DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT, nn1Address.getPort());
-    InetSocketAddress nn2Address = nnMap.get("nn2");
-    assertEquals(NN2_HOST, nn2Address.getHostName());
-    assertEquals(DFS_NAMENODE_SERVICE_RPC_PORT_DEFAULT, nn2Address.getPort());
-
-    // With Service RPC keys
-    final int CUSTOM_SERVICE_PORT = 9191;
-    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
-        NS, "nn1"), NN1_HOST + ":" + CUSTOM_SERVICE_PORT);
-    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
-        NS, "nn2"), NN2_HOST + ":" + CUSTOM_SERVICE_PORT);
-    nsMap = DFSUtil.getNNServiceRpcAddresses(conf);
-    assertEquals(1, nsMap.size());
-    nnMap = nsMap.get(NS);
-    assertEquals(2, nnMap.size());
-    nn1Address = nnMap.get("nn1");
-    assertEquals(NN1_HOST, nn1Address.getHostName());
-    assertEquals(CUSTOM_SERVICE_PORT, nn1Address.getPort());
-    nn2Address = nnMap.get("nn2");
-    assertEquals(NN2_HOST, nn2Address.getHostName());
-    assertEquals(CUSTOM_SERVICE_PORT, nn2Address.getPort());
-  }
-
-  @Test
   public void testGetNNServiceRpcAddressesForNsIds() throws IOException {
     Configuration conf = new HdfsConfiguration();
     conf.set(DFS_NAMESERVICES, "nn1,nn2");
@@ -1116,13 +1017,13 @@ public class TestDFSUtil {
     }
 
     Map<String, Map<String, InetSocketAddress>> nnMap = DFSUtil
-            .getNNServiceRpcAddresses(conf);
+            .getNNServiceRpcAddressesForCluster(conf);
     assertEquals(1, nnMap.size());
     assertTrue(nnMap.containsKey("nn1"));
 
     conf.set(DFS_INTERNAL_NAMESERVICES_KEY, "nn3");
     try {
-      DFSUtil.getNNServiceRpcAddresses(conf);
+      DFSUtil.getNNServiceRpcAddressesForCluster(conf);
       fail("Should fail for misconfiguration");
     } catch (IOException ignored) {
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java
index 1914b78..59e8555 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.test.PathUtils;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import java.io.File;
@@ -278,14 +277,17 @@ public class TestHDFSServerPorts {
       // different http port
       conf2.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, THIS_HOST);
       started = canStartNameNode(conf2);
-      assertFalse("Should've failed on service port", started);
 
-      // reset conf2 since NameNode modifies it
-      FileSystem.setDefaultUri(conf2, "hdfs://" + THIS_HOST);
-      conf2.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, THIS_HOST);
-      // Set Service address
-      conf2.set(DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,  THIS_HOST);
-      started = canStartNameNode(conf2);
+      if (withService) {
+        assertFalse("Should've failed on service port", started);
+
+        // reset conf2 since NameNode modifies it
+        FileSystem.setDefaultUri(conf2, "hdfs://" + THIS_HOST);
+        conf2.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, THIS_HOST);
+        // Set Service address      
+        conf2.set(DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,  THIS_HOST);
+        started = canStartNameNode(conf2);        
+      }
       assertTrue(started);
     } finally {
       stopNameNode(nn);
@@ -357,39 +359,38 @@ public class TestHDFSServerPorts {
     }
   }
     
-  /**
-   * Verify BackupNode port usage.
-   */
-  @Ignore
-  @Test(timeout = 300000)
-  public void testBackupNodePorts() throws Exception {
-    NameNode nn = null;
-    try {
-      nn = startNameNode();
-
-      Configuration backup_config = new HdfsConfiguration(config);
-      backup_config.set(
-          DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY, THIS_HOST);
-      // bind http server to the same port as name-node
-      backup_config.set(DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY,
-          backup_config.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY));
-
-      LOG.info("= Starting 1 on: " + backup_config.get(
-          DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY));
-
-      assertFalse("Backup started on same port as Namenode",
-                         canStartBackupNode(backup_config)); // should fail
-
-      // bind http server to a different port
-      backup_config.set(
-          DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY, THIS_HOST);
-      LOG.info("= Starting 2 on: " + backup_config.get(
-          DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY));
-
-      boolean started = canStartBackupNode(backup_config);
-      assertTrue("Backup Namenode should've started", started); // should start now
-    } finally {
-      stopNameNode(nn);
-    }
+    /**
+     * Verify BackupNode port usage.
+     */
+    @Test(timeout = 300000)
+    public void testBackupNodePorts() throws Exception {
+      NameNode nn = null;
+      try {
+        nn = startNameNode();
+
+        Configuration backup_config = new HdfsConfiguration(config);
+        backup_config.set(
+            DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY, THIS_HOST);
+        // bind http server to the same port as name-node
+        backup_config.set(DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY, 
+            backup_config.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY));
+
+        LOG.info("= Starting 1 on: " + backup_config.get(
+            DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY));
+
+        assertFalse("Backup started on same port as Namenode", 
+                           canStartBackupNode(backup_config)); // should fail
+
+        // bind http server to a different port
+        backup_config.set(
+            DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY, THIS_HOST);
+        LOG.info("= Starting 2 on: " + backup_config.get(
+            DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY));
+
+        boolean started = canStartBackupNode(backup_config);
+        assertTrue("Backup Namenode should've started", started); // should start now
+      } finally {
+        stopNameNode(nn);
+      }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
index df6dc03..f25d28f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
@@ -324,7 +324,7 @@ public class TestSafeMode {
     } catch (RemoteException re) {
       assertEquals(SafeModeException.class.getName(), re.getClassName());
       GenericTestUtils.assertExceptionContains(
-          NameNode.getServiceAddress(conf).getHostName(), re);
+          NameNode.getServiceAddress(conf, true).getHostName(), re);
     } catch (IOException ioe) {
       fail("Encountered exception" + " " + StringUtils.stringifyException(ioe));
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
index 501ba77..c163894 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
@@ -77,9 +77,7 @@ public class MiniQJMHACluster {
   public static MiniDFSNNTopology createDefaultTopology(int nns, int startingPort) {
     MiniDFSNNTopology.NSConf nameservice = new MiniDFSNNTopology.NSConf(NAMESERVICE);
     for (int i = 0; i < nns; i++) {
-      nameservice.addNN(new MiniDFSNNTopology.NNConf("nn" + i)
-          .setIpcPort(startingPort++)
-          .setServicePort(startingPort++)
+      nameservice.addNN(new MiniDFSNNTopology.NNConf("nn" + i).setIpcPort(startingPort++)
           .setHttpPort(startingPort++));
     }
 
@@ -150,9 +148,8 @@ public class MiniQJMHACluster {
     int port = basePort;
     for (int i = 0; i < numNNs; i++) {
       nns.add("127.0.0.1:" + port);
-      // increment by 3 each time to account for the http and the service port
-      // in the config setting
-      port += 3;
+      // increment by 2 each time to account for the http port in the config setting
+      port += 2;
     }
 
     // use standard failover configurations

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java
index 516f159..1444193 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java
@@ -89,9 +89,8 @@ public class TestBalancerWithHANameNodes {
           / numOfDatanodes, (short) numOfDatanodes, 1);
 
       // start up an empty node with the same capacity and on the same rack
-      cluster.startDataNodes(cluster.getClusterConfiguration(),
-          1, true, null, new String[] {newNodeRack},
-          new long[] {newNodeCapacity});
+      cluster.startDataNodes(conf, 1, true, null, new String[] { newNodeRack },
+          new long[] { newNodeCapacity });
       totalCapacity += newNodeCapacity;
       TestBalancer.waitForHeartBeat(totalUsedSpace, totalCapacity, client,
           cluster);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
index c199c9c..876a854 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
@@ -105,11 +105,8 @@ public class InternalDataNodeTestUtils {
    *
    * @throws IOException
    */
-  public static DataNode startDNWithMockNN(
-      Configuration conf,
-      final InetSocketAddress nnSocketAddr,
-      final InetSocketAddress nnServiceAddr,
-      final String dnDataDir)
+  public static DataNode startDNWithMockNN(Configuration conf,
+      final InetSocketAddress nnSocketAddr, final String dnDataDir)
       throws IOException {
 
     FileSystem.setDefaultUri(conf, "hdfs://" + nnSocketAddr.getHostName() + ":"
@@ -152,7 +149,7 @@ public class InternalDataNodeTestUtils {
       @Override
       DatanodeProtocolClientSideTranslatorPB connectToNN(
           InetSocketAddress nnAddr) throws IOException {
-        Assert.assertEquals(nnServiceAddr, nnAddr);
+        Assert.assertEquals(nnSocketAddr, nnAddr);
         return namenode;
       }
     };

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
index 98450f6..311d5a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
@@ -124,6 +124,8 @@ public class TestBlockRecovery {
   private final static long RECOVERY_ID = 3000L;
   private final static String CLUSTER_ID = "testClusterID";
   private final static String POOL_ID = "BP-TEST";
+  private final static InetSocketAddress NN_ADDR = new InetSocketAddress(
+      "localhost", 5020);
   private final static long BLOCK_ID = 1000L;
   private final static long GEN_STAMP = 2000L;
   private final static long BLOCK_LEN = 3000L;
@@ -186,7 +188,7 @@ public class TestBlockRecovery {
     }
     conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
     FileSystem.setDefaultUri(conf,
-        "hdfs://localhost:5020");
+        "hdfs://" + NN_ADDR.getHostName() + ":" + NN_ADDR.getPort());
     ArrayList<StorageLocation> locations = new ArrayList<StorageLocation>();
     File dataDir = new File(DATA_DIR);
     FileUtil.fullyDelete(dataDir);
@@ -229,7 +231,7 @@ public class TestBlockRecovery {
       @Override
       DatanodeProtocolClientSideTranslatorPB connectToNN(
           InetSocketAddress nnAddr) throws IOException {
-        Assert.assertEquals("localhost:9840", nnAddr.toString());
+        Assert.assertEquals(NN_ADDR, nnAddr);
         return namenode;
       }
     };

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java
index bee6c1d..32fda37 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java
@@ -61,16 +61,11 @@ import com.google.common.base.Supplier;
 public class TestDataNodeMetricsLogger {
   static final Log LOG = LogFactory.getLog(TestDataNodeMetricsLogger.class);
 
-  @Rule
-  public Timeout globalTimeout = new Timeout(120_000);
-
   private static final String DATA_DIR = MiniDFSCluster.getBaseDirectory()
       + "data";
 
   private final static InetSocketAddress NN_ADDR = new InetSocketAddress(
       "localhost", 5020);
-  private final static InetSocketAddress NN_SERVICE_ADDR =
-      new InetSocketAddress("localhost", 5021);
 
   private DataNode dn;
 
@@ -91,13 +86,10 @@ public class TestDataNodeMetricsLogger {
     conf.set(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
     conf.set(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY, "0.0.0.0:0");
     conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
-    conf.set(DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
-        NN_SERVICE_ADDR.getHostName() + ":" + NN_SERVICE_ADDR.getPort());
     conf.setInt(DFS_DATANODE_METRICS_LOGGER_PERIOD_SECONDS_KEY,
         enableMetricsLogging ? 1 : 0); // If enabled, log early and log often
 
-    dn = InternalDataNodeTestUtils.startDNWithMockNN(
-        conf, NN_ADDR, NN_SERVICE_ADDR, DATA_DIR);
+    dn = InternalDataNodeTestUtils.startDNWithMockNN(conf, NN_ADDR, DATA_DIR);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java
index 25650fd..8e1e236 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java
@@ -109,16 +109,16 @@ public class TestDataNodeMultipleRegistrations {
       BPOfferService bpos2 = dn.getAllBpOs().get(1);
 
       // The order of bpos is not guaranteed, so fix the order
-      if (getNNSocketAddress(bpos1).equals(nn2.getServiceRpcAddress())) {
+      if (getNNSocketAddress(bpos1).equals(nn2.getNameNodeAddress())) {
         BPOfferService tmp = bpos1;
         bpos1 = bpos2;
         bpos2 = tmp;
       }
 
       assertEquals("wrong nn address", getNNSocketAddress(bpos1),
-          nn1.getServiceRpcAddress());
+          nn1.getNameNodeAddress());
       assertEquals("wrong nn address", getNNSocketAddress(bpos2),
-          nn2.getServiceRpcAddress());
+          nn2.getNameNodeAddress());
       assertEquals("wrong bpid", bpos1.getBlockPoolId(), bpid1);
       assertEquals("wrong bpid", bpos2.getBlockPoolId(), bpid2);
       assertEquals("wrong cid", dn.getClusterId(), cid1);
@@ -182,7 +182,7 @@ public class TestDataNodeMultipleRegistrations {
 
       assertEquals("wrong nn address",
           getNNSocketAddress(bpos1),
-          nn1.getServiceRpcAddress());
+          nn1.getNameNodeAddress());
       assertEquals("wrong bpid", bpos1.getBlockPoolId(), bpid1);
       assertEquals("wrong cid", dn.getClusterId(), cid1);
       cluster.shutdown();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java
index 884c93d..1dfd3c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java
@@ -51,10 +51,8 @@ public class TestDataNodeReconfiguration {
   private static final Log LOG = LogFactory.getLog(TestBlockRecovery.class);
   private static final String DATA_DIR = MiniDFSCluster.getBaseDirectory()
       + "data";
-  private final static InetSocketAddress NN_ADDR =
-      new InetSocketAddress("localhost", 5020);
-  private final static InetSocketAddress NN_SERVICE_ADDR =
-      new InetSocketAddress("localhost", 5021);
+  private final static InetSocketAddress NN_ADDR = new InetSocketAddress(
+      "localhost", 5020);
   private final int NUM_NAME_NODE = 1;
   private final int NUM_DATA_NODE = 10;
   private MiniDFSCluster cluster;
@@ -101,13 +99,10 @@ public class TestDataNodeReconfiguration {
     conf.set(DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
     conf.set(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY, "0.0.0.0:0");
     conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
-    conf.set(DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
-        NN_SERVICE_ADDR.getHostName() + ":" + NN_SERVICE_ADDR.getPort());
 
     DataNode[] result = new DataNode[numDateNode];
     for (int i = 0; i < numDateNode; i++) {
-      result[i] = InternalDataNodeTestUtils.startDNWithMockNN(
-          conf, NN_ADDR, NN_SERVICE_ADDR, DATA_DIR);
+      result[i] = InternalDataNodeTestUtils.startDNWithMockNN(conf, NN_ADDR, DATA_DIR);
     }
     return result;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
index 5218021..bb1d9ef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
@@ -78,6 +78,8 @@ public class TestDatanodeProtocolRetryPolicy {
   ArrayList<StorageLocation> locations = new ArrayList<StorageLocation>();
   private final static String CLUSTER_ID = "testClusterID";
   private final static String POOL_ID = "BP-TEST";
+  private final static InetSocketAddress NN_ADDR = new InetSocketAddress(
+      "localhost", 5020);
   private static DatanodeRegistration datanodeRegistration =
       DFSTestUtil.getLocalDatanodeRegistration();
 
@@ -99,7 +101,7 @@ public class TestDatanodeProtocolRetryPolicy {
     conf.set(DFSConfigKeys.DFS_DATANODE_IPC_ADDRESS_KEY, "0.0.0.0:0");
     conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
     FileSystem.setDefaultUri(conf,
-        "hdfs://localhost:5020");
+        "hdfs://" + NN_ADDR.getHostName() + ":" + NN_ADDR.getPort());
     File dataDir = new File(DATA_DIR);
     FileUtil.fullyDelete(dataDir);
     dataDir.mkdirs();
@@ -226,7 +228,7 @@ public class TestDatanodeProtocolRetryPolicy {
       @Override
       DatanodeProtocolClientSideTranslatorPB connectToNN(
           InetSocketAddress nnAddr) throws IOException {
-        Assert.assertEquals("localhost:9840", nnAddr.toString());
+        Assert.assertEquals(NN_ADDR, nnAddr);
         return namenode;
       }
     };

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java
index 37d1b57..f8594ca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java
@@ -44,11 +44,6 @@ public class TestRefreshNamenodes {
   private final int nnPort3 = 2227;
   private final int nnPort4 = 2230;
 
-  private final int nnServicePort1 = 2222;
-  private final int nnServicePort2 = 2225;
-  private final int nnServicePort3 = 2228;
-  private final int nnServicePort4 = 2231;
-
   @Test
   public void testRefreshNamenodes() throws IOException {
     // Start cluster with a single NN and DN
@@ -57,9 +52,7 @@ public class TestRefreshNamenodes {
     try {
       MiniDFSNNTopology topology = new MiniDFSNNTopology()
         .addNameservice(new NSConf("ns1").addNN(
-            new NNConf(null)
-                .setIpcPort(nnPort1)
-                .setServicePort(nnServicePort1)))
+            new NNConf(null).setIpcPort(nnPort1)))
         .setFederation(true);
       cluster = new MiniDFSCluster.Builder(conf)
         .nnTopology(topology)
@@ -68,20 +61,20 @@ public class TestRefreshNamenodes {
       DataNode dn = cluster.getDataNodes().get(0);
       assertEquals(1, dn.getAllBpOs().size());
 
-      cluster.addNameNode(conf, nnPort2, nnServicePort2);
+      cluster.addNameNode(conf, nnPort2);
       assertEquals(2, dn.getAllBpOs().size());
 
-      cluster.addNameNode(conf, nnPort3, nnServicePort3);
+      cluster.addNameNode(conf, nnPort3);
       assertEquals(3, dn.getAllBpOs().size());
 
-      cluster.addNameNode(conf, nnPort4, nnServicePort4);
+      cluster.addNameNode(conf, nnPort4);
 
       // Ensure a BPOfferService in the datanodes corresponds to
       // a namenode in the cluster
       Set<InetSocketAddress> nnAddrsFromCluster = Sets.newHashSet();
       for (int i = 0; i < 4; i++) {
         assertTrue(nnAddrsFromCluster.add(
-            cluster.getNameNode(i).getServiceRpcAddress()));
+            cluster.getNameNode(i).getNameNodeAddress()));
       }
       
       Set<InetSocketAddress> nnAddrsFromDN = Sets.newHashSet();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
index 5c58e0a..10d9f11 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
@@ -54,7 +54,6 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import com.google.common.base.Supplier;
@@ -62,7 +61,6 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
-@Ignore("Temporarily disabling the BackupNode unit test.")
 public class TestBackupNode {
   public static final Log LOG = LogFactory.getLog(TestBackupNode.class);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
index 4282c22..2e49674 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
@@ -1364,9 +1364,9 @@ public class TestCheckpoint {
       Configuration snConf1 = new HdfsConfiguration(cluster.getConfiguration(0));
       Configuration snConf2 = new HdfsConfiguration(cluster.getConfiguration(1));
       InetSocketAddress nn1RpcAddress = cluster.getNameNode(0)
-          .getServiceRpcAddress();
+          .getNameNodeAddress();
       InetSocketAddress nn2RpcAddress = cluster.getNameNode(1)
-          .getServiceRpcAddress();
+          .getNameNodeAddress();
       String nn1 = nn1RpcAddress.getHostName() + ":" + nn1RpcAddress.getPort();
       String nn2 = nn2RpcAddress.getHostName() + ":" + nn2RpcAddress.getPort();
 
@@ -1923,7 +1923,6 @@ public class TestCheckpoint {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
           .format(true).build();
       int origPort = cluster.getNameNodePort();
-      int origServicePort = cluster.getNameNodeServicePort();
       int origHttpPort = cluster.getNameNode().getHttpAddress().getPort();
       Configuration snnConf = new Configuration(conf);
       File checkpointDir = new File(MiniDFSCluster.getBaseDirectory(),
@@ -1950,7 +1949,6 @@ public class TestCheckpoint {
       cluster = new MiniDFSCluster.Builder(conf)
           .numDataNodes(0)
           .nameNodePort(origPort)
-          .nameNodeServicePort(origServicePort)
           .nameNodeHttpPort(origHttpPort)
           .format(true).build();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
index d21b275..36638e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
@@ -661,15 +661,12 @@ public class TestNameNodeMXBean {
     for (int i = 0; i < 5; i++) {
       try{
         // Have to specify IPC ports so the NNs can talk to each other.
-        int[] ports = ServerSocketUtil.getPorts(4);
+        int[] ports = ServerSocketUtil.getPorts(2);
         MiniDFSNNTopology topology = new MiniDFSNNTopology()
             .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
-                .addNN(new MiniDFSNNTopology.NNConf("nn1")
-                    .setIpcPort(ports[0])
-                    .setServicePort(ports[1]))
-                .addNN(new MiniDFSNNTopology.NNConf("nn2")
-                    .setIpcPort(ports[2])
-                    .setServicePort(ports[3])));
+                .addNN(new MiniDFSNNTopology.NNConf("nn1").setIpcPort(ports[0]))
+                .addNN(
+                    new MiniDFSNNTopology.NNConf("nn2").setIpcPort(ports[1])));
 
         cluster = new MiniDFSCluster.Builder(conf)
             .nnTopology(topology).numDataNodes(0)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetricsLogger.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetricsLogger.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetricsLogger.java
index d7216c0..9a0e67c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetricsLogger.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetricsLogger.java
@@ -110,7 +110,6 @@ public class TestNameNodeMetricsLogger {
       throws IOException {
     Configuration conf = new HdfsConfiguration();
     conf.set(FS_DEFAULT_NAME_KEY, "hdfs://localhost:0");
-    conf.set(DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, "0.0.0.0:0");
     conf.set(DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
     conf.setInt(DFS_NAMENODE_METRICS_LOGGER_PERIOD_SECONDS_KEY,
         enableMetricsLogging ? 1 : 0);  // If enabled, log early and log often

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
index 981785a..0cf1fed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
@@ -125,8 +125,6 @@ public class TestValidateConfigurationSettings {
     // Set ephemeral ports 
     conf.set(DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY,
         "127.0.0.1:0");
-    conf.set(DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY,
-        "127.0.0.1:0");
     conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY,
         "127.0.0.1:0");
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/65a94100/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
index a367167..169bbee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.test.GenericTestUtils;


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