Merged revision(s) 1601144-1601868, 1598456-1601149 from hadoop/common/trunk

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5442@1601869 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 842f529..a70221e 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -297,9 +297,6 @@
     HADOOP-9394. Port findHangingTest.sh from HBase to Hadoop. (Andrew Wang
     via atm)
 
-    HADOOP-9099. NetUtils.normalizeHostName fails on domains where 
-    UnknownHost resolves to an IP address. (Ivan Mitic via suresh)
-
     HADOOP-9431 TestSecurityUtil#testLocalHostNameForNullOrWild on systems where hostname
     contains capital letters  (Chris Nauroth via sanjay)
 
@@ -373,6 +370,11 @@
 
     HADOOP-9704. Write metrics sink plugin for Hadoop/Graphite (Chu Tong, Alex Newman and Babak Behzad via raviprak)
 
+    HADOOP-9629. Support Windows Azure Storage - Blob as a file system in Hadoop.
+    (Dexter Bradshaw, Mostafa Elhemali, Xi Fang, Johannes Klein, David Lao,
+    Mike Liddell, Chuan Liu, Lengning Liu, Ivan Mitic, Michael Rys,
+    Alexander Stojanovic, Brian Swan, and Min Wei via cnauroth)
+
   IMPROVEMENTS
 
     HADOOP-10451. Remove unused field and imports from SaslRpcServer.
@@ -429,6 +431,10 @@
     HADOOP-10448. Support pluggable mechanism to specify proxy user settings.
     (Benoy Antony via Arpit Agarwal)
 
+    HADOOP-9555. HA functionality that uses ZooKeeper may experience inadvertent
+    TCP RST and miss session expiration event due to bug in client connection
+    management. (cnauroth)
+
   OPTIMIZATIONS
 
   BUG FIXES 
@@ -536,6 +542,11 @@
     HADOOP-10647. String Format Exception in SwiftNativeFileSystemStore.java.
     (Gene Kim via stevel)
 
+    HADOOP-9099. NetUtils.normalizeHostName fails on domains where
+    UnknownHost resolves to an IP address. (Ivan Mitic via suresh)
+
+    HADOOP-10664. TestNetUtils.testNormalizeHostName fails. (atm)
+
 Release 2.4.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
index 5fb9b75..2cfa243 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
@@ -883,15 +883,16 @@
    * @param seed Used to seed the pseudo-random generator that randomizes the
    *          set of nodes at each network distance.
    */
-  public void sortByDistance(Node reader, Node[] nodes, long seed) {
+  public void sortByDistance(Node reader, Node[] nodes,
+      int activeLen, long seed) {
     /** Sort weights for the nodes array */
-    int[] weights = new int[nodes.length];
-    for (int i=0; i<nodes.length; i++) {
+    int[] weights = new int[activeLen];
+    for (int i=0; i<activeLen; i++) {
       weights[i] = getWeight(reader, nodes[i]);
     }
     // Add weight/node pairs to a TreeMap to sort
     TreeMap<Integer, List<Node>> tree = new TreeMap<Integer, List<Node>>();
-    for (int i=0; i<nodes.length; i++) {
+    for (int i=0; i<activeLen; i++) {
       int weight = weights[i];
       Node node = nodes[i];
       List<Node> list = tree.get(weight);
@@ -917,7 +918,7 @@
         }
       }
     }
-    Preconditions.checkState(idx == nodes.length,
+    Preconditions.checkState(idx == activeLen,
         "Sorted the wrong number of nodes!");
   }
 }
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopologyWithNodeGroup.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopologyWithNodeGroup.java
index 975fe4d..7243f72 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopologyWithNodeGroup.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopologyWithNodeGroup.java
@@ -279,7 +279,8 @@
    *          set of nodes at each network distance.
    */
   @Override
-  public void sortByDistance( Node reader, Node[] nodes, long seed) {
+  public void sortByDistance( Node reader, Node[] nodes,
+      int activeLen, long seed) {
     // If reader is not a datanode (not in NetworkTopology tree), we need to
     // replace this reader with a sibling leaf node in tree.
     if (reader != null && !this.contains(reader)) {
@@ -292,7 +293,7 @@
         return;
       }
     }
-    super.sortByDistance(reader, nodes, seed);
+    super.sortByDistance(reader, nodes, nodes.length, seed);
   }
 
   /** InnerNodeWithNodeGroup represents a switch/router of a data center, rack
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java
index 995392b..b03afca 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java
@@ -605,7 +605,7 @@
   @Test
   public void testNormalizeHostName() {	
     List<String> hosts = Arrays.asList(new String[] {"127.0.0.1",
-        "localhost", "3w.org", "UnknownHost123"});
+        "localhost", "1.kanyezone.appspot.com", "UnknownHost123"});
     List<String> normalizedHosts = NetUtils.normalizeHostNames(hosts);
     // when ipaddress is normalized, same address is expected in return
     assertEquals(normalizedHosts.get(0), hosts.get(0));
@@ -636,4 +636,4 @@
     String gotStr = StringUtils.join(got, ", ");
     assertEquals(expectStr, gotStr);
   }
-}
\ No newline at end of file
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java
index c082cce..ca61c1e 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetworkTopologyWithNodeGroup.java
@@ -104,7 +104,8 @@
     testNodes[1] = dataNodes[2];
     testNodes[2] = dataNodes[3];
     testNodes[3] = dataNodes[0];
-    cluster.sortByDistance(dataNodes[0], testNodes, 0xDEADBEEF);
+    cluster.sortByDistance(dataNodes[0], testNodes,
+        testNodes.length, 0xDEADBEEF);
     assertTrue(testNodes[0] == dataNodes[0]);
     assertTrue(testNodes[1] == dataNodes[1]);
     assertTrue(testNodes[2] == dataNodes[2]);
@@ -115,7 +116,8 @@
     testNodes[1] = dataNodes[4];
     testNodes[2] = dataNodes[1];
     testNodes[3] = dataNodes[0];
-    cluster.sortByDistance(dataNodes[0], testNodes, 0xDEADBEEF);
+    cluster.sortByDistance(dataNodes[0], testNodes,
+        testNodes.length, 0xDEADBEEF);
     assertTrue(testNodes[0] == dataNodes[0]);
     assertTrue(testNodes[1] == dataNodes[1]);
 
@@ -124,7 +126,8 @@
     testNodes[1] = dataNodes[3];
     testNodes[2] = dataNodes[2];
     testNodes[3] = dataNodes[0];
-    cluster.sortByDistance(dataNodes[0], testNodes, 0xDEADBEEF);
+    cluster.sortByDistance(dataNodes[0], testNodes,
+        testNodes.length, 0xDEADBEEF);
     assertTrue(testNodes[0] == dataNodes[0]);
     assertTrue(testNodes[1] == dataNodes[2]);
 
@@ -133,7 +136,8 @@
     testNodes[1] = dataNodes[7];
     testNodes[2] = dataNodes[2];
     testNodes[3] = dataNodes[0];
-    cluster.sortByDistance(computeNode, testNodes, 0xDEADBEEF);
+    cluster.sortByDistance(computeNode, testNodes,
+        testNodes.length, 0xDEADBEEF);
     assertTrue(testNodes[0] == dataNodes[0]);
     assertTrue(testNodes[1] == dataNodes[2]);
   }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 1c8674e..8714d15 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -482,10 +482,16 @@
     HDFS-6297. Add CLI testcases to reflect new features of dfs and dfsadmin
     (Dasha Boudnik via cos)
 
+    HDFS-6399. Add note about setfacl in HDFS permissions guide.
+    (cnauroth via wang)
+
   OPTIMIZATIONS
 
     HDFS-6214. Webhdfs has poor throughput for files >2GB (daryn)
 
+    HDFS-6460. Ignore stale and decommissioned nodes in
+    NetworkTopology#sortByDistance. (Yongjun Zhang via wang)
+
   BUG FIXES 
 
     HDFS-6112. NFS Gateway docs are incorrect for allowed hosts configuration.
@@ -654,6 +660,15 @@
     HDFS-6497. Make TestAvailableSpaceVolumeChoosingPolicy deterministic
     (cmccabe)
 
+    HDFS-6500. Snapshot shouldn't be removed silently after renaming to an 
+    existing snapshot. (Nicholas SZE via junping_du)
+
+    HDFS-6257. TestCacheDirectives#testExceedsCapacity fails occasionally
+    (cmccabe)
+
+    HDFS-6364. Incorrect check for unknown datanode in Balancer. (Benoy
+    Antony via Arpit Agarwal)
+
 Release 2.4.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
index 925db17..16bcc0b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
@@ -26,7 +26,6 @@
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.security.token.Token;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
 /**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
index 8061b05..a936901 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
@@ -696,7 +696,7 @@
             // update locations
             for (String datanodeUuid : blk.getDatanodeUuids()) {
               final BalancerDatanode d = datanodeMap.get(datanodeUuid);
-              if (datanode != null) { // not an unknown datanode
+              if (d != null) { // not an unknown datanode
                 block.addLocation(d);
               }
             }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 8806fb9..aea34ec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -331,6 +331,18 @@
     return heartbeatManager;
   }
 
+  private boolean isInactive(DatanodeInfo datanode) {
+    if (datanode.isDecommissioned()) {
+      return true;
+    }
+
+    if (avoidStaleDataNodesForRead) {
+      return datanode.isStale(staleInterval);
+    }
+      
+    return false;
+  }
+  
   /** Sort the located blocks by the distance to the target host. */
   public void sortLocatedBlocks(final String targethost,
       final List<LocatedBlock> locatedblocks) {
@@ -351,10 +363,17 @@
         DFSUtil.DECOM_COMPARATOR;
         
     for (LocatedBlock b : locatedblocks) {
-      networktopology.sortByDistance(client, b.getLocations(), b
-          .getBlock().getBlockId());
+      DatanodeInfo[] di = b.getLocations();
       // Move decommissioned/stale datanodes to the bottom
-      Arrays.sort(b.getLocations(), comparator);
+      Arrays.sort(di, comparator);
+      
+      int lastActiveIndex = di.length - 1;
+      while (lastActiveIndex > 0 && isInactive(di[lastActiveIndex])) {
+          --lastActiveIndex;
+      }
+      int activeLen = lastActiveIndex + 1;      
+      networktopology.sortByDistance(client, b.getLocations(), activeLen,
+          b.getBlock().getBlockId());
     }
   }
   
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectorySnapshottable.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectorySnapshottable.java
index 8944e90..4288ec2a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectorySnapshottable.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/INodeDirectorySnapshottable.java
@@ -234,7 +234,7 @@
    *           name does not exist or a snapshot with the new name already
    *           exists
    */
-  public void renameSnapshot(String path, String oldName, String newName)
+  void renameSnapshot(String path, String oldName, String newName)
       throws SnapshotException {
     if (newName.equals(oldName)) {
       return;
@@ -246,7 +246,7 @@
     } else {
       final byte[] newNameBytes = DFSUtil.string2Bytes(newName);
       int indexOfNew = searchSnapshot(newNameBytes);
-      if (indexOfNew > 0) {
+      if (indexOfNew >= 0) {
         throw new SnapshotException("The snapshot " + newName
             + " already exists for directory " + path);
       }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsPermissionsGuide.apt.vm b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsPermissionsGuide.apt.vm
index e0a387f..f9aac0b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsPermissionsGuide.apt.vm
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsPermissionsGuide.apt.vm
@@ -395,8 +395,8 @@
        permission checking is turned off, but all other behavior is
        unchanged. Switching from one parameter value to the other does not
        change the mode, owner or group of files or directories.
-       Regardless of whether permissions are on or off, chmod, chgrp and
-       chown always check permissions. These functions are only useful in
+       Regardless of whether permissions are on or off, chmod, chgrp, chown and
+       setfacl always check permissions. These functions are only useful in
        the permissions context, and so there is no backwards compatibility
        issue. Furthermore, this allows administrators to reliably set
        owners and permissions in advance of turning on regular permissions
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSnapshotCommands.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSnapshotCommands.java
index d2c7a09..e2db636 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSnapshotCommands.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSnapshotCommands.java
@@ -186,6 +186,17 @@
     FsShellRun("-ls /sub1/.snapshot", 0, "/sub1/.snapshot/sn.rename");
     FsShellRun("-ls /sub1/.snapshot/sn.rename", 0, "/sub1/.snapshot/sn.rename/sub1sub1");
     FsShellRun("-ls /sub1/.snapshot/sn.rename", 0, "/sub1/.snapshot/sn.rename/sub1sub2");
+
+    //try renaming from a non-existing snapshot
+    FsShellRun("-renameSnapshot /sub1 sn.nonexist sn.rename", 1,
+        "renameSnapshot: The snapshot sn.nonexist does not exist for directory /sub1");
+
+    //try renaming to existing snapshots
+    FsShellRun("-createSnapshot /sub1 sn.new");
+    FsShellRun("-renameSnapshot /sub1 sn.new sn.rename", 1,
+        "renameSnapshot: The snapshot sn.rename already exists for directory /sub1");
+    FsShellRun("-renameSnapshot /sub1 sn.rename sn.new", 1,
+        "renameSnapshot: The snapshot sn.new already exists for directory /sub1");
   }
 
   @Test
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
index ed12460..8ef3887 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
@@ -72,7 +72,9 @@
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.io.nativeio.NativeIO;
@@ -1401,6 +1403,20 @@
         .build());
   }
 
+  /**
+   * Check that the NameNode is not attempting to cache anything.
+   */
+  private void checkPendingCachedEmpty(MiniDFSCluster cluster)
+      throws Exception {
+    final DatanodeManager datanodeManager =
+        cluster.getNamesystem().getBlockManager().getDatanodeManager();
+    for (DataNode dn : cluster.getDataNodes()) {
+      DatanodeDescriptor descriptor =
+          datanodeManager.getDatanode(dn.getDatanodeId());
+      Assert.assertTrue(descriptor.getPendingCached().isEmpty());
+    }
+  }
+
   @Test(timeout=60000)
   public void testExceedsCapacity() throws Exception {
     // Create a giant file
@@ -1418,21 +1434,16 @@
         .setPath(fileName).setReplication((short) 1).build());
     waitForCachedBlocks(namenode, -1, numCachedReplicas,
         "testExceeds:1");
-    // Check that no DNs saw an excess CACHE message
-    int lines = appender.countLinesWithMessage(
-        "more bytes in the cache: " +
-        DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY);
-    assertEquals("Namenode should not send extra CACHE commands", 0, lines);
+    checkPendingCachedEmpty(cluster);
+    Thread.sleep(1000);
+    checkPendingCachedEmpty(cluster);
+
     // Try creating a file with giant-sized blocks that exceed cache capacity
     dfs.delete(fileName, false);
     DFSTestUtil.createFile(dfs, fileName, 4096, fileLen, CACHE_CAPACITY * 2,
         (short) 1, 0xFADED);
-    // Nothing will get cached, so just force sleep for a bit
-    Thread.sleep(4000);
-    // Still should not see any excess commands
-    lines = appender.countLinesWithMessage(
-        "more bytes in the cache: " +
-        DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY);
-    assertEquals("Namenode should not send extra CACHE commands", 0, lines);
+    checkPendingCachedEmpty(cluster);
+    Thread.sleep(1000);
+    checkPendingCachedEmpty(cluster);
   }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java
index 4f98ae4..2e6383c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java
@@ -55,11 +55,18 @@
         DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/d1/r2"),
         DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/d2/r3"),
         DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/d2/r3"),
-        DFSTestUtil.getDatanodeDescriptor("8.8.8.8", "/d2/r3")
+        DFSTestUtil.getDatanodeDescriptor("8.8.8.8", "/d2/r3"),
+        DFSTestUtil.getDatanodeDescriptor("9.9.9.9", "/d3/r1"),
+        DFSTestUtil.getDatanodeDescriptor("10.10.10.10", "/d3/r1"),
+        DFSTestUtil.getDatanodeDescriptor("11.11.11.11", "/d3/r1"),
+        DFSTestUtil.getDatanodeDescriptor("12.12.12.12", "/d3/r2"),
+        DFSTestUtil.getDatanodeDescriptor("13.13.13.13", "/d3/r2")        
     };
     for (int i = 0; i < dataNodes.length; i++) {
       cluster.add(dataNodes[i]);
     }
+    dataNodes[9].setDecommissioned();
+    dataNodes[10].setDecommissioned();
   }
   
   @Test
@@ -100,7 +107,7 @@
 
   @Test
   public void testRacks() throws Exception {
-    assertEquals(cluster.getNumOfRacks(), 3);
+    assertEquals(cluster.getNumOfRacks(), 5);
     assertTrue(cluster.isOnSameRack(dataNodes[0], dataNodes[1]));
     assertFalse(cluster.isOnSameRack(dataNodes[1], dataNodes[2]));
     assertTrue(cluster.isOnSameRack(dataNodes[2], dataNodes[3]));
@@ -125,16 +132,33 @@
     testNodes[0] = dataNodes[1];
     testNodes[1] = dataNodes[2];
     testNodes[2] = dataNodes[0];
-    cluster.sortByDistance(dataNodes[0], testNodes, 0xDEADBEEF);
+    cluster.sortByDistance(dataNodes[0], testNodes,
+        testNodes.length, 0xDEADBEEF);
     assertTrue(testNodes[0] == dataNodes[0]);
     assertTrue(testNodes[1] == dataNodes[1]);
     assertTrue(testNodes[2] == dataNodes[2]);
 
+    // array contains both local node & local rack node & decommissioned node
+    DatanodeDescriptor[] dtestNodes = new DatanodeDescriptor[5];
+    dtestNodes[0] = dataNodes[8];
+    dtestNodes[1] = dataNodes[12];
+    dtestNodes[2] = dataNodes[11];
+    dtestNodes[3] = dataNodes[9];
+    dtestNodes[4] = dataNodes[10];
+    cluster.sortByDistance(dataNodes[8], dtestNodes,
+        dtestNodes.length - 2, 0xDEADBEEF);
+    assertTrue(dtestNodes[0] == dataNodes[8]);
+    assertTrue(dtestNodes[1] == dataNodes[11]);
+    assertTrue(dtestNodes[2] == dataNodes[12]);
+    assertTrue(dtestNodes[3] == dataNodes[9]);
+    assertTrue(dtestNodes[4] == dataNodes[10]);
+
     // array contains local node
     testNodes[0] = dataNodes[1];
     testNodes[1] = dataNodes[3];
     testNodes[2] = dataNodes[0];
-    cluster.sortByDistance(dataNodes[0], testNodes, 0xDEADBEEF);
+    cluster.sortByDistance(dataNodes[0], testNodes,
+        testNodes.length, 0xDEADBEEF);
     assertTrue(testNodes[0] == dataNodes[0]);
     assertTrue(testNodes[1] == dataNodes[1]);
     assertTrue(testNodes[2] == dataNodes[3]);
@@ -143,7 +167,8 @@
     testNodes[0] = dataNodes[5];
     testNodes[1] = dataNodes[3];
     testNodes[2] = dataNodes[1];
-    cluster.sortByDistance(dataNodes[0], testNodes, 0xDEADBEEF);
+    cluster.sortByDistance(dataNodes[0], testNodes,
+        testNodes.length, 0xDEADBEEF);
     assertTrue(testNodes[0] == dataNodes[1]);
     assertTrue(testNodes[1] == dataNodes[3]);
     assertTrue(testNodes[2] == dataNodes[5]);
@@ -152,7 +177,8 @@
     testNodes[0] = dataNodes[1];
     testNodes[1] = dataNodes[5];
     testNodes[2] = dataNodes[3];
-    cluster.sortByDistance(dataNodes[0], testNodes, 0xDEADBEEF);
+    cluster.sortByDistance(dataNodes[0], testNodes,
+        testNodes.length, 0xDEADBEEF);
     assertTrue(testNodes[0] == dataNodes[1]);
     assertTrue(testNodes[1] == dataNodes[3]);
     assertTrue(testNodes[2] == dataNodes[5]);
@@ -161,7 +187,8 @@
     testNodes[0] = dataNodes[1];
     testNodes[1] = dataNodes[5];
     testNodes[2] = dataNodes[3];
-    cluster.sortByDistance(dataNodes[0], testNodes, 0xDEAD);
+    cluster.sortByDistance(dataNodes[0], testNodes,
+        testNodes.length, 0xDEAD);
     // sortByDistance does not take the "data center" layer into consideration
     // and it doesn't sort by getDistance, so 1, 5, 3 is also valid here
     assertTrue(testNodes[0] == dataNodes[1]);
@@ -176,7 +203,8 @@
       testNodes[0] = dataNodes[5];
       testNodes[1] = dataNodes[6];
       testNodes[2] = dataNodes[7];
-      cluster.sortByDistance(dataNodes[i], testNodes, 0xBEADED+i);
+      cluster.sortByDistance(dataNodes[i], testNodes,
+          testNodes.length, 0xBEADED+i);
       if (first == null) {
         first = testNodes[0];
       } else {
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 79ba4d0..0686809 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -207,6 +207,12 @@
 
     MAPREDUCE-5899. Support incremental data copy in DistCp. (jing9)
 
+    MAPREDUCE-5886. Allow wordcount example job to accept multiple input paths.
+    (cnauroth)
+
+    MAPREDUCE-5834. Increased test-timeouts in TestGridMixClasses to avoid
+    occassional failures. (Mit Desai via vinodkv)
+
   OPTIMIZATIONS
 
   BUG FIXES 
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordCount.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordCount.java
index 48c02fe..8634130 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordCount.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/WordCount.java
@@ -68,8 +68,8 @@
   public static void main(String[] args) throws Exception {
     Configuration conf = new Configuration();
     String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
-    if (otherArgs.length != 2) {
-      System.err.println("Usage: wordcount <in> <out>");
+    if (otherArgs.length < 2) {
+      System.err.println("Usage: wordcount <in> [<in>...] <out>");
       System.exit(2);
     }
     Job job = new Job(conf, "word count");
@@ -79,8 +79,11 @@
     job.setReducerClass(IntSumReducer.class);
     job.setOutputKeyClass(Text.class);
     job.setOutputValueClass(IntWritable.class);
-    FileInputFormat.addInputPath(job, new Path(otherArgs[0]));
-    FileOutputFormat.setOutputPath(job, new Path(otherArgs[1]));
+    for (int i = 0; i < otherArgs.length - 1; ++i) {
+      FileInputFormat.addInputPath(job, new Path(otherArgs[i]));
+    }
+    FileOutputFormat.setOutputPath(job,
+      new Path(otherArgs[otherArgs.length - 1]));
     System.exit(job.waitForCompletion(true) ? 0 : 1);
   }
 }
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index d8bfaa2..f359a4d 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -65,6 +65,8 @@
     <!-- define the protobuf JAR version                               -->
     <protobuf.version>2.5.0</protobuf.version>
     <protoc.path>${env.HADOOP_PROTOC_PATH}</protoc.path>
+
+    <zookeeper.version>3.4.6</zookeeper.version>
   </properties>
 
   <dependencyManagement>
@@ -307,6 +309,12 @@
         <artifactId>hadoop-openstack</artifactId>
         <version>${project.version}</version>
       </dependency>
+      
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-azure</artifactId>
+        <version>${project.version}</version>
+      </dependency>
 
       <dependency>
         <groupId>com.google.guava</groupId>
@@ -672,7 +680,7 @@
       <dependency>
         <groupId>org.apache.zookeeper</groupId>
         <artifactId>zookeeper</artifactId>
-        <version>3.4.5</version>
+        <version>${zookeeper.version}</version>
         <exclusions>
           <exclusion>
             <!-- otherwise seems to drag in junit 3.8.1 via jline -->
@@ -696,7 +704,7 @@
       <dependency>
         <groupId>org.apache.zookeeper</groupId>
         <artifactId>zookeeper</artifactId>
-        <version>3.4.5</version>
+        <version>${zookeeper.version}</version>
         <type>test-jar</type>
         <scope>test</scope>
         <exclusions>
@@ -766,6 +774,12 @@
         <version>1.8</version>
       </dependency>
 
+      <dependency>
+        <groupId>com.microsoft.windowsazure.storage</groupId>
+        <artifactId>microsoft-windowsazure-storage-sdk</artifactId>
+        <version>0.6.0</version>
+    </dependency>
+      
     </dependencies>
   </dependencyManagement>
 
diff --git a/hadoop-tools/hadoop-azure/.gitignore b/hadoop-tools/hadoop-azure/.gitignore
new file mode 100644
index 0000000..09c10b1
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/.gitignore
@@ -0,0 +1 @@
+.checkstyle
\ No newline at end of file
diff --git a/hadoop-tools/hadoop-azure/README.txt b/hadoop-tools/hadoop-azure/README.txt
new file mode 100644
index 0000000..4a06747
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/README.txt
@@ -0,0 +1,104 @@
+=============
+Building
+=============
+basic compilation:
+> mvn clean compile test-compile
+
+Compile, run tests and produce jar 
+> mvn clean package
+
+=============
+Unit tests
+=============
+Most of the tests will run without additional configuration.
+For complete testing, configuration in src/test/resources is required:
+  src/test/resources/azure-test.xml
+  src/test/resources/log4j.properties
+
+From command-line
+------------------
+Basic execution:
+> mvn test
+
+NOTES:
+ - The mvn pom.xml includes src/test/resources in the runtime classpath
+ - detailed output (such as log4j) appears in target\surefire-reports\TEST-{testName}.xml
+   including log4j messages.
+   
+Run the tests and generate report:
+> mvn site (at least once to setup some basics including images for the report)
+> mvn surefire-report:report  (run and produce report)
+> mvn mvn surefire-report:report-only  (produce report from last run)
+> mvn mvn surefire-report:report-only -DshowSuccess=false (produce report from last run, only show errors)
+> .\target\site\surefire-report.html (view the report)
+
+Via eclipse
+-------------
+Manually add src\test\resources to the classpath for test run configuration:
+  - run menu|run configurations|{configuration}|classpath|User Entries|advanced|add folder
+
+Then run via junit test runner.
+NOTE:
+ - if you change log4.properties, rebuild the project to refresh the eclipse cache.
+
+Run Tests against Mocked storage.
+---------------------------------
+These run automatically and make use of an in-memory emulation of azure storage.
+
+
+Running tests against the Azure storage emulator  
+---------------------------------------------------
+A selection of tests can run against the Azure Storage Emulator 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
+See http://msdn.microsoft.com/en-us/library/azure/hh403989.aspx
+
+Enable the Azure emulator tests by setting 
+  fs.azure.test.emulator -> true 
+in src\test\resources\azure-test.xml
+
+Running tests against live Azure storage 
+-------------------------------------------------------------------------
+In order to run WASB unit tests against a live Azure Storage account, add credentials to 
+src\test\resources\azure-test.xml. These settings augment the hadoop configuration object.
+
+For live tests, set the following in azure-test.xml:
+ 1. "fs.azure.test.account.name -> {azureStorageAccountName} 
+ 2. "fs.azure.account.key.{AccountName} -> {fullStorageKey}"
+
+=============
+Findbugs
+=============
+Run findbugs and show interactive GUI for review of problems
+> mvn findbugs:gui 
+
+Run findbugs and fail build if errors are found:
+> mvn findbugs:check
+
+For help with findbugs plugin.
+> mvn findbugs:help
+
+=============
+Checkstyle
+=============
+Rules for checkstyle @ src\config\checkstyle.xml
+ - these are based on a core set of standards, with exclusions for non-serious issues
+ - as a general plan it would be good to turn on more rules over time.
+ - Occasionally, run checkstyle with the default Sun rules by editing pom.xml.
+
+Command-line:
+> mvn checkstyle:check --> just test & fail build if violations found
+> mvn site checkstyle:checkstyle --> produce html report
+> . target\site\checkstyle.html  --> view report.
+
+Eclipse:
+- add the checkstyle plugin: Help|Install, site=http://eclipse-cs.sf.net/update
+- window|preferences|checkstyle. Add src/config/checkstyle.xml. Set as default.
+- project|properties|create configurations as required, eg src/main/java -> src/config/checkstyle.xml
+
+NOTE:
+- After any change to the checkstyle rules xml, use window|preferences|checkstyle|{refresh}|OK
+
+ 
\ No newline at end of file
diff --git a/hadoop-tools/hadoop-azure/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-azure/dev-support/findbugs-exclude.xml
new file mode 100644
index 0000000..cc63141
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/dev-support/findbugs-exclude.xml
@@ -0,0 +1,19 @@
+<!--
+   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.
+-->
+<FindBugsFilter>
+     
+ </FindBugsFilter>
diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml
new file mode 100644
index 0000000..582c07c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/pom.xml
@@ -0,0 +1,202 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>3.0.0-SNAPSHOT</version>
+    <relativePath>../../hadoop-project</relativePath>
+  </parent>
+  <artifactId>hadoop-azure</artifactId>
+  <name>Apache Hadoop Azure support</name>
+  <description>
+    This module contains code to support integration with Azure.
+    Currently this consists of a filesystem client to read data from
+    and write data to Azure Storage.
+  </description>
+  <packaging>jar</packaging>
+
+  <properties>
+    <file.encoding>UTF-8</file.encoding>
+    <downloadSources>true</downloadSources>
+  </properties>
+
+  <build>
+  
+    <testResources>
+      <testResource>
+        <directory>src/test/resources</directory>
+        <includes>
+          <include>log4j.properties</include>
+        </includes>
+      </testResource>
+      <testResource>
+        <directory>src/test/resources</directory>
+        <includes>
+          <include>azure-test.xml</include>
+        </includes>
+      </testResource>
+    </testResources>
+  
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <findbugsXmlOutput>true</findbugsXmlOutput>
+          <xmlOutput>true</xmlOutput>
+          <excludeFilterFile>${basedir}/dev-support/findbugs-exclude.xml
+          </excludeFilterFile>
+          <effort>Max</effort>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-site-plugin</artifactId>
+        <dependencies>
+          <dependency>
+            <groupId>org.apache.maven.doxia</groupId>
+            <artifactId>doxia-module-markdown</artifactId>
+            <version>1.3</version>
+          </dependency>
+        </dependencies>
+        <configuration>
+          <inputEncoding>UTF-8</inputEncoding>
+          <outputEncoding>UTF-8</outputEncoding>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-project-info-reports-plugin</artifactId>
+
+        <configuration>
+          <dependencyDetailsEnabled>false</dependencyDetailsEnabled>
+          <dependencyLocationsEnabled>false</dependencyLocationsEnabled>
+        </configuration>
+      </plugin>
+      
+      <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-checkstyle-plugin</artifactId>
+          <configuration>
+            <!-- To run with the default Sun ruleset, 
+                 comment out the configLocation line -->
+            <configLocation>src/config/checkstyle.xml</configLocation>
+          </configuration>
+          
+      </plugin>
+    </plugins>
+
+    
+    <!--
+       The following is to suppress a m2e warning in eclipse
+       (m2e doesn't know how to handle maven-enforcer:enforce, so we have to tell m2e to ignore it)
+       see: http://stackoverflow.com/questions/13040788/how-to-elimate-the-maven-enforcer-plugin-goal-enforce-is-ignored-by-m2e-wa
+    -->
+    <pluginManagement>
+      <plugins>
+        <plugin>
+          <groupId>org.eclipse.m2e</groupId>
+          <artifactId>lifecycle-mapping</artifactId>
+          <version>1.0.0</version>
+          <configuration>
+            <lifecycleMappingMetadata>
+              <pluginExecutions>
+                <pluginExecution>
+                  <pluginExecutionFilter>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-enforcer-plugin</artifactId>
+                    <versionRange>[1.0.0,)</versionRange>
+                    <goals>
+                      <goal>enforce</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <ignore />
+                  </action>
+                </pluginExecution>
+              </pluginExecutions>
+            </lifecycleMappingMetadata>
+          </configuration>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+
+  </build>
+
+  <!-- see hadoop-project/pom.xml for version number declarations -->
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-core-asl</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpclient</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.microsoft.windowsazure.storage</groupId>
+      <artifactId>microsoft-windowsazure-storage-sdk</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    
+    
+    <!-- dependencies use for test only -->
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    
+  </dependencies>
+</project>
diff --git a/hadoop-tools/hadoop-azure/src/config/checkstyle.xml b/hadoop-tools/hadoop-azure/src/config/checkstyle.xml
new file mode 100644
index 0000000..3bfc23d
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/config/checkstyle.xml
@@ -0,0 +1,184 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!DOCTYPE module PUBLIC
+    "-//Puppy Crawl//DTD Check Configuration 1.2//EN"
+    "http://www.puppycrawl.com/dtds/configuration_1_2.dtd">
+
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<!--
+  Checkstyle configuration that checks the sun coding conventions from:
+
+    - the Java Language Specification at
+      http://java.sun.com/docs/books/jls/second_edition/html/index.html
+
+    - the Sun Code Conventions at http://java.sun.com/docs/codeconv/
+
+    - the Javadoc guidelines at
+      http://java.sun.com/j2se/javadoc/writingdoccomments/index.html
+
+    - the JDK Api documentation http://java.sun.com/j2se/docs/api/index.html
+
+    - some best practices
+
+  Checkstyle is very configurable. Be sure to read the documentation at
+  http://checkstyle.sf.net (or in your downloaded distribution).
+
+  Most Checks are configurable, be sure to consult the documentation.
+  To completely disable a check, just comment it out or delete it from the file.
+  Finally, it is worth reading the documentation.
+-->
+
+<module name="Checker">
+    <!-- Checks that each Java package has a Javadoc file used for commenting. -->
+    <!-- See http://checkstyle.sf.net/config_javadoc.html#JavadocPackage       -->
+    <module name="JavadocPackage">
+      <property name="allowLegacy" value="true"/>
+    </module>
+
+    <!-- Checks that property files contain the same keys.         -->
+    <!-- See http://checkstyle.sf.net/config_misc.html#Translation -->
+    <module name="Translation"/>
+
+    <module name="FileLength">
+        <property name="max" value="3000"/>
+    </module>
+
+    <module name="FileTabCharacter">
+        <property name="eachLine" value="true"/>
+    </module>
+
+    <module name="TreeWalker">
+
+        <property name="cacheFile" value="${checkstyle.cache.file}"/>
+
+        <!-- Checks for Javadoc comments.                     -->
+        <!-- See http://checkstyle.sf.net/config_javadoc.html -->
+
+        <module name="JavadocType">
+          <property name="scope" value="public"/>
+        </module>  
+
+        <!-- 
+        <module name="JavadocMethod"/>
+        <module name="JavadocVariable"/>
+        <module name="JavadocStyle"/>
+        -->
+
+
+        <!-- Checks for Naming Conventions.                  -->
+        <!-- See http://checkstyle.sf.net/config_naming.html -->
+        <module name="ConstantName"/>
+        <module name="LocalFinalVariableName"/>
+        <module name="LocalVariableName"/>
+        <module name="MemberName"/>
+        <module name="MethodName"/>
+        <module name="PackageName"/>
+        <module name="ParameterName"/>
+        <module name="StaticVariableName"/>
+        <module name="TypeName"/>
+
+        <!-- Checks for imports                              -->
+        <!-- See http://checkstyle.sf.net/config_import.html -->
+        <module name="AvoidStarImport"/>
+        <module name="IllegalImport"/> <!-- defaults to sun.* packages -->
+        <module name="RedundantImport"/>
+        <module name="UnusedImports"/>
+
+
+        <!-- Checks for Size Violations.                    -->
+        <!-- See http://checkstyle.sf.net/config_sizes.html -->
+
+        <module name="LineLength">
+          <property name="max" value="160" />
+        </module>
+        
+        <module name="MethodLength">
+          <property name="max" value="3000"/>
+        </module>
+        
+        <module name="ParameterNumber"/>
+
+
+        <!-- Checks for whitespace                               -->
+        <!-- See http://checkstyle.sf.net/config_whitespace.html -->
+        
+        <module name="EmptyForIteratorPad"/>
+        <module name="MethodParamPad"/>
+        <!-- module name="NoWhitespaceAfter"/> -->
+        <module name="NoWhitespaceBefore"/>
+        <module name="OperatorWrap"/>
+        <module name="ParenPad"/>
+        <module name="TypecastParenPad"/>
+        <module name="WhitespaceAfter"/>
+        <!-- <module name="WhitespaceAround"/> -->
+
+
+        <!-- Modifier Checks                                    -->
+        <!-- See http://checkstyle.sf.net/config_modifiers.html -->
+        <module name="ModifierOrder"/>
+        <module name="RedundantModifier"/>
+
+
+        <!-- Checks for blocks. You know, those {}'s         -->
+        <!-- See http://checkstyle.sf.net/config_blocks.html -->
+        <module name="AvoidNestedBlocks"/>
+        <!-- <module name="EmptyBlock"/> -->
+        <module name="LeftCurly"/>
+        <module name="NeedBraces"/>
+
+        <!-- <module name="RightCurly"/> -->
+
+
+        <!-- Checks for common coding problems               -->
+        <!-- See http://checkstyle.sf.net/config_coding.html -->
+        <!-- <module name="AvoidInlineConditionals"/> -->
+        <!-- <module name="DoubleCheckedLocking"/> -->
+        <module name="EmptyStatement"/>
+        <module name="EqualsHashCode"/>
+        <!-- <module name="HiddenField"/> -->
+
+        <module name="IllegalInstantiation"/>
+        <module name="InnerAssignment"/>
+        <module name="MagicNumber">
+            <property name="ignoreNumbers" value="-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 10, 1000"/>
+        </module>
+        <module name="MissingSwitchDefault"/>
+        <module name="RedundantThrows"/>
+        <module name="SimplifyBooleanExpression"/>
+        <module name="SimplifyBooleanReturn"/>
+
+        <!-- Checks for class design                         -->
+        <!-- See http://checkstyle.sf.net/config_design.html -->
+
+        <!-- <module name="DesignForExtension"/> -->
+
+        <module name="FinalClass"/>
+        <module name="HideUtilityClassConstructor"/>
+        <module name="InterfaceIsType"/>
+        <module name="VisibilityModifier"/>
+
+
+        <!-- Miscellaneous other checks.                   -->
+        <!-- See http://checkstyle.sf.net/config_misc.html -->
+        <module name="ArrayTypeStyle"/>
+
+        <!-- <module name="FinalParameters"/> -->
+
+        <module name="TodoComment"/>
+        <module name="UpperEll"/>
+
+    </module>
+
+</module>
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureException.java
new file mode 100644
index 0000000..48ec064
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureException.java
@@ -0,0 +1,44 @@
+/**
+ * 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 org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown if there is a problem communicating with Azure Storage service.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class AzureException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public AzureException(String message) {
+    super(message);
+  }
+
+  public AzureException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public AzureException(Throwable t) {
+    super(t);
+  }
+}
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
new file mode 100644
index 0000000..9dcaddc
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
@@ -0,0 +1,2222 @@
+/**
+ * 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.NativeAzureFileSystem.PATH_DELIMITER;
+
+import java.io.BufferedInputStream;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.security.InvalidKeyException;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.azure.StorageInterface.CloudBlobContainerWrapper;
+import org.apache.hadoop.fs.azure.StorageInterface.CloudBlobDirectoryWrapper;
+import org.apache.hadoop.fs.azure.StorageInterface.CloudBlockBlobWrapper;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.mortbay.util.ajax.JSON;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.microsoft.windowsazure.storage.CloudStorageAccount;
+import com.microsoft.windowsazure.storage.OperationContext;
+import com.microsoft.windowsazure.storage.RetryExponentialRetry;
+import com.microsoft.windowsazure.storage.RetryNoRetry;
+import com.microsoft.windowsazure.storage.StorageCredentials;
+import com.microsoft.windowsazure.storage.StorageCredentialsAccountAndKey;
+import com.microsoft.windowsazure.storage.StorageCredentialsSharedAccessSignature;
+import com.microsoft.windowsazure.storage.StorageErrorCode;
+import com.microsoft.windowsazure.storage.StorageException;
+import com.microsoft.windowsazure.storage.blob.BlobListingDetails;
+import com.microsoft.windowsazure.storage.blob.BlobProperties;
+import com.microsoft.windowsazure.storage.blob.BlobRequestOptions;
+import com.microsoft.windowsazure.storage.blob.CloudBlob;
+import com.microsoft.windowsazure.storage.blob.CopyStatus;
+import com.microsoft.windowsazure.storage.blob.DeleteSnapshotsOption;
+import com.microsoft.windowsazure.storage.blob.ListBlobItem;
+import com.microsoft.windowsazure.storage.core.Utility;
+
+@InterfaceAudience.Private
+class AzureNativeFileSystemStore implements NativeFileSystemStore {
+  
+  /**
+   * Configuration knob on whether we do block-level MD5 validation on
+   * upload/download.
+   */
+  static final String KEY_CHECK_BLOCK_MD5 = "fs.azure.check.block.md5";
+  /**
+   * Configuration knob on whether we store blob-level MD5 on upload.
+   */
+  static final String KEY_STORE_BLOB_MD5 = "fs.azure.store.blob.md5";
+  static final String DEFAULT_STORAGE_EMULATOR_ACCOUNT_NAME = "storageemulator";
+  static final String STORAGE_EMULATOR_ACCOUNT_NAME_PROPERTY_NAME = "fs.azure.storage.emulator.account.name";
+
+  public static final Log LOG = LogFactory
+      .getLog(AzureNativeFileSystemStore.class);
+
+  private StorageInterface storageInteractionLayer;
+  private CloudBlobDirectoryWrapper rootDirectory;
+  private CloudBlobContainerWrapper container;
+
+  // Constants local to this class.
+  //
+  private static final String KEY_ACCOUNT_KEYPROVIDER_PREFIX = "fs.azure.account.keyprovider.";
+  private static final String KEY_ACCOUNT_SAS_PREFIX = "fs.azure.sas.";
+
+  // note: this value is not present in core-default.xml as our real default is
+  // computed as min(2*cpu,8)
+  private static final String KEY_CONCURRENT_CONNECTION_VALUE_OUT = "fs.azure.concurrentRequestCount.out";
+
+  private static final String KEY_STREAM_MIN_READ_SIZE = "fs.azure.read.request.size";
+  private static final String KEY_STORAGE_CONNECTION_TIMEOUT = "fs.azure.storage.timeout";
+  private static final String KEY_WRITE_BLOCK_SIZE = "fs.azure.write.request.size";
+
+  // Property controlling whether to allow reads on blob which are concurrently
+  // appended out-of-band.
+  private static final String KEY_READ_TOLERATE_CONCURRENT_APPEND = "fs.azure.io.read.tolerate.concurrent.append";
+
+  // Configurable throttling parameter properties. These properties are located
+  // in the core-site.xml configuration file.
+  private static final String KEY_MIN_BACKOFF_INTERVAL = "fs.azure.io.retry.min.backoff.interval";
+  private static final String KEY_MAX_BACKOFF_INTERVAL = "fs.azure.io.retry.max.backoff.interval";
+  private static final String KEY_BACKOFF_INTERVAL = "fs.azure.io.retry.backoff.interval";
+  private static final String KEY_MAX_IO_RETRIES = "fs.azure.io.retry.max.retries";
+
+  private static final String KEY_SELF_THROTTLE_ENABLE = "fs.azure.selfthrottling.enable";
+  private static final String KEY_SELF_THROTTLE_READ_FACTOR = "fs.azure.selfthrottling.read.factor";
+  private static final String KEY_SELF_THROTTLE_WRITE_FACTOR = "fs.azure.selfthrottling.write.factor";
+
+  private static final String PERMISSION_METADATA_KEY = "hdi_permission";
+  private static final String OLD_PERMISSION_METADATA_KEY = "asv_permission";
+  private static final String IS_FOLDER_METADATA_KEY = "hdi_isfolder";
+  private static final String OLD_IS_FOLDER_METADATA_KEY = "asv_isfolder";
+  static final String VERSION_METADATA_KEY = "hdi_version";
+  static final String OLD_VERSION_METADATA_KEY = "asv_version";
+  static final String FIRST_WASB_VERSION = "2013-01-01";
+  static final String CURRENT_WASB_VERSION = "2013-09-01";
+  static final String LINK_BACK_TO_UPLOAD_IN_PROGRESS_METADATA_KEY = "hdi_tmpupload";
+  static final String OLD_LINK_BACK_TO_UPLOAD_IN_PROGRESS_METADATA_KEY = "asv_tmpupload";
+
+  private static final String HTTP_SCHEME = "http";
+  private static final String HTTPS_SCHEME = "https";
+  private static final String WASB_AUTHORITY_DELIMITER = "@";
+  private static final String AZURE_ROOT_CONTAINER = "$root";
+
+  private static final int DEFAULT_CONCURRENT_WRITES = 8;
+
+  // Concurrent reads reads of data written out of band are disable by default.
+  private static final boolean DEFAULT_READ_TOLERATE_CONCURRENT_APPEND = false;
+
+  // Default block sizes
+  public static final int DEFAULT_DOWNLOAD_BLOCK_SIZE = 4 * 1024 * 1024;
+  public static final int DEFAULT_UPLOAD_BLOCK_SIZE = 4 * 1024 * 1024;
+
+  // Retry parameter defaults.
+  private static final int DEFAULT_MIN_BACKOFF_INTERVAL = 1 * 1000; // 1s
+  private static final int DEFAULT_MAX_BACKOFF_INTERVAL = 30 * 1000; // 30s
+  private static final int DEFAULT_BACKOFF_INTERVAL = 1 * 1000; // 1s
+  private static final int DEFAULT_MAX_RETRY_ATTEMPTS = 15;
+
+  // Self-throttling defaults. Allowed range = (0,1.0]
+  // Value of 1.0 means no self-throttling.
+  // Value of x means process data at factor x of unrestricted rate
+  private static final boolean DEFAULT_SELF_THROTTLE_ENABLE = true;
+  private static final float DEFAULT_SELF_THROTTLE_READ_FACTOR = 1.0f;
+  private static final float DEFAULT_SELF_THROTTLE_WRITE_FACTOR = 1.0f;
+
+  private static final int STORAGE_CONNECTION_TIMEOUT_DEFAULT = 90;
+
+  /**
+   * MEMBER VARIABLES
+   */
+
+  private URI sessionUri;
+  private Configuration sessionConfiguration;
+  private int concurrentWrites = DEFAULT_CONCURRENT_WRITES;
+  private boolean isAnonymousCredentials = false;
+  // Set to true if we are connecting using shared access signatures.
+  private boolean connectingUsingSAS = false;
+  private static final JSON PERMISSION_JSON_SERIALIZER = createPermissionJsonSerializer();
+
+  private boolean suppressRetryPolicy = false;
+  private boolean canCreateOrModifyContainer = false;
+  private ContainerState currentKnownContainerState = ContainerState.Unknown;
+  private final Object containerStateLock = new Object();
+
+  private boolean tolerateOobAppends = DEFAULT_READ_TOLERATE_CONCURRENT_APPEND;
+
+  private int downloadBlockSizeBytes = DEFAULT_DOWNLOAD_BLOCK_SIZE;
+  private int uploadBlockSizeBytes = DEFAULT_UPLOAD_BLOCK_SIZE;
+
+  // Bandwidth throttling exponential back-off parameters
+  //
+  private int minBackoff; // the minimum back-off interval (ms) between retries.
+  private int maxBackoff; // the maximum back-off interval (ms) between retries.
+  private int deltaBackoff; // the back-off interval (ms) between retries.
+  private int maxRetries; // the maximum number of retry attempts.
+
+  // Self-throttling parameters
+  private boolean selfThrottlingEnabled;
+  private float selfThrottlingReadFactor;
+  private float selfThrottlingWriteFactor;
+
+  private TestHookOperationContext testHookOperationContext = null;
+
+  // Set if we're running against a storage emulator..
+  private boolean isStorageEmulator = false;
+
+  /**
+   * A test hook interface that can modify the operation context we use for
+   * Azure Storage operations, e.g. to inject errors.
+   */
+  @VisibleForTesting 
+  interface TestHookOperationContext {
+    OperationContext modifyOperationContext(OperationContext original);
+  }
+
+  /**
+   * Suppress the default retry policy for the Storage, useful in unit tests to
+   * test negative cases without waiting forever.
+   */
+  @VisibleForTesting
+  void suppressRetryPolicy() {
+    suppressRetryPolicy = true;
+  }
+
+  /**
+   * Add a test hook to modify the operation context we use for Azure Storage
+   * operations.
+   * 
+   * @param testHook
+   *          The test hook, or null to unset previous hooks.
+   */
+  @VisibleForTesting 
+  void addTestHookToOperationContext(TestHookOperationContext testHook) {
+    this.testHookOperationContext = testHook;
+  }
+
+  /**
+   * If we're asked by unit tests to not retry, set the retry policy factory in
+   * the client accordingly.
+   */
+  private void suppressRetryPolicyInClientIfNeeded() {
+    if (suppressRetryPolicy) {
+      storageInteractionLayer.setRetryPolicyFactory(new RetryNoRetry());
+    }
+  }
+
+  /**
+   * Creates a JSON serializer that can serialize a PermissionStatus object into
+   * the JSON string we want in the blob metadata.
+   * 
+   * @return The JSON serializer.
+   */
+  private static JSON createPermissionJsonSerializer() {
+    JSON serializer = new JSON();
+    serializer.addConvertor(PermissionStatus.class,
+        new PermissionStatusJsonSerializer());
+    return serializer;
+  }
+
+  /**
+   * A converter for PermissionStatus to/from JSON as we want it in the blob
+   * metadata.
+   */
+  private static class PermissionStatusJsonSerializer implements JSON.Convertor {
+    private static final String OWNER_TAG = "owner";
+    private static final String GROUP_TAG = "group";
+    private static final String PERMISSIONS_TAG = "permissions";
+
+    @Override
+    public void toJSON(Object obj, JSON.Output out) {
+      PermissionStatus permissionStatus = (PermissionStatus) obj;
+      // Don't store group as null, just store it as empty string
+      // (which is FileStatus behavior).
+      String group = permissionStatus.getGroupName() == null ? ""
+          : permissionStatus.getGroupName();
+      out.add(OWNER_TAG, permissionStatus.getUserName());
+      out.add(GROUP_TAG, group);
+      out.add(PERMISSIONS_TAG, permissionStatus.getPermission().toString());
+    }
+
+    @Override
+    public Object fromJSON(@SuppressWarnings("rawtypes") Map object) {
+      return PermissionStatusJsonSerializer.fromJSONMap(object);
+    }
+
+    @SuppressWarnings("rawtypes")
+    public static PermissionStatus fromJSONString(String jsonString) {
+      // The JSON class can only find out about an object's class (and call me)
+      // if we store the class name in the JSON string. Since I don't want to
+      // do that (it's an implementation detail), I just deserialize as a
+      // the default Map (JSON's default behavior) and parse that.
+      return fromJSONMap((Map) PERMISSION_JSON_SERIALIZER.fromJSON(jsonString));
+    }
+
+    private static PermissionStatus fromJSONMap(
+        @SuppressWarnings("rawtypes") Map object) {
+      return new PermissionStatus((String) object.get(OWNER_TAG),
+          (String) object.get(GROUP_TAG),
+          // The initial - below is the Unix file type,
+          // which FsPermission needs there but ignores.
+          FsPermission.valueOf("-" + (String) object.get(PERMISSIONS_TAG)));
+    }
+  }
+
+  @VisibleForTesting
+  void setAzureStorageInteractionLayer(StorageInterface storageInteractionLayer) {
+    this.storageInteractionLayer = storageInteractionLayer;
+  }
+
+  /**
+   * Check if concurrent reads and writes on the same blob are allowed.
+   * 
+   * @return true if concurrent reads and OOB writes has been configured, false
+   *         otherwise.
+   */
+  private boolean isConcurrentOOBAppendAllowed() {
+    return tolerateOobAppends;
+  }
+
+  /**
+   * Method for the URI and configuration object necessary to create a storage
+   * session with an Azure session. It parses the scheme to ensure it matches
+   * the storage protocol supported by this file system.
+   * 
+   * @param uri
+   *          - URI for target storage blob.
+   * @param conf
+   *          - reference to configuration object.
+   * 
+   * @throws IllegalArgumentException
+   *           if URI or job object is null, or invalid scheme.
+   */
+  @Override
+  public void initialize(URI uri, Configuration conf) throws AzureException {
+
+    if (null == this.storageInteractionLayer) {
+      this.storageInteractionLayer = new StorageInterfaceImpl();
+    }
+
+    // Check that URI exists.
+    //
+    if (null == uri) {
+      throw new IllegalArgumentException(
+          "Cannot initialize WASB file system, URI is null");
+    }
+
+    // Check that configuration object is non-null.
+    //
+    if (null == conf) {
+      throw new IllegalArgumentException(
+          "Cannot initialize WASB file system, URI is null");
+    }
+
+    // Incoming parameters validated. Capture the URI and the job configuration
+    // object.
+    //
+    sessionUri = uri;
+    sessionConfiguration = conf;
+
+    // Start an Azure storage session.
+    //
+    createAzureStorageSession();
+  }
+
+  /**
+   * Method to extract the account name from an Azure URI.
+   * 
+   * @param uri
+   *          -- WASB blob URI
+   * @returns accountName -- the account name for the URI.
+   * @throws URISyntaxException
+   *           if the URI does not have an authority it is badly formed.
+   */
+  private String getAccountFromAuthority(URI uri) throws URISyntaxException {
+
+    // Check to make sure that the authority is valid for the URI.
+    //
+    String authority = uri.getRawAuthority();
+    if (null == authority) {
+      // Badly formed or illegal URI.
+      //
+      throw new URISyntaxException(uri.toString(),
+          "Expected URI with a valid authority");
+    }
+
+    // Check if authority container the delimiter separating the account name
+    // from the
+    // the container.
+    //
+    if (!authority.contains(WASB_AUTHORITY_DELIMITER)) {
+      return authority;
+    }
+
+    // Split off the container name and the authority.
+    //
+    String[] authorityParts = authority.split(WASB_AUTHORITY_DELIMITER, 2);
+
+    // Because the string contains an '@' delimiter, a container must be
+    // specified.
+    //
+    if (authorityParts.length < 2 || "".equals(authorityParts[0])) {
+      // Badly formed WASB authority since there is no container.
+      //
+      final String errMsg = String
+          .format(
+              "URI '%s' has a malformed WASB authority, expected container name. "
+                  + "Authority takes the form wasb://[<container name>@]<account name>",
+              uri.toString());
+      throw new IllegalArgumentException(errMsg);
+    }
+
+    // Return with the account name. It is possible that this name is NULL.
+    //
+    return authorityParts[1];
+  }
+
+  /**
+   * Method to extract the container name from an Azure URI.
+   * 
+   * @param uri
+   *          -- WASB blob URI
+   * @returns containerName -- the container name for the URI. May be null.
+   * @throws URISyntaxException
+   *           if the uri does not have an authority it is badly formed.
+   */
+  private String getContainerFromAuthority(URI uri) throws URISyntaxException {
+
+    // Check to make sure that the authority is valid for the URI.
+    //
+    String authority = uri.getRawAuthority();
+    if (null == authority) {
+      // Badly formed or illegal URI.
+      //
+      throw new URISyntaxException(uri.toString(),
+          "Expected URI with a valid authority");
+    }
+
+    // The URI has a valid authority. Extract the container name. It is the
+    // second component of the WASB URI authority.
+    if (!authority.contains(WASB_AUTHORITY_DELIMITER)) {
+      // The authority does not have a container name. Use the default container
+      // by
+      // setting the container name to the default Azure root container.
+      //
+      return AZURE_ROOT_CONTAINER;
+    }
+
+    // Split off the container name and the authority.
+    String[] authorityParts = authority.split(WASB_AUTHORITY_DELIMITER, 2);
+
+    // Because the string contains an '@' delimiter, a container must be
+    // specified.
+    if (authorityParts.length < 2 || "".equals(authorityParts[0])) {
+      // Badly formed WASB authority since there is no container.
+      final String errMsg = String
+          .format(
+              "URI '%s' has a malformed WASB authority, expected container name."
+                  + "Authority takes the form wasb://[<container name>@]<account name>",
+              uri.toString());
+      throw new IllegalArgumentException(errMsg);
+    }
+
+    // Set the container name from the first entry for the split parts of the
+    // authority.
+    return authorityParts[0];
+  }
+
+  /**
+   * Get the appropriate return the appropriate scheme for communicating with
+   * Azure depending on whether wasb or wasbs is specified in the target URI.
+   * 
+   * return scheme - HTTPS or HTTP as appropriate.
+   */
+  private String getHTTPScheme() {
+    String sessionScheme = sessionUri.getScheme();
+    // Check if we're on a secure URI scheme: wasbs or the legacy asvs scheme.
+    if (sessionScheme != null
+        && (sessionScheme.equalsIgnoreCase("asvs") || sessionScheme
+            .equalsIgnoreCase("wasbs"))) {
+      return HTTPS_SCHEME;
+    } else {
+      // At this point the scheme should be either null or asv or wasb.
+      // Intentionally I'm not going to validate it though since I don't feel
+      // it's this method's job to ensure a valid URI scheme for this file
+      // system.
+      return HTTP_SCHEME;
+    }
+  }
+
+  /**
+   * Set the configuration parameters for this client storage session with
+   * Azure.
+   * 
+   * @throws AzureException
+   * @throws ConfigurationException
+   * 
+   */
+  private void configureAzureStorageSession() throws AzureException {
+
+    // Assertion: Target session URI already should have been captured.
+    if (sessionUri == null) {
+      throw new AssertionError(
+          "Expected a non-null session URI when configuring storage session");
+    }
+
+    // Assertion: A client session already should have been established with
+    // Azure.
+    if (storageInteractionLayer == null) {
+      throw new AssertionError(String.format(
+          "Cannot configure storage session for URI '%s' "
+              + "if storage session has not been established.",
+          sessionUri.toString()));
+    }
+
+    // Determine whether or not reads are allowed concurrent with OOB writes.
+    tolerateOobAppends = sessionConfiguration.getBoolean(
+        KEY_READ_TOLERATE_CONCURRENT_APPEND,
+        DEFAULT_READ_TOLERATE_CONCURRENT_APPEND);
+
+    // Retrieve configuration for the minimum stream read and write block size.
+    //
+    this.downloadBlockSizeBytes = sessionConfiguration.getInt(
+        KEY_STREAM_MIN_READ_SIZE, DEFAULT_DOWNLOAD_BLOCK_SIZE);
+    this.uploadBlockSizeBytes = sessionConfiguration.getInt(
+        KEY_WRITE_BLOCK_SIZE, DEFAULT_UPLOAD_BLOCK_SIZE);
+
+    // The job may want to specify a timeout to use when engaging the
+    // storage service. The default is currently 90 seconds. It may
+    // be necessary to increase this value for long latencies in larger
+    // jobs. If the timeout specified is greater than zero seconds use
+    // it, otherwise use the default service client timeout.
+    int storageConnectionTimeout = sessionConfiguration.getInt(
+        KEY_STORAGE_CONNECTION_TIMEOUT, 0);
+
+    if (0 < storageConnectionTimeout) {
+      storageInteractionLayer.setTimeoutInMs(storageConnectionTimeout * 1000);
+    }
+
+    // Set the concurrency values equal to the that specified in the
+    // configuration file. If it does not exist, set it to the default
+    // value calculated as double the number of CPU cores on the client
+    // machine. The concurrency value is minimum of double the cores and
+    // the read/write property.
+    int cpuCores = 2 * Runtime.getRuntime().availableProcessors();
+
+    concurrentWrites = sessionConfiguration.getInt(
+        KEY_CONCURRENT_CONNECTION_VALUE_OUT,
+        Math.min(cpuCores, DEFAULT_CONCURRENT_WRITES));
+
+    // Set up the exponential retry policy.
+    minBackoff = sessionConfiguration.getInt(KEY_MIN_BACKOFF_INTERVAL,
+        DEFAULT_MIN_BACKOFF_INTERVAL);
+
+    maxBackoff = sessionConfiguration.getInt(KEY_MAX_BACKOFF_INTERVAL,
+        DEFAULT_MAX_BACKOFF_INTERVAL);
+
+    deltaBackoff = sessionConfiguration.getInt(KEY_BACKOFF_INTERVAL,
+        DEFAULT_BACKOFF_INTERVAL);
+
+    maxRetries = sessionConfiguration.getInt(KEY_MAX_IO_RETRIES,
+        DEFAULT_MAX_RETRY_ATTEMPTS);
+
+    storageInteractionLayer.setRetryPolicyFactory(new RetryExponentialRetry(
+        minBackoff, deltaBackoff, maxBackoff, maxRetries));
+
+    // read the self-throttling config.
+    selfThrottlingEnabled = sessionConfiguration.getBoolean(
+        KEY_SELF_THROTTLE_ENABLE, DEFAULT_SELF_THROTTLE_ENABLE);
+
+    selfThrottlingReadFactor = sessionConfiguration.getFloat(
+        KEY_SELF_THROTTLE_READ_FACTOR, DEFAULT_SELF_THROTTLE_READ_FACTOR);
+
+    selfThrottlingWriteFactor = sessionConfiguration.getFloat(
+        KEY_SELF_THROTTLE_WRITE_FACTOR, DEFAULT_SELF_THROTTLE_WRITE_FACTOR);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(String
+          .format(
+              "AzureNativeFileSystemStore init. Settings=%d,%b,%d,{%d,%d,%d,%d},{%b,%f,%f}",
+              concurrentWrites, tolerateOobAppends,
+              ((storageConnectionTimeout > 0) ? storageConnectionTimeout
+                  : STORAGE_CONNECTION_TIMEOUT_DEFAULT), minBackoff,
+              deltaBackoff, maxBackoff, maxRetries, selfThrottlingEnabled,
+              selfThrottlingReadFactor, selfThrottlingWriteFactor));
+    }
+  }
+
+  /**
+   * Connect to Azure storage using anonymous credentials.
+   * 
+   * @param uri
+   *          - URI to target blob (R/O access to public blob)
+   * 
+   * @throws StorageException
+   *           raised on errors communicating with Azure storage.
+   * @throws IOException
+   *           raised on errors performing I/O or setting up the session.
+   * @throws URISyntaxExceptions
+   *           raised on creating mal-formed URI's.
+   */
+  private void connectUsingAnonymousCredentials(final URI uri)
+      throws StorageException, IOException, URISyntaxException {
+    // Use an HTTP scheme since the URI specifies a publicly accessible
+    // container. Explicitly create a storage URI corresponding to the URI
+    // parameter for use in creating the service client.
+    String accountName = getAccountFromAuthority(uri);
+    URI storageUri = new URI(getHTTPScheme() + ":" + PATH_DELIMITER
+        + PATH_DELIMITER + accountName);
+
+    // Create the service client with anonymous credentials.
+    String containerName = getContainerFromAuthority(uri);
+    storageInteractionLayer.createBlobClient(storageUri);
+    suppressRetryPolicyInClientIfNeeded();
+
+    // Capture the container reference.
+    container = storageInteractionLayer.getContainerReference(containerName);
+    rootDirectory = container.getDirectoryReference("");
+
+    // Check for container existence, and our ability to access it.
+    try {
+      if (!container.exists(getInstrumentedContext())) {
+        throw new AzureException("Container " + containerName + " in account "
+            + accountName + " not found, and we can't create "
+            + " it using anoynomous credentials.");
+      }
+    } catch (StorageException ex) {
+      throw new AzureException("Unable to access container " + containerName
+          + " in account " + accountName
+          + " using anonymous credentials, and no credentials found for them "
+          + " in the configuration.", ex);
+    }
+
+    // Accessing the storage server unauthenticated using
+    // anonymous credentials.
+    isAnonymousCredentials = true;
+
+    // Configure Azure storage session.
+    configureAzureStorageSession();
+  }
+
+  private void connectUsingCredentials(String accountName,
+      StorageCredentials credentials, String containerName)
+      throws URISyntaxException, StorageException, AzureException {
+
+    if (isStorageEmulatorAccount(accountName)) {
+      isStorageEmulator = true;
+      CloudStorageAccount account = CloudStorageAccount
+          .getDevelopmentStorageAccount();
+      storageInteractionLayer.createBlobClient(account);
+    } else {
+      URI blobEndPoint = new URI(getHTTPScheme() + "://" + accountName);
+      storageInteractionLayer.createBlobClient(blobEndPoint, credentials);
+    }
+    suppressRetryPolicyInClientIfNeeded();
+
+    // Capture the container reference for debugging purposes.
+    container = storageInteractionLayer.getContainerReference(containerName);
+    rootDirectory = container.getDirectoryReference("");
+
+    // Can only create container if using account key credentials
+    canCreateOrModifyContainer = credentials instanceof StorageCredentialsAccountAndKey;
+
+    // Configure Azure storage session.
+    configureAzureStorageSession();
+  }
+
+  /**
+   * Connect to Azure storage using account key credentials.
+   */
+  private void connectUsingConnectionStringCredentials(
+      final String accountName, final String containerName,
+      final String accountKey) throws InvalidKeyException, StorageException,
+      IOException, URISyntaxException {
+    // If the account name is "acc.blob.core.windows.net", then the
+    // rawAccountName is just "acc"
+    String rawAccountName = accountName.split("\\.")[0];
+    StorageCredentials credentials = new StorageCredentialsAccountAndKey(
+        rawAccountName, accountKey);
+    connectUsingCredentials(accountName, credentials, containerName);
+  }
+
+  /**
+   * Connect to Azure storage using shared access signature credentials.
+   */
+  private void connectUsingSASCredentials(final String accountName,
+      final String containerName, final String sas) throws InvalidKeyException,
+      StorageException, IOException, URISyntaxException {
+    StorageCredentials credentials = new StorageCredentialsSharedAccessSignature(
+        sas);
+    connectingUsingSAS = true;
+    connectUsingCredentials(accountName, credentials, containerName);
+  }
+
+  private boolean isStorageEmulatorAccount(final String accountName) {
+    return accountName.equalsIgnoreCase(sessionConfiguration.get(
+        STORAGE_EMULATOR_ACCOUNT_NAME_PROPERTY_NAME,
+        DEFAULT_STORAGE_EMULATOR_ACCOUNT_NAME));
+  }
+
+  static String getAccountKeyFromConfiguration(String accountName,
+      Configuration conf) throws KeyProviderException {
+    String key = null;
+    String keyProviderClass = conf.get(KEY_ACCOUNT_KEYPROVIDER_PREFIX
+        + accountName);
+    KeyProvider keyProvider = null;
+
+    if (keyProviderClass == null) {
+      // No key provider was provided so use the provided key as is.
+      keyProvider = new SimpleKeyProvider();
+    } else {
+      // create an instance of the key provider class and verify it
+      // implements KeyProvider
+      Object keyProviderObject = null;
+      try {
+        Class<?> clazz = conf.getClassByName(keyProviderClass);
+        keyProviderObject = clazz.newInstance();
+      } catch (Exception e) {
+        throw new KeyProviderException("Unable to load key provider class.", e);
+      }
+      if (!(keyProviderObject instanceof KeyProvider)) {
+        throw new KeyProviderException(keyProviderClass
+            + " specified in config is not a valid KeyProvider class.");
+      }
+      keyProvider = (KeyProvider) keyProviderObject;
+    }
+    key = keyProvider.getStorageAccountKey(accountName, conf);
+
+    return key;
+  }
+
+  /**
+   * Establish a session with Azure blob storage based on the target URI. The
+   * method determines whether or not the URI target contains an explicit
+   * account or an implicit default cluster-wide account.
+   * 
+   * @throws AzureException
+   * @throws IOException
+   */
+  private void createAzureStorageSession() throws AzureException {
+
+    // Make sure this object was properly initialized with references to
+    // the sessionUri and sessionConfiguration.
+    if (null == sessionUri || null == sessionConfiguration) {
+      throw new AzureException("Filesystem object not initialized properly."
+          + "Unable to start session with Azure Storage server.");
+    }
+
+    // File system object initialized, attempt to establish a session
+    // with the Azure storage service for the target URI string.
+    try {
+      // Inspect the URI authority to determine the account and use the account
+      // to start an Azure blob client session using an account key for the
+      // the account or anonymously.
+      // For all URI's do the following checks in order:
+      // 1. Validate that <account> can be used with the current Hadoop
+      // cluster by checking it exists in the list of configured accounts
+      // for the cluster.
+      // 2. Look up the AccountKey in the list of configured accounts for the
+      // cluster.
+      // 3. If there is no AccountKey, assume anonymous public blob access
+      // when accessing the blob.
+      //
+      // If the URI does not specify a container use the default root container
+      // under the account name.
+
+      // Assertion: Container name on the session Uri should be non-null.
+      if (getContainerFromAuthority(sessionUri) == null) {
+        throw new AssertionError(String.format(
+            "Non-null container expected from session URI: %s.",
+            sessionUri.toString()));
+      }
+
+      // Get the account name.
+      String accountName = getAccountFromAuthority(sessionUri);
+      if (null == accountName) {
+        // Account name is not specified as part of the URI. Throw indicating
+        // an invalid account name.
+        final String errMsg = String.format(
+            "Cannot load WASB file system account name not"
+                + " specified in URI: %s.", sessionUri.toString());
+        throw new AzureException(errMsg);
+      }
+
+      String containerName = getContainerFromAuthority(sessionUri);
+
+      // Check whether this is a storage emulator account.
+      if (isStorageEmulatorAccount(accountName)) {
+        // It is an emulator account, connect to it with no credentials.
+        connectUsingCredentials(accountName, null, containerName);
+        return;
+      }
+
+      // Check whether we have a shared access signature for that container.
+      String propertyValue = sessionConfiguration.get(KEY_ACCOUNT_SAS_PREFIX
+          + containerName + "." + accountName);
+      if (propertyValue != null) {
+        // SAS was found. Connect using that.
+        connectUsingSASCredentials(accountName, containerName, propertyValue);
+        return;
+      }
+
+      // Check whether the account is configured with an account key.
+      propertyValue = getAccountKeyFromConfiguration(accountName,
+          sessionConfiguration);
+      if (propertyValue != null) {
+
+        // Account key was found.
+        // Create the Azure storage session using the account key and container.
+        connectUsingConnectionStringCredentials(
+            getAccountFromAuthority(sessionUri),
+            getContainerFromAuthority(sessionUri), propertyValue);
+
+        // Return to caller
+        return;
+      }
+
+      // The account access is not configured for this cluster. Try anonymous
+      // access.
+      connectUsingAnonymousCredentials(sessionUri);
+
+    } catch (Exception e) {
+      // Caught exception while attempting to initialize the Azure File
+      // System store, re-throw the exception.
+      throw new AzureException(e);
+    }
+  }
+
+  private enum ContainerState {
+    /**
+     * We haven't checked the container state yet.
+     */
+    Unknown,
+    /**
+     * We checked and the container doesn't exist.
+     */
+    DoesntExist,
+    /**
+     * The container exists and doesn't have an WASB version stamp on it.
+     */
+    ExistsNoVersion,
+    /**
+     * The container exists and has an unsupported WASB version stamped on it.
+     */
+    ExistsAtWrongVersion,
+    /**
+     * The container exists and has the proper WASB version stamped on it.
+     */
+    ExistsAtRightVersion
+  }
+
+  private enum ContainerAccessType {
+    /**
+     * We're accessing the container for a pure read operation, e.g. read a
+     * file.
+     */
+    PureRead,
+    /**
+     * We're accessing the container purely to write something, e.g. write a
+     * file.
+     */
+    PureWrite,
+    /**
+     * We're accessing the container to read something then write, e.g. rename a
+     * file.
+     */
+    ReadThenWrite
+  }
+
+  /**
+   * This should be called from any method that does any modifications to the
+   * underlying container: it makes sure to put the WASB current version in the
+   * container's metadata if it's not already there.
+   */
+  private ContainerState checkContainer(ContainerAccessType accessType)
+      throws StorageException, AzureException {
+    synchronized (containerStateLock) {
+      if (isOkContainerState(accessType)) {
+        return currentKnownContainerState;
+      }
+      if (currentKnownContainerState == ContainerState.ExistsAtWrongVersion) {
+        String containerVersion = retrieveVersionAttribute(container);
+        throw wrongVersionException(containerVersion);
+      }
+      // This means I didn't check it before or it didn't exist or
+      // we need to stamp the version. Since things may have changed by
+      // other machines since then, do the check again and don't depend
+      // on past information.
+
+      // Sanity check: we don't expect this at this point.
+      if (currentKnownContainerState == ContainerState.ExistsAtRightVersion) {
+        throw new AssertionError("Unexpected state: "
+            + currentKnownContainerState);
+      }
+
+      // Download the attributes - doubles as an existence check with just
+      // one service call
+      try {
+        container.downloadAttributes(getInstrumentedContext());
+        currentKnownContainerState = ContainerState.Unknown;
+      } catch (StorageException ex) {
+        if (ex.getErrorCode().equals(
+            StorageErrorCode.RESOURCE_NOT_FOUND.toString())) {
+          currentKnownContainerState = ContainerState.DoesntExist;
+        } else {
+          throw ex;
+        }
+      }
+
+      if (currentKnownContainerState == ContainerState.DoesntExist) {
+        // If the container doesn't exist and we intend to write to it,
+        // create it now.
+        if (needToCreateContainer(accessType)) {
+          storeVersionAttribute(container);
+          container.create(getInstrumentedContext());
+          currentKnownContainerState = ContainerState.ExistsAtRightVersion;
+        }
+      } else {
+        // The container exists, check the version.
+        String containerVersion = retrieveVersionAttribute(container);
+        if (containerVersion != null) {
+          if (containerVersion.equals(FIRST_WASB_VERSION)) {
+            // It's the version from when WASB was called ASV, just
+            // fix the version attribute if needed and proceed.
+            // We should be good otherwise.
+            if (needToStampVersion(accessType)) {
+              storeVersionAttribute(container);
+              container.uploadMetadata(getInstrumentedContext());
+            }
+          } else if (!containerVersion.equals(CURRENT_WASB_VERSION)) {
+            // Don't know this version - throw.
+            currentKnownContainerState = ContainerState.ExistsAtWrongVersion;
+            throw wrongVersionException(containerVersion);
+          } else {
+            // It's our correct version.
+            currentKnownContainerState = ContainerState.ExistsAtRightVersion;
+          }
+        } else {
+          // No version info exists.
+          currentKnownContainerState = ContainerState.ExistsNoVersion;
+          if (needToStampVersion(accessType)) {
+            // Need to stamp the version
+            storeVersionAttribute(container);
+            container.uploadMetadata(getInstrumentedContext());
+            currentKnownContainerState = ContainerState.ExistsAtRightVersion;
+          }
+        }
+      }
+      return currentKnownContainerState;
+    }
+  }
+
+  private AzureException wrongVersionException(String containerVersion) {
+    return new AzureException("The container " + container.getName()
+        + " is at an unsupported version: " + containerVersion
+        + ". Current supported version: " + FIRST_WASB_VERSION);
+  }
+
+  private boolean needToStampVersion(ContainerAccessType accessType) {
+    // We need to stamp the version on the container any time we write to
+    // it and we have the correct credentials to be able to write container
+    // metadata.
+    return accessType != ContainerAccessType.PureRead
+        && canCreateOrModifyContainer;
+  }
+
+  private static boolean needToCreateContainer(ContainerAccessType accessType) {
+    // We need to pro-actively create the container (if it doesn't exist) if
+    // we're doing a pure write. No need to create it for pure read or read-
+    // then-write access.
+    return accessType == ContainerAccessType.PureWrite;
+  }
+
+  // Determines whether we have to pull the container information again
+  // or we can work based off what we already have.
+  private boolean isOkContainerState(ContainerAccessType accessType) {
+    switch (currentKnownContainerState) {
+    case Unknown:
+      // When using SAS, we can't discover container attributes
+      // so just live with Unknown state and fail later if it
+      // doesn't exist.
+      return connectingUsingSAS;
+    case DoesntExist:
+      return false; // the container could have been created
+    case ExistsAtRightVersion:
+      return true; // fine to optimize
+    case ExistsAtWrongVersion:
+      return false;
+    case ExistsNoVersion:
+      // If there's no version, it's OK if we don't need to stamp the version
+      // or we can't anyway even if we wanted to.
+      return !needToStampVersion(accessType);
+    default:
+      throw new AssertionError("Unknown access type: " + accessType);
+    }
+  }
+
+  private boolean getUseTransactionalContentMD5() {
+    return sessionConfiguration.getBoolean(KEY_CHECK_BLOCK_MD5, true);
+  }
+
+  private BlobRequestOptions getUploadOptions() {
+    BlobRequestOptions options = new BlobRequestOptions();
+    options.setStoreBlobContentMD5(sessionConfiguration.getBoolean(
+        KEY_STORE_BLOB_MD5, false));
+    options.setUseTransactionalContentMD5(getUseTransactionalContentMD5());
+    options.setConcurrentRequestCount(concurrentWrites);
+
+    options.setRetryPolicyFactory(new RetryExponentialRetry(minBackoff,
+        deltaBackoff, maxBackoff, maxRetries));
+
+    return options;
+  }
+
+  private BlobRequestOptions getDownloadOptions() {
+    BlobRequestOptions options = new BlobRequestOptions();
+    options.setRetryPolicyFactory(new RetryExponentialRetry(minBackoff,
+        deltaBackoff, maxBackoff, maxRetries));
+    options.setUseTransactionalContentMD5(getUseTransactionalContentMD5());
+    return options;
+  }
+
+  @Override
+  public DataOutputStream storefile(String key,
+      PermissionStatus permissionStatus) throws AzureException {
+    try {
+
+      // Check if a session exists, if not create a session with the
+      // Azure storage server.
+      if (null == storageInteractionLayer) {
+        final String errMsg = String.format(
+            "Storage session expected for URI '%s' but does not exist.",
+            sessionUri);
+        throw new AzureException(errMsg);
+      }
+
+      // Check if there is an authenticated account associated with the
+      // file this instance of the WASB file system. If not the file system
+      // has not been authenticated and all access is anonymous.
+      if (!isAuthenticatedAccess()) {
+        // Preemptively raise an exception indicating no uploads are
+        // allowed to anonymous accounts.
+        throw new AzureException(new IOException(
+            "Uploads to public accounts using anonymous "
+                + "access is prohibited."));
+      }
+
+      checkContainer(ContainerAccessType.PureWrite);
+
+      /**
+       * Note: Windows Azure Blob Storage does not allow the creation of
+       * arbitrary directory paths under the default $root directory. This is by
+       * design to eliminate ambiguity in specifying a implicit blob address. A
+       * blob in the $root container cannot include a / in its name and must be
+       * careful not to include a trailing '/' when referencing blobs in the
+       * $root container. A '/; in the $root container permits ambiguous blob
+       * names as in the following example involving two containers $root and
+       * mycontainer: http://myaccount.blob.core.windows.net/$root
+       * http://myaccount.blob.core.windows.net/mycontainer If the URL
+       * "mycontainer/somefile.txt were allowed in $root then the URL:
+       * http://myaccount.blob.core.windows.net/mycontainer/myblob.txt could
+       * mean either: (1) container=mycontainer; blob=myblob.txt (2)
+       * container=$root; blob=mycontainer/myblob.txt
+       * 
+       * To avoid this type of ambiguity the Azure blob storage prevents
+       * arbitrary path under $root. For a simple and more consistent user
+       * experience it was decided to eliminate the opportunity for creating
+       * such paths by making the $root container read-only under WASB. 
+       */
+
+      // Check that no attempt is made to write to blobs on default
+      // $root containers.
+      if (AZURE_ROOT_CONTAINER.equals(getContainerFromAuthority(sessionUri))) {
+        // Azure containers are restricted to non-root containers.
+        final String errMsg = String.format(
+            "Writes to '%s' container for URI '%s' are prohibited, "
+                + "only updates on non-root containers permitted.",
+            AZURE_ROOT_CONTAINER, sessionUri.toString());
+        throw new AzureException(errMsg);
+      }
+
+      // Get the block blob reference from the store's container and
+      // return it.
+      CloudBlockBlobWrapper blob = getBlobReference(key);
+      storePermissionStatus(blob, permissionStatus);
+
+      // Create the output stream for the Azure blob.
+      OutputStream outputStream = blob.openOutputStream(getUploadOptions(),
+          getInstrumentedContext());
+
+      // Return to caller with DataOutput stream.
+      DataOutputStream dataOutStream = new DataOutputStream(outputStream);
+      return dataOutStream;
+    } catch (Exception e) {
+      // Caught exception while attempting to open the blob output stream.
+      // Re-throw as an Azure storage exception.
+      throw new AzureException(e);
+    }
+  }
+
+  /**
+   * Default permission to use when no permission metadata is found.
+   * 
+   * @return The default permission to use.
+   */
+  private static PermissionStatus defaultPermissionNoBlobMetadata() {
+    return new PermissionStatus("", "", FsPermission.getDefault());
+  }
+
+  private static void storeMetadataAttribute(CloudBlockBlobWrapper blob,
+      String key, String value) {
+    HashMap<String, String> metadata = blob.getMetadata();
+    if (null == metadata) {
+      metadata = new HashMap<String, String>();
+    }
+    metadata.put(key, value);
+    blob.setMetadata(metadata);
+  }
+
+  private static String getMetadataAttribute(CloudBlockBlobWrapper blob,
+      String... keyAlternatives) {
+    HashMap<String, String> metadata = blob.getMetadata();
+    if (null == metadata) {
+      return null;
+    }
+    for (String key : keyAlternatives) {
+      if (metadata.containsKey(key)) {
+        return metadata.get(key);
+      }
+    }
+    return null;
+  }
+
+  private static void removeMetadataAttribute(CloudBlockBlobWrapper blob,
+      String key) {
+    HashMap<String, String> metadata = blob.getMetadata();
+    if (metadata != null) {
+      metadata.remove(key);
+      blob.setMetadata(metadata);
+    }
+  }
+
+  private void storePermissionStatus(CloudBlockBlobWrapper blob,
+      PermissionStatus permissionStatus) {
+    storeMetadataAttribute(blob, PERMISSION_METADATA_KEY,
+        PERMISSION_JSON_SERIALIZER.toJSON(permissionStatus));
+    // Remove the old metadata key if present
+    removeMetadataAttribute(blob, OLD_PERMISSION_METADATA_KEY);
+  }
+
+  private PermissionStatus getPermissionStatus(CloudBlockBlobWrapper blob) {
+    String permissionMetadataValue = getMetadataAttribute(blob,
+        PERMISSION_METADATA_KEY, OLD_PERMISSION_METADATA_KEY);
+    if (permissionMetadataValue != null) {
+      return PermissionStatusJsonSerializer
+          .fromJSONString(permissionMetadataValue);
+    } else {
+      return defaultPermissionNoBlobMetadata();
+    }
+  }
+
+  private static void storeFolderAttribute(CloudBlockBlobWrapper blob) {
+    storeMetadataAttribute(blob, IS_FOLDER_METADATA_KEY, "true");
+    // Remove the old metadata key if present
+    removeMetadataAttribute(blob, OLD_IS_FOLDER_METADATA_KEY);
+  }
+
+  private static void storeLinkAttribute(CloudBlockBlobWrapper blob,
+      String linkTarget) {
+    storeMetadataAttribute(blob, LINK_BACK_TO_UPLOAD_IN_PROGRESS_METADATA_KEY,
+        linkTarget);
+    // Remove the old metadata key if present
+    removeMetadataAttribute(blob,
+        OLD_LINK_BACK_TO_UPLOAD_IN_PROGRESS_METADATA_KEY);
+  }
+
+  private static String getLinkAttributeValue(CloudBlockBlobWrapper blob) {
+    return getMetadataAttribute(blob,
+        LINK_BACK_TO_UPLOAD_IN_PROGRESS_METADATA_KEY,
+        OLD_LINK_BACK_TO_UPLOAD_IN_PROGRESS_METADATA_KEY);
+  }
+
+  private static boolean retrieveFolderAttribute(CloudBlockBlobWrapper blob) {
+    HashMap<String, String> metadata = blob.getMetadata();
+    return null != metadata
+        && (metadata.containsKey(IS_FOLDER_METADATA_KEY) || metadata
+            .containsKey(OLD_IS_FOLDER_METADATA_KEY));
+  }
+
+  private static void storeVersionAttribute(CloudBlobContainerWrapper container) {
+    HashMap<String, String> metadata = container.getMetadata();
+    if (null == metadata) {
+      metadata = new HashMap<String, String>();
+    }
+    metadata.put(VERSION_METADATA_KEY, CURRENT_WASB_VERSION);
+    if (metadata.containsKey(OLD_VERSION_METADATA_KEY)) {
+      metadata.remove(OLD_VERSION_METADATA_KEY);
+    }
+    container.setMetadata(metadata);
+  }
+
+  private static String retrieveVersionAttribute(
+      CloudBlobContainerWrapper container) {
+    HashMap<String, String> metadata = container.getMetadata();
+    if (metadata == null) {
+      return null;
+    } else if (metadata.containsKey(VERSION_METADATA_KEY)) {
+      return metadata.get(VERSION_METADATA_KEY);
+    } else if (metadata.containsKey(OLD_VERSION_METADATA_KEY)) {
+      return metadata.get(OLD_VERSION_METADATA_KEY);
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public void storeEmptyFolder(String key, PermissionStatus permissionStatus)
+      throws AzureException {
+
+    if (null == storageInteractionLayer) {
+      final String errMsg = String.format(
+          "Storage session expected for URI '%s' but does not exist.",
+          sessionUri);
+      throw new AssertionError(errMsg);
+    }
+
+    // Check if there is an authenticated account associated with the file
+    // this instance of the WASB file system. If not the file system has not
+    // been authenticated and all access is anonymous.
+    if (!isAuthenticatedAccess()) {
+      // Preemptively raise an exception indicating no uploads are
+      // allowed to anonymous accounts.
+      throw new AzureException(
+          "Uploads to to public accounts using anonymous access is prohibited.");
+    }
+
+    try {
+      checkContainer(ContainerAccessType.PureWrite);
+
+      CloudBlockBlobWrapper blob = getBlobReference(key);
+      storePermissionStatus(blob, permissionStatus);
+      storeFolderAttribute(blob);
+      blob.upload(new ByteArrayInputStream(new byte[0]),
+          getInstrumentedContext());
+    } catch (Exception e) {
+      // Caught exception while attempting upload. Re-throw as an Azure
+      // storage exception.
+      throw new AzureException(e);
+    }
+  }
+
+  /**
+   * Stores an empty blob that's linking to the temporary file where're we're
+   * uploading the initial data.
+   */
+  @Override
+  public void storeEmptyLinkFile(String key, String tempBlobKey,
+      PermissionStatus permissionStatus) throws AzureException {
+    if (null == storageInteractionLayer) {
+      final String errMsg = String.format(
+          "Storage session expected for URI '%s' but does not exist.",
+          sessionUri);
+      throw new AssertionError(errMsg);
+    }
+    // Check if there is an authenticated account associated with the file
+    // this instance of the WASB file system. If not the file system has not
+    // been authenticated and all access is anonymous.
+    if (!isAuthenticatedAccess()) {
+      // Preemptively raise an exception indicating no uploads are
+      // allowed to anonymous accounts.
+      throw new AzureException(
+          "Uploads to to public accounts using anonymous access is prohibited.");
+    }
+
+    try {
+      checkContainer(ContainerAccessType.PureWrite);
+
+      CloudBlockBlobWrapper blob = getBlobReference(key);
+      storePermissionStatus(blob, permissionStatus);
+      storeLinkAttribute(blob, tempBlobKey);
+      blob.upload(new ByteArrayInputStream(new byte[0]),
+          getInstrumentedContext());
+    } catch (Exception e) {
+      // Caught exception while attempting upload. Re-throw as an Azure
+      // storage exception.
+      throw new AzureException(e);
+    }
+  }
+
+  /**
+   * If the blob with the given key exists and has a link in its metadata to a
+   * temporary file (see storeEmptyLinkFile), this method returns the key to
+   * that temporary file. Otherwise, returns null.
+   */
+  @Override
+  public String getLinkInFileMetadata(String key) throws AzureException {
+    if (null == storageInteractionLayer) {
+      final String errMsg = String.format(
+          "Storage session expected for URI '%s' but does not exist.",
+          sessionUri);
+      throw new AssertionError(errMsg);
+    }
+
+    try {
+      checkContainer(ContainerAccessType.PureRead);
+
+      CloudBlockBlobWrapper blob = getBlobReference(key);
+      blob.downloadAttributes(getInstrumentedContext());
+      return getLinkAttributeValue(blob);
+    } catch (Exception e) {
+      // Caught exception while attempting download. Re-throw as an Azure
+      // storage exception.
+      throw new AzureException(e);
+    }
+  }
+
+  /**
+   * Private method to check for authenticated access.
+   * 
+   * @ returns boolean -- true if access is credentialed and authenticated and
+   * false otherwise.
+   */
+  private boolean isAuthenticatedAccess() throws AzureException {
+
+    if (isAnonymousCredentials) {
+      // Access to this storage account is unauthenticated.
+      return false;
+    }
+    // Access is authenticated.
+    return true;
+  }
+
+  /**
+   * This private method uses the root directory or the original container to
+   * list blobs under the directory or container depending on whether the
+   * original file system object was constructed with a short- or long-form URI.
+   * If the root directory is non-null the URI in the file constructor was in
+   * the long form.
+   * 
+   * @param includeMetadata
+   *          if set, the listed items will have their metadata populated
+   *          already.
+   * 
+   * @returns blobItems : iterable collection of blob items.
+   * @throws URISyntaxException
+   * 
+   */
+  private Iterable<ListBlobItem> listRootBlobs(boolean includeMetadata)
+      throws StorageException, URISyntaxException {
+    return rootDirectory.listBlobs(
+        null,
+        false,
+        includeMetadata ? EnumSet.of(BlobListingDetails.METADATA) : EnumSet
+            .noneOf(BlobListingDetails.class), null, getInstrumentedContext());
+  }
+
+  /**
+   * This private method uses the root directory or the original container to
+   * list blobs under the directory or container given a specified prefix for
+   * the directory depending on whether the original file system object was
+   * constructed with a short- or long-form URI. If the root directory is
+   * non-null the URI in the file constructor was in the long form.
+   * 
+   * @param aPrefix
+   *          : string name representing the prefix of containing blobs.
+   * @param includeMetadata
+   *          if set, the listed items will have their metadata populated
+   *          already.
+   * 
+   * @returns blobItems : iterable collection of blob items.
+   * @throws URISyntaxException
+   * 
+   */
+  private Iterable<ListBlobItem> listRootBlobs(String aPrefix,
+      boolean includeMetadata) throws StorageException, URISyntaxException {
+
+    return rootDirectory.listBlobs(
+        aPrefix,
+        false,
+        includeMetadata ? EnumSet.of(BlobListingDetails.METADATA) : EnumSet
+            .noneOf(BlobListingDetails.class), null, getInstrumentedContext());
+  }
+
+  /**
+   * This private method uses the root directory or the original container to
+   * list blobs under the directory or container given a specified prefix for
+   * the directory depending on whether the original file system object was
+   * constructed with a short- or long-form URI. It also uses the specified flat
+   * or hierarchical option, listing details options, request options, and
+   * operation context.
+   * 
+   * @param aPrefix
+   *          string name representing the prefix of containing blobs.
+   * @param useFlatBlobListing
+   *          - the list is flat if true, or hierarchical otherwise.
+   * @param listingDetails
+   *          - determine whether snapshots, metadata, committed/uncommitted
+   *          data
+   * @param options
+   *          - object specifying additional options for the request. null =
+   *          default options
+   * @param opContext
+   *          - context of the current operation
+   * @returns blobItems : iterable collection of blob items.
+   * @throws URISyntaxException
+   * 
+   */
+  private Iterable<ListBlobItem> listRootBlobs(String aPrefix,
+      boolean useFlatBlobListing, EnumSet<BlobListingDetails> listingDetails,
+      BlobRequestOptions options, OperationContext opContext)
+      throws StorageException, URISyntaxException {
+
+    CloudBlobDirectoryWrapper directory = this.container
+        .getDirectoryReference(aPrefix);
+    return directory.listBlobs(null, useFlatBlobListing, listingDetails,
+        options, opContext);
+  }
+
+  /**
+   * This private method uses the root directory or the original container to
+   * get the block blob reference depending on whether the original file system
+   * object was constructed with a short- or long-form URI. If the root
+   * directory is non-null the URI in the file constructor was in the long form.
+   * 
+   * @param aKey
+   *          : a key used to query Azure for the block blob.
+   * @returns blob : a reference to the Azure block blob corresponding to the
+   *          key.
+   * @throws URISyntaxException
+   * 
+   */
+  private CloudBlockBlobWrapper getBlobReference(String aKey)
+      throws StorageException, URISyntaxException {
+
+    CloudBlockBlobWrapper blob = this.container.getBlockBlobReference(aKey);
+
+    blob.setStreamMinimumReadSizeInBytes(downloadBlockSizeBytes);
+    blob.setWriteBlockSizeInBytes(uploadBlockSizeBytes);
+
+    // Return with block blob.
+    return blob;
+  }
+
+  /**
+   * This private method normalizes the key by stripping the container name from
+   * the path and returns a path relative to the root directory of the
+   * container.
+   * 
+   * @param keyUri
+   *          - adjust this key to a path relative to the root directory
+   * 
+   * @returns normKey
+   */
+  private String normalizeKey(URI keyUri) {
+    String normKey;
+
+    // Strip the container name from the path and return the path
+    // relative to the root directory of the container.
+    int parts = isStorageEmulator ? 4 : 3;
+    normKey = keyUri.getPath().split("/", parts)[(parts - 1)];
+
+    // Return the fixed key.
+    return normKey;
+  }
+
+  /**
+   * This private method normalizes the key by stripping the container name from
+   * the path and returns a path relative to the root directory of the
+   * container.
+   * 
+   * @param blob
+   *          - adjust the key to this blob to a path relative to the root
+   *          directory
+   * 
+   * @returns normKey
+   */
+  private String normalizeKey(CloudBlockBlobWrapper blob) {
+    return normalizeKey(blob.getUri());
+  }
+
+  /**
+   * This private method normalizes the key by stripping the container name from
+   * the path and returns a path relative to the root directory of the
+   * container.
+   * 
+   * @param blob
+   *          - adjust the key to this directory to a path relative to the root
+   *          directory
+   * 
+   * @returns normKey
+   */
+  private String normalizeKey(CloudBlobDirectoryWrapper directory) {
+    String dirKey = normalizeKey(directory.getUri());
+    // Strip the last delimiter
+    if (dirKey.endsWith(PATH_DELIMITER)) {
+      dirKey = dirKey.substring(0, dirKey.length() - 1);
+    }
+    return dirKey;
+  }
+
+  /**
+   * Default method to creates a new OperationContext for the Azure Storage
+   * operation that has listeners hooked to it that will update the metrics for
+   * this file system. This method does not bind to receive send request
+   * callbacks by default.
+   * 
+   * @return The OperationContext object to use.
+   */
+  private OperationContext getInstrumentedContext() {
+    // Default is to not bind to receive send callback events.
+    return getInstrumentedContext(false);
+  }
+
+  /**
+   * Creates a new OperationContext for the Azure Storage operation that has
+   * listeners hooked to it that will update the metrics for this file system.
+   * 
+   * @param bindConcurrentOOBIo
+   *          - bind to intercept send request call backs to handle OOB I/O.
+   * 
+   * @return The OperationContext object to use.
+   */
+  private OperationContext getInstrumentedContext(boolean bindConcurrentOOBIo) {
+
+    OperationContext operationContext = new OperationContext();
+
+    if (selfThrottlingEnabled) {
+      SelfThrottlingIntercept.hook(operationContext, selfThrottlingReadFactor,
+          selfThrottlingWriteFactor);
+    }
+
+    // Bind operation context to receive send request callbacks on this
+    // operation.
+    // If reads concurrent to OOB writes are allowed, the interception will
+    // reset the conditional header on all Azure blob storage read requests.
+    if (bindConcurrentOOBIo) {
+      SendRequestIntercept.bind(storageInteractionLayer.getCredentials(),
+          operationContext, true);
+    }
+
+    if (testHookOperationContext != null) {
+      operationContext = testHookOperationContext
+          .modifyOperationContext(operationContext);
+    }
+
+    // Return the operation context.
+    return operationContext;
+  }
+
+  @Override
+  public FileMetadata retrieveMetadata(String key) throws IOException {
+
+    // Attempts to check status may occur before opening any streams so first,
+    // check if a session exists, if not create a session with the Azure storage
+    // server.
+    if (null == storageInteractionLayer) {
+      final String errMsg = String.format(
+          "Storage session expected for URI '%s' but does not exist.",
+          sessionUri);
+      throw new AssertionError(errMsg);
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrieving metadata for " + key);
+    }
+
+    try {
+      if (checkContainer(ContainerAccessType.PureRead) == ContainerState.DoesntExist) {
+        // The container doesn't exist, so spare some service calls and just
+        // return null now.
+        return null;
+      }
+
+      // Handle the degenerate cases where the key does not exist or the
+      // key is a container.
+      if (key.equals("/")) {
+        // The key refers to root directory of container.
+        // Set the modification time for root to zero.
+        return new FileMetadata(key, 0, defaultPermissionNoBlobMetadata(),
+            BlobMaterialization.Implicit);
+      }
+
+      CloudBlockBlobWrapper blob = getBlobReference(key);
+
+      // Download attributes and return file metadata only if the blob
+      // exists.
+      if (null != blob && blob.exists(getInstrumentedContext())) {
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Found " + key
+              + " as an explicit blob. Checking if it's a file or folder.");
+        }
+
+        // The blob exists, so capture the metadata from the blob
+        // properties.
+        blob.downloadAttributes(getInstrumentedContext());
+        BlobProperties properties = blob.getProperties();
+
+        if (retrieveFolderAttribute(blob)) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(key + " is a folder blob.");
+          }
+          return new FileMetadata(key, properties.getLastModified().getTime(),
+              getPermissionStatus(blob), BlobMaterialization.Explicit);
+        } else {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(key + " is a normal blob.");
+          }
+
+          return new FileMetadata(
+              key, // Always return denormalized key with metadata.
+              properties.getLength(), properties.getLastModified().getTime(),
+              getPermissionStatus(blob));
+        }
+      }
+
+      // There is no file with that key name, but maybe it is a folder.
+      // Query the underlying folder/container to list the blobs stored
+      // there under that key.
+      Iterable<ListBlobItem> objects = listRootBlobs(key, true,
+          EnumSet.of(BlobListingDetails.METADATA), null,
+          getInstrumentedContext());
+
+      // Check if the directory/container has the blob items.
+      for (ListBlobItem blobItem : objects) {
+        if (blobItem instanceof CloudBlockBlobWrapper) {
+          LOG.debug("Found blob as a directory-using this file under it to infer its properties "
+              + blobItem.getUri());
+
+          blob = (CloudBlockBlobWrapper) blobItem;
+          // The key specifies a directory. Create a FileMetadata object which
+          // specifies as such.
+          BlobProperties properties = blob.getProperties();
+
+          return new FileMetadata(key, properties.getLastModified().getTime(),
+              getPermissionStatus(blob), BlobMaterialization.Implicit);
+        }
+      }
+
+      // Return to caller with a null metadata object.
+      return null;
+
+    } catch (Exception e) {
+      // Re-throw the exception as an Azure storage exception.
+      throw new AzureException(e);
+    }
+  }
+
+  @Override
+  public DataInputStream retrieve(String key) throws AzureException {
+    InputStream inStream = null;
+    BufferedInputStream inBufStream = null;
+    try {
+      try {
+        // Check if a session exists, if not create a session with the
+        // Azure storage server.
+        if (null == storageInteractionLayer) {
+          final String errMsg = String.format(
+              "Storage session expected for URI '%s' but does not exist.",
+              sessionUri);
+          throw new AssertionError(errMsg);
+        }
+        checkContainer(ContainerAccessType.PureRead);
+
+        // Get blob reference and open the input buffer stream.
+        CloudBlockBlobWrapper blob = getBlobReference(key);
+        inStream = blob.openInputStream(getDownloadOptions(),
+            getInstrumentedContext(isConcurrentOOBAppendAllowed()));
+
+        inBufStream = new BufferedInputStream(inStream);
+
+        // Return a data input stream.
+        DataInputStream inDataStream = new DataInputStream(inBufStream);
+        return inDataStream;
+      }
+      catch (Exception e){
+        // close the streams on error.
+        // We use nested try-catch as stream.close() can throw IOException.
+        if(inBufStream != null){
+          inBufStream.close();
+        }
+        if(inStream != null){
+          inStream.close();
+        }
+        throw e;
+      }
+    } catch (Exception e) {
+      // Re-throw as an Azure storage exception.
+      throw new AzureException(e);
+    }
+  }
+
+  @Override
+  public DataInputStream retrieve(String key, long startByteOffset)
+      throws AzureException {
+
+    InputStream in = null;
+    DataInputStream inDataStream = null;
+    try {
+      try {
+        // Check if a session exists, if not create a session with the
+        // Azure storage server.
+        if (null == storageInteractionLayer) {
+          final String errMsg = String.format(
+              "Storage session expected for URI '%s' but does not exist.",
+              sessionUri);
+          throw new AssertionError(errMsg);
+        }
+        checkContainer(ContainerAccessType.PureRead);
+
+        // Get blob reference and open the input buffer stream.
+        CloudBlockBlobWrapper blob = getBlobReference(key);
+
+        // Open input stream and seek to the start offset.
+        in = blob.openInputStream(getDownloadOptions(),
+            getInstrumentedContext(isConcurrentOOBAppendAllowed()));
+
+        // Create a data input stream.
+        inDataStream = new DataInputStream(in);
+        long skippedBytes = inDataStream.skip(startByteOffset);
+        if (skippedBytes != startByteOffset) {
+          throw new IOException("Couldn't skip the requested number of bytes");
+        }
+        return inDataStream;
+      }
+      catch (Exception e){
+        // close the streams on error.
+        // We use nested try-catch as stream.close() can throw IOException.
+        if(inDataStream != null){
+          inDataStream.close();
+        }
+        if(in != null){
+          inDataStream.close();
+        }
+        throw e;
+      }
+    } catch (Exception e) {
+      // Re-throw as an Azure storage exception.
+      throw new AzureException(e);
+    }
+  }
+
+  @Override
+  public PartialListing list(String prefix, final int maxListingCount,
+      final int maxListingDepth) throws IOException {
+    return list(prefix, maxListingCount, maxListingDepth, null);
+  }
+
+  @Override
+  public PartialListing list(String prefix, final int maxListingCount,
+      final int maxListingDepth, String priorLastKey) throws IOException {
+    return list(prefix, PATH_DELIMITER, maxListingCount, maxListingDepth,
+        priorLastKey);
+  }
+
+  @Override
+  public PartialListing listAll(String prefix, final int maxListingCount,
+      final int maxListingDepth, String priorLastKey) throws IOException {
+    return list(prefix, null, maxListingCount, maxListingDepth, priorLastKey);
+  }
+
+  /**
+   * Searches the given list of {@link FileMetadata} objects for a directory
+   * with the given key.
+   * 
+   * @param list
+   *          The list to search.
+   * @param key
+   *          The key to search for.
+   * @return The wanted directory, or null if not found.
+   */
+  private static FileMetadata getDirectoryInList(
+      final Iterable<FileMetadata> list, String key) {
+    for (FileMetadata current : list) {
+      if (current.isDir() && current.getKey().equals(key)) {
+        return current;
+      }
+    }
+    return null;
+  }
+
+  private PartialListing list(String prefix, String delimiter,
+      final int maxListingCount, final int maxListingDepth, String priorLastKey)
+      throws IOException {
+    try {
+      checkContainer(ContainerAccessType.PureRead);
+
+      if (0 < prefix.length() && !prefix.endsWith(PATH_DELIMITER)) {
+        prefix += PATH_DELIMITER;
+      }
+
+      Iterable<ListBlobItem> objects;
+      if (prefix.equals("/")) {
+        objects = listRootBlobs(true);
+      } else {
+        objects = listRootBlobs(prefix, true);
+      }
+
+      ArrayList<FileMetadata> fileMetadata = new ArrayList<FileMetadata>();
+      for (ListBlobItem blobItem : objects) {
+        // Check that the maximum listing count is not exhausted.
+        //
+        if (0 < maxListingCount && fileMetadata.size() >= maxListingCount) {
+          break;
+        }
+
+        if (blobItem instanceof CloudBlockBlobWrapper) {
+          String blobKey = null;
+          CloudBlockBlobWrapper blob = (CloudBlockBlobWrapper) blobItem;
+          BlobProperties properties = blob.getProperties();
+
+          // Determine format of the blob name depending on whether an absolute
+          // path is being used or not.
+          blobKey = normalizeKey(blob);
+
+          FileMetadata metadata;
+          if (retrieveFolderAttribute(blob)) {
+            metadata = new FileMetadata(blobKey, properties.getLastModified()
+                .getTime(), getPermissionStatus(blob),
+                BlobMaterialization.Explicit);
+          } else {
+            metadata = new FileMetadata(blobKey, properties.getLength(),
+                properties.getLastModified().getTime(),
+                getPermissionStatus(blob));
+          }
+
+          // Add the metadata to the list, but remove any existing duplicate
+          // entries first that we may have added by finding nested files.
+          FileMetadata existing = getDirectoryInList(fileMetadata, blobKey);
+          if (existing != null) {
+            fileMetadata.remove(existing);
+          }
+          fileMetadata.add(metadata);
+        } else if (blobItem instanceof CloudBlobDirectoryWrapper) {
+          CloudBlobDirectoryWrapper directory = (CloudBlobDirectoryWrapper) blobItem;
+          // Determine format of directory name depending on whether an absolute
+          // path is being used or not.
+          //
+          String dirKey = normalizeKey(directory);
+          // Strip the last /
+          if (dirKey.endsWith(PATH_DELIMITER)) {
+            dirKey = dirKey.substring(0, dirKey.length() - 1);
+          }
+
+          // Reached the targeted listing depth. Return metadata for the
+          // directory using default permissions.
+          //
+          // Note: Something smarter should be done about permissions. Maybe
+          // inherit the permissions of the first non-directory blob.
+          // Also, getting a proper value for last-modified is tricky.
+          FileMetadata directoryMetadata = new FileMetadata(dirKey, 0,
+              defaultPermissionNoBlobMetadata(), BlobMaterialization.Implicit);
+
+          // Add the directory metadata to the list only if it's not already
+          // there.
+          if (getDirectoryInList(fileMetadata, dirKey) == null) {
+            fileMetadata.add(directoryMetadata);
+          }
+
+          // Currently at a depth of one, decrement the listing depth for
+          // sub-directories.
+          buildUpList(directory, fileMetadata, maxListingCount,
+              maxListingDepth - 1);
+        }
+      }
+      // Note: Original code indicated that this may be a hack.
+      priorLastKey = null;
+      return new PartialListing(priorLastKey,
+          fileMetadata.toArray(new FileMetadata[] {}),
+          0 == fileMetadata.size() ? new String[] {} : new String[] { prefix });
+    } catch (Exception e) {
+      // Re-throw as an Azure storage exception.
+      //
+      throw new AzureException(e);
+    }
+  }
+
+  /**
+   * Build up a metadata list of blobs in an Azure blob directory. This method
+   * uses a in-order first traversal of blob directory structures to maintain
+   * the sorted order of the blob names.
+   * 
+   * @param dir
+   *          -- Azure blob directory
+   * 
+   * @param list
+   *          -- a list of file metadata objects for each non-directory blob.
+   * 
+   * @param maxListingLength
+   *          -- maximum length of the built up list.
+   */
+  private void buildUpList(CloudBlobDirectoryWrapper aCloudBlobDirectory,
+      ArrayList<FileMetadata> aFileMetadataList, final int maxListingCount,
+      final int maxListingDepth) throws Exception {
+
+    // Push the blob directory onto the stack.
+    LinkedList<Iterator<ListBlobItem>> dirIteratorStack = new LinkedList<Iterator<ListBlobItem>>();
+
+    Iterable<ListBlobItem> blobItems = aCloudBlobDirectory.listBlobs(null,
+        false, EnumSet.of(BlobListingDetails.METADATA), null,
+        getInstrumentedContext());
+    Iterator<ListBlobItem> blobItemIterator = blobItems.iterator();
+
+    if (0 == maxListingDepth || 0 == maxListingCount) {
+      // Recurrence depth and listing count are already exhausted. Return
+      // immediately.
+      return;
+    }
+
+    // The directory listing depth is unbounded if the maximum listing depth
+    // is negative.
+    final boolean isUnboundedDepth = (maxListingDepth < 0);
+
+    // Reset the current directory listing depth.
+    int listingDepth = 1;
+
+    // Loop until all directories have been traversed in-order. Loop only
+    // the following conditions are satisfied:
+    // (1) The stack is not empty, and
+    // (2) maxListingCount > 0 implies that the number of items in the
+    // metadata list is less than the max listing count.
+    while (null != blobItemIterator
+        && (maxListingCount <= 0 || aFileMetadataList.size() < maxListingCount)) {
+      while (blobItemIterator.hasNext()) {
+        // Check if the count of items on the list exhausts the maximum
+        // listing count.
+        //
+        if (0 < maxListingCount && aFileMetadataList.size() >= maxListingCount) {
+          break;
+        }
+
+        ListBlobItem blobItem = blobItemIterator.next();
+
+        // Add the file metadata to the list if this is not a blob
+        // directory item.
+        if (blobItem instanceof CloudBlockBlobWrapper) {
+          String blobKey = null;
+          CloudBlockBlobWrapper blob = (CloudBlockBlobWrapper) blobItem;
+          BlobProperties properties = blob.getProperties();
+
+          // Determine format of the blob name depending on whether an absolute
+          // path is being used or not.
+          blobKey = normalizeKey(blob);
+
+          FileMetadata metadata;
+          if (retrieveFolderAttribute(blob)) {
+            metadata = new FileMetadata(blobKey, properties.getLastModified()
+                .getTime(), getPermissionStatus(blob),
+                BlobMaterialization.Explicit);
+          } else {
+            metadata = new FileMetadata(blobKey, properties.getLength(),
+                properties.getLastModified().getTime(),
+                getPermissionStatus(blob));
+          }
+
+          // Add the directory metadata to the list only if it's not already
+          // there.
+          FileMetadata existing = getDirectoryInList(aFileMetadataList, blobKey);
+          if (existing != null) {
+            aFileMetadataList.remove(existing);
+          }
+          aFileMetadataList.add(metadata);
+        } else if (blobItem instanceof CloudBlobDirectoryWrapper) {
+          CloudBlobDirectoryWrapper directory = (CloudBlobDirectoryWrapper) blobItem;
+
+          // This is a directory blob, push the current iterator onto
+          // the stack of iterators and start iterating through the current
+          // directory.
+          if (isUnboundedDepth || maxListingDepth > listingDepth) {
+            // Push the current directory on the stack and increment the listing
+            // depth.
+            dirIteratorStack.push(blobItemIterator);
+            ++listingDepth;
+
+            // The current blob item represents the new directory. Get
+            // an iterator for this directory and continue by iterating through
+            // this directory.
+            blobItems = directory.listBlobs(null, false,
+                EnumSet.noneOf(BlobListingDetails.class), null,
+                getInstrumentedContext());
+            blobItemIterator = blobItems.iterator();
+          } else {
+            // Determine format of directory name depending on whether an
+            // absolute path is being used or not.
+            String dirKey = normalizeKey(directory);
+
+            if (getDirectoryInList(aFileMetadataList, dirKey) == null) {
+              // Reached the targeted listing depth. Return metadata for the
+              // directory using default permissions.
+              //
+              // Note: Something smarter should be done about permissions. Maybe
+              // inherit the permissions of the first non-directory blob.
+              // Also, getting a proper value for last-modified is tricky.
+              FileMetadata directoryMetadata = new FileMetadata(dirKey, 0,
+                  defaultPermissionNoBlobMetadata(),
+                  BlobMaterialization.Implicit);
+
+              // Add the directory metadata to the list.
+              aFileMetadataList.add(directoryMetadata);
+            }
+          }
+        }
+      }
+
+      // Traversal of directory tree
+
+      // Check if the iterator stack is empty. If it is set the next blob
+      // iterator to null. This will act as a terminator for the for-loop.
+      // Otherwise pop the next iterator from the stack and continue looping.
+      //
+      if (dirIteratorStack.isEmpty()) {
+        blobItemIterator = null;
+      } else {
+        // Pop the next directory item from the stack and decrement the
+        // depth.
+        blobItemIterator = dirIteratorStack.pop();
+        --listingDepth;
+
+        // Assertion: Listing depth should not be less than zero.
+        if (listingDepth < 0) {
+          throw new AssertionError("Non-negative listing depth expected");
+        }
+      }
+    }
+  }
+
+  /**
+   * Deletes the given blob, taking special care that if we get a blob-not-found
+   * exception upon retrying the operation, we just swallow the error since what
+   * most probably happened is that the first operation succeeded on the server.
+   * 
+   * @param blob
+   *          The blob to delete.
+   * @throws StorageException
+   */
+  private void safeDelete(CloudBlockBlobWrapper blob) throws StorageException {
+    OperationContext operationContext = getInstrumentedContext();
+    try {
+      blob.delete(operationContext);
+    } catch (StorageException e) {
+      // On exception, check that if:
+      // 1. It's a BlobNotFound exception AND
+      // 2. It got there after one-or-more retries THEN
+      // we swallow the exception.
+      if (e.getErrorCode() != null && e.getErrorCode().equals("BlobNotFound")
+          && operationContext.getRequestResults().size() > 1
+          && operationContext.getRequestResults().get(0).getException() != null) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Swallowing delete exception on retry: " + e.getMessage());
+        }
+        return;
+      } else {
+        throw e;
+      }
+    }
+  }
+
+  @Override
+  public void delete(String key) throws IOException {
+    try {
+      if (checkContainer(ContainerAccessType.ReadThenWrite) == ContainerState.DoesntExist) {
+        // Container doesn't exist, no need to do anything
+        return;
+      }
+
+      // Get the blob reference an delete it.
+      CloudBlockBlobWrapper blob = getBlobReference(key);
+      if (blob.exists(getInstrumentedContext())) {
+        safeDelete(blob);
+      }
+    } catch (Exception e) {
+      // Re-throw as an Azure storage exception.
+      throw new AzureException(e);
+    }
+  }
+
+  @Override
+  public void rename(String srcKey, String dstKey) throws IOException {
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Moving " + srcKey + " to " + dstKey);
+    }
+
+    try {
+      // Attempts rename may occur before opening any streams so first,
+      // check if a session exists, if not create a session with the Azure
+      // storage server.
+      if (null == storageInteractionLayer) {
+        final String errMsg = String.format(
+            "Storage session expected for URI '%s' but does not exist.",
+            sessionUri);
+        throw new AssertionError(errMsg);
+      }
+
+      checkContainer(ContainerAccessType.ReadThenWrite);
+      // Get the source blob and assert its existence. If the source key
+      // needs to be normalized then normalize it.
+      CloudBlockBlobWrapper srcBlob = getBlobReference(srcKey);
+
+      if (!srcBlob.exists(getInstrumentedContext())) {
+        throw new AzureException("Source blob " + srcKey + " does not exist.");
+      }
+
+      // Get the destination blob. The destination key always needs to be
+      // normalized.
+      CloudBlockBlobWrapper dstBlob = getBlobReference(dstKey);
+
+      // Rename the source blob to the destination blob by copying it to
+      // the destination blob then deleting it.
+      //
+      dstBlob.startCopyFromBlob(srcBlob, getInstrumentedContext());
+      waitForCopyToComplete(dstBlob, getInstrumentedContext());
+
+      safeDelete(srcBlob);
+    } catch (Exception e) {
+      // Re-throw exception as an Azure storage exception.
+      throw new AzureException(e);
+    }
+  }
+
+  private void waitForCopyToComplete(CloudBlockBlobWrapper blob,
+      OperationContext opContext) throws AzureException {
+    boolean copyInProgress = true;
+    int exceptionCount = 0;
+    while (copyInProgress) {
+      try {
+        blob.downloadAttributes(opContext);
+      } catch (StorageException se) {
+        exceptionCount++;
+        if(exceptionCount > 10){
+          throw new AzureException("Too many storage exceptions during waitForCopyToComplete", se);
+        }
+      }
+
+      // test for null because mocked filesystem doesn't know about copystates
+      // yet.
+      copyInProgress = (blob.getCopyState() != null && blob.getCopyState()
+          .getStatus() == CopyStatus.PENDING);
+      if (copyInProgress) {
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException ie) {
+          Thread.currentThread().interrupt();
+        }
+      }
+    }
+  }
+
+  /**
+   * Changes the permission status on the given key.
+   */
+  @Override
+  public void changePermissionStatus(String key, PermissionStatus newPermission)
+      throws AzureException {
+    try {
+      checkContainer(ContainerAccessType.ReadThenWrite);
+      CloudBlockBlobWrapper blob = getBlobReference(key);
+      blob.downloadAttributes(getInstrumentedContext());
+      storePermissionStatus(blob, newPermission);
+      blob.uploadMetadata(getInstrumentedContext());
+    } catch (Exception e) {
+      throw new AzureException(e);
+    }
+  }
+
+  @Override
+  public void purge(String prefix) throws IOException {
+    try {
+
+      // Attempts to purge may occur before opening any streams so first,
+      // check if a session exists, if not create a session with the Azure
+      // storage server.
+      if (null == storageInteractionLayer) {
+        final String errMsg = String.format(
+            "Storage session expected for URI '%s' but does not exist.",
+            sessionUri);
+        throw new AssertionError(errMsg);
+      }
+
+      if (checkContainer(ContainerAccessType.ReadThenWrite) == ContainerState.DoesntExist) {
+        // Container doesn't exist, no need to do anything.
+        return;
+      }
+      // Get all blob items with the given prefix from the container and delete
+      // them.
+      Iterable<ListBlobItem> objects = listRootBlobs(prefix, false);
+      for (ListBlobItem blobItem : objects) {
+        ((CloudBlob) blobItem).delete(DeleteSnapshotsOption.NONE, null, null,
+            getInstrumentedContext());
+      }
+    } catch (Exception e) {
+      // Re-throw as an Azure storage exception.
+      //
+      throw new AzureException(e);
+    }
+  }
+
+  @Override
+  public void updateFolderLastModifiedTime(String key, Date lastModified)
+      throws AzureException {
+    try {
+      checkContainer(ContainerAccessType.ReadThenWrite);
+      CloudBlockBlobWrapper blob = getBlobReference(key);
+      blob.getProperties().setLastModified(lastModified);
+      blob.uploadProperties(getInstrumentedContext());
+    } catch (Exception e) {
+      // Caught exception while attempting update the properties. Re-throw as an
+      // Azure storage exception.
+      throw new AzureException(e);
+    }
+  }
+
+  @Override
+  public void updateFolderLastModifiedTime(String key) throws AzureException {
+    final Calendar lastModifiedCalendar = Calendar
+        .getInstance(Utility.LOCALE_US);
+    lastModifiedCalendar.setTimeZone(Utility.UTC_ZONE);
+    Date lastModified = lastModifiedCalendar.getTime();
+    updateFolderLastModifiedTime(key, lastModified);
+  }
+
+  @Override
+  public void dump() throws IOException {
+  }
+
+  @Override
+  public void close() {
+  }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlobMaterialization.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlobMaterialization.java
new file mode 100644
index 0000000..a1f8242
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/BlobMaterialization.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Indicates whether there are actual blobs indicating the existence of
+ * directories or whether we're inferring their existence from them having files
+ * in there.
+ */
+@InterfaceAudience.Private
+enum BlobMaterialization {
+  /**
+   * Indicates a directory that isn't backed by an actual blob, but its
+   * existence is implied by the fact that there are files in there. For
+   * example, if the blob /a/b exists then it implies the existence of the /a
+   * directory if there's no /a blob indicating it.
+   */
+  Implicit,
+  /**
+   * Indicates that the directory is backed by an actual blob that has the
+   * isFolder metadata on it.
+   */
+  Explicit,
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/FileMetadata.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/FileMetadata.java
new file mode 100644
index 0000000..5085a0f
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/FileMetadata.java
@@ -0,0 +1,120 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+
+/**
+ * <p>
+ * Holds basic metadata for a file stored in a {@link NativeFileSystemStore}.
+ * </p>
+ */
+@InterfaceAudience.Private
+class FileMetadata {
+  private final String key;
+  private final long length;
+  private final long lastModified;
+  private final boolean isDir;
+  private final PermissionStatus permissionStatus;
+  private final BlobMaterialization blobMaterialization;
+
+  /**
+   * Constructs a FileMetadata object for a file.
+   * 
+   * @param key
+   *          The key (path) to the file.
+   * @param length
+   *          The length in bytes of the file.
+   * @param lastModified
+   *          The last modified date (milliseconds since January 1, 1970 UTC.)
+   * @param permissionStatus
+   *          The permission for the file.
+   */
+  public FileMetadata(String key, long length, long lastModified,
+      PermissionStatus permissionStatus) {
+    this.key = key;
+    this.length = length;
+    this.lastModified = lastModified;
+    this.isDir = false;
+    this.permissionStatus = permissionStatus;
+    this.blobMaterialization = BlobMaterialization.Explicit; // File are never
+                                                             // implicit.
+  }
+
+  /**
+   * Constructs a FileMetadata object for a directory.
+   * 
+   * @param key
+   *          The key (path) to the directory.
+   * @param lastModified
+   *          The last modified date (milliseconds since January 1, 1970 UTC.)
+   * @param permissionStatus
+   *          The permission for the directory.
+   * @param blobMaterialization
+   *          Whether this is an implicit (no real blob backing it) or explicit
+   *          directory.
+   */
+  public FileMetadata(String key, long lastModified,
+      PermissionStatus permissionStatus, BlobMaterialization blobMaterialization) {
+    this.key = key;
+    this.isDir = true;
+    this.length = 0;
+    this.lastModified = lastModified;
+    this.permissionStatus = permissionStatus;
+    this.blobMaterialization = blobMaterialization;
+  }
+
+  public boolean isDir() {
+    return isDir;
+  }
+
+  public String getKey() {
+    return key;
+  }
+
+  public long getLength() {
+    return length;
+  }
+
+  public long getLastModified() {
+    return lastModified;
+  }
+
+  public PermissionStatus getPermissionStatus() {
+    return permissionStatus;
+  }
+
+  /**
+   * Indicates whether this is an implicit directory (no real blob backing it)
+   * or an explicit one.
+   * 
+   * @return Implicit if this is an implicit directory, or Explicit if it's an
+   *         explicit directory or a file.
+   */
+  public BlobMaterialization getBlobMaterialization() {
+    return blobMaterialization;
+  }
+
+  @Override
+  public String toString() {
+    return "FileMetadata[" + key + ", " + length + ", " + lastModified + ", "
+        + permissionStatus + "]";
+  }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/KeyProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/KeyProvider.java
new file mode 100644
index 0000000..4c3a369
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/KeyProvider.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * The interface that every Azure file system key provider must implement.
+ */
+@InterfaceAudience.Private
+public interface KeyProvider {
+  /**
+   * Key providers must implement this method. Given a list of configuration
+   * parameters for the specified Azure storage account, retrieve the plaintext
+   * storage account key.
+   * 
+   * @param accountName
+   *          the storage account name
+   * @param conf
+   *          Hadoop configuration parameters
+   * @return the plaintext storage account key
+   * @throws KeyProviderException
+   */
+  String getStorageAccountKey(String accountName, Configuration conf)
+      throws KeyProviderException;
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/KeyProviderException.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/KeyProviderException.java
new file mode 100644
index 0000000..b65b2e6
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/KeyProviderException.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Thrown if there is a problem instantiating a KeyProvider or retrieving a key
+ * using a KeyProvider object.
+ */
+@InterfaceAudience.Private
+public class KeyProviderException extends Exception {
+  private static final long serialVersionUID = 1L;
+
+  public KeyProviderException(String message) {
+    super(message);
+  }
+
+  public KeyProviderException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public KeyProviderException(Throwable t) {
+    super(t);
+  }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
new file mode 100644
index 0000000..30e6b30
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
@@ -0,0 +1,1465 @@
+/**
+ * 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.DataInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.BufferedFSInputStream;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FSInputStream;
+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.fs.permission.PermissionStatus;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Progressable;
+
+
+import com.google.common.annotations.VisibleForTesting;
+import com.microsoft.windowsazure.storage.core.Utility;
+
+/**
+ * <p>
+ * A {@link FileSystem} for reading and writing files stored on <a
+ * href="http://store.azure.com/">Windows Azure</a>. This implementation is
+ * blob-based and stores files on Azure in their native form so they can be read
+ * by other Azure tools.
+ * </p>
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class NativeAzureFileSystem extends FileSystem {
+
+  @Override
+  public String getScheme() {
+    return "wasb";
+  }
+
+  
+  /**
+   * <p>
+   * A {@link FileSystem} for reading and writing files stored on <a
+   * href="http://store.azure.com/">Windows Azure</a>. This implementation is
+   * blob-based and stores files on Azure in their native form so they can be read
+   * by other Azure tools. This implementation uses HTTPS for secure network communication.
+   * </p>
+   */
+  public static class Secure extends NativeAzureFileSystem {
+    @Override
+    public String getScheme() {
+      return "wasbs";
+    }
+  }
+
+  public static final Log LOG = LogFactory.getLog(NativeAzureFileSystem.class);
+
+  static final String AZURE_BLOCK_SIZE_PROPERTY_NAME = "fs.azure.block.size";
+  /**
+   * The time span in seconds before which we consider a temp blob to be
+   * dangling (not being actively uploaded to) and up for reclamation.
+   * 
+   * So e.g. if this is 60, then any temporary blobs more than a minute old
+   * would be considered dangling.
+   */
+  static final String AZURE_TEMP_EXPIRY_PROPERTY_NAME = "fs.azure.fsck.temp.expiry.seconds";
+  private static final int AZURE_TEMP_EXPIRY_DEFAULT = 3600;
+  static final String PATH_DELIMITER = Path.SEPARATOR;
+  static final String AZURE_TEMP_FOLDER = "_$azuretmpfolder$";
+
+  private static final int AZURE_LIST_ALL = -1;
+  private static final int AZURE_UNBOUNDED_DEPTH = -1;
+
+  private static final long MAX_AZURE_BLOCK_SIZE = 512 * 1024 * 1024L;
+
+  /**
+   * The configuration property that determines which group owns files created
+   * in WASB.
+   */
+  private static final String AZURE_DEFAULT_GROUP_PROPERTY_NAME = "fs.azure.permissions.supergroup";
+  /**
+   * The default value for fs.azure.permissions.supergroup. Chosen as the same
+   * default as DFS.
+   */
+  static final String AZURE_DEFAULT_GROUP_DEFAULT = "supergroup";
+
+  static final String AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME = "fs.azure.block.location.impersonatedhost";
+  private static final String AZURE_BLOCK_LOCATION_HOST_DEFAULT = "localhost";
+
+  private class NativeAzureFsInputStream extends FSInputStream {
+    private InputStream in;
+    private final String key;
+    private long pos = 0;
+
+    public NativeAzureFsInputStream(DataInputStream in, String key) {
+      this.in = in;
+      this.key = key;
+    }
+
+    /*
+     * Reads the next byte of data from the input stream. The value byte is
+     * returned as an integer in the range 0 to 255. If no byte is available
+     * because the end of the stream has been reached, the value -1 is returned.
+     * This method blocks until input data is available, the end of the stream
+     * is detected, or an exception is thrown.
+     * 
+     * @returns int An integer corresponding to the byte read.
+     */
+    @Override
+    public synchronized int read() throws IOException {
+      int result = 0;
+      result = in.read();
+      if (result != -1) {
+        pos++;
+        if (statistics != null) {
+          statistics.incrementBytesRead(1);
+        }
+      }
+
+      // Return to the caller with the result.
+      //
+      return result;
+    }
+
+    /*
+     * Reads up to len bytes of data from the input stream into an array of
+     * bytes. An attempt is made to read as many as len bytes, but a smaller
+     * number may be read. The number of bytes actually read is returned as an
+     * integer. This method blocks until input data is available, end of file is
+     * detected, or an exception is thrown. If len is zero, then no bytes are
+     * read and 0 is returned; otherwise, there is an attempt to read at least
+     * one byte. If no byte is available because the stream is at end of file,
+     * the value -1 is returned; otherwise, at least one byte is read and stored
+     * into b.
+     * 
+     * @param b -- the buffer into which data is read
+     * 
+     * @param off -- the start offset in the array b at which data is written
+     * 
+     * @param len -- the maximum number of bytes read
+     * 
+     * @ returns int The total number of byes read into the buffer, or -1 if
+     * there is no more data because the end of stream is reached.
+     */
+    @Override
+    public synchronized int read(byte[] b, int off, int len) throws IOException {
+      int result = 0;
+      result = in.read(b, off, len);
+      if (result > 0) {
+        pos += result;
+      }
+
+      if (null != statistics) {
+        statistics.incrementBytesRead(result);
+      }
+
+      // Return to the caller with the result.
+      return result;
+    }
+
+    @Override
+    public void close() throws IOException {
+      in.close();
+    }
+
+    @Override
+    public synchronized void seek(long pos) throws IOException {
+      in.close();
+      in = store.retrieve(key, pos);
+      this.pos = pos;
+    }
+
+    @Override
+    public synchronized long getPos() throws IOException {
+      return pos;
+    }
+
+    @Override
+    public boolean seekToNewSource(long targetPos) throws IOException {
+      return false;
+    }
+  }
+
+  private class NativeAzureFsOutputStream extends OutputStream {
+    // We should not override flush() to actually close current block and flush
+    // to DFS, this will break applications that assume flush() is a no-op.
+    // Applications are advised to use Syncable.hflush() for that purpose.
+    // NativeAzureFsOutputStream needs to implement Syncable if needed.
+    private String key;
+    private String keyEncoded;
+    private OutputStream out;
+
+    public NativeAzureFsOutputStream(OutputStream out, String aKey,
+        String anEncodedKey) throws IOException {
+      // Check input arguments. The output stream should be non-null and the
+      // keys
+      // should be valid strings.
+      if (null == out) {
+        throw new IllegalArgumentException(
+            "Illegal argument: the output stream is null.");
+      }
+
+      if (null == aKey || 0 == aKey.length()) {
+        throw new IllegalArgumentException(
+            "Illegal argument the key string is null or empty");
+      }
+
+      if (null == anEncodedKey || 0 == anEncodedKey.length()) {
+        throw new IllegalArgumentException(
+            "Illegal argument the encoded key string is null or empty");
+      }
+
+      // Initialize the member variables with the incoming parameters.
+      this.out = out;
+
+      setKey(aKey);
+      setEncodedKey(anEncodedKey);
+    }
+
+    @Override
+    public synchronized void close() throws IOException {
+      if (out != null) {
+        // Close the output stream and decode the key for the output stream
+        // before returning to the caller.
+        //
+        out.close();
+        restoreKey();
+        out = null;
+      }
+    }
+
+    /**
+     * Writes the specified byte to this output stream. The general contract for
+     * write is that one byte is written to the output stream. The byte to be
+     * written is the eight low-order bits of the argument b. The 24 high-order
+     * bits of b are ignored.
+     * 
+     * @param b
+     *          32-bit integer of block of 4 bytes
+     */
+    @Override
+    public void write(int b) throws IOException {
+      out.write(b);
+    }
+
+    /**
+     * Writes b.length bytes from the specified byte array to this output
+     * stream. The general contract for write(b) is that it should have exactly
+     * the same effect as the call write(b, 0, b.length).
+     * 
+     * @param b
+     *          Block of bytes to be written to the output stream.
+     */
+    @Override
+    public void write(byte[] b) throws IOException {
+      out.write(b);
+    }
+
+    /**
+     * Writes <code>len</code> from the specified byte array starting at offset
+     * <code>off</code> to the output stream. The general contract for write(b,
+     * off, len) is that some of the bytes in the array <code>
+     * b</code b> are written to the output stream in order; element
+     * <code>b[off]</code> is the first byte written and
+     * <code>b[off+len-1]</code> is the last byte written by this operation.
+     * 
+     * @param b
+     *          Byte array to be written.
+     * @param off
+     *          Write this offset in stream.
+     * @param len
+     *          Number of bytes to be written.
+     */
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+      out.write(b, off, len);
+    }
+
+    /**
+     * Get the blob name.
+     * 
+     * @return String Blob name.
+     */
+    public String getKey() {
+      return key;
+    }
+
+    /**
+     * Set the blob name.
+     * 
+     * @param key
+     *          Blob name.
+     */
+    public void setKey(String key) {
+      this.key = key;
+    }
+
+    /**
+     * Get the blob name.
+     * 
+     * @return String Blob name.
+     */
+    public String getEncodedKey() {
+      return keyEncoded;
+    }
+
+    /**
+     * Set the blob name.
+     * 
+     * @param anEncodedKey
+     *          Blob name.
+     */
+    public void setEncodedKey(String anEncodedKey) {
+      this.keyEncoded = anEncodedKey;
+    }
+
+    /**
+     * Restore the original key name from the m_key member variable. Note: The
+     * output file stream is created with an encoded blob store key to guarantee
+     * load balancing on the front end of the Azure storage partition servers.
+     * The create also includes the name of the original key value which is
+     * stored in the m_key member variable. This method should only be called
+     * when the stream is closed.
+     * 
+     * @param anEncodedKey
+     *          Encoding of the original key stored in m_key member.
+     */
+    private void restoreKey() throws IOException {
+      store.rename(getEncodedKey(), getKey());
+    }
+  }
+
+  private URI uri;
+  private NativeFileSystemStore store;
+  private AzureNativeFileSystemStore actualStore;
+  private Path workingDir;
+  private long blockSize = MAX_AZURE_BLOCK_SIZE;
+  private static boolean suppressRetryPolicy = false;
+
+  public NativeAzureFileSystem() {
+    // set store in initialize()
+  }
+
+  public NativeAzureFileSystem(NativeFileSystemStore store) {
+    this.store = store;
+  }
+
+  /**
+   * Suppress the default retry policy for the Storage, useful in unit tests to
+   * test negative cases without waiting forever.
+   */
+  @VisibleForTesting
+  static void suppressRetryPolicy() {
+    suppressRetryPolicy = true;
+  }
+
+  /**
+   * Undo the effect of suppressRetryPolicy.
+   */
+  @VisibleForTesting
+  static void resumeRetryPolicy() {
+    suppressRetryPolicy = false;
+  }
+
+  /**
+   * Checks if the given URI scheme is a scheme that's affiliated with the Azure
+   * File System.
+   * 
+   * @param scheme
+   *          The URI scheme.
+   * @return true iff it's an Azure File System URI scheme.
+   */
+  private static boolean isWasbScheme(String scheme) {
+    // The valid schemes are: asv (old name), asvs (old name over HTTPS),
+    // wasb (new name), wasbs (new name over HTTPS).
+    return scheme != null
+        && (scheme.equalsIgnoreCase("asv") || scheme.equalsIgnoreCase("asvs")
+            || scheme.equalsIgnoreCase("wasb") || scheme
+              .equalsIgnoreCase("wasbs"));
+  }
+
+  /**
+   * Puts in the authority of the default file system if it is a WASB file
+   * system and the given URI's authority is null.
+   * 
+   * @return The URI with reconstructed authority if necessary and possible.
+   */
+  private static URI reconstructAuthorityIfNeeded(URI uri, Configuration conf) {
+    if (null == uri.getAuthority()) {
+      // If WASB is the default file system, get the authority from there
+      URI defaultUri = FileSystem.getDefaultUri(conf);
+      if (defaultUri != null && isWasbScheme(defaultUri.getScheme())) {
+        try {
+          // Reconstruct the URI with the authority from the default URI.
+          return new URI(uri.getScheme(), defaultUri.getAuthority(),
+              uri.getPath(), uri.getQuery(), uri.getFragment());
+        } catch (URISyntaxException e) {
+          // This should never happen.
+          throw new Error("Bad URI construction", e);
+        }
+      }
+    }
+    return uri;
+  }
+
+  @Override
+  protected void checkPath(Path path) {
+    // Make sure to reconstruct the path's authority if needed
+    super.checkPath(new Path(reconstructAuthorityIfNeeded(path.toUri(),
+        getConf())));
+  }
+
+  @Override
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    // Check authority for the URI to guarantee that it is non-null.
+    uri = reconstructAuthorityIfNeeded(uri, conf);
+    if (null == uri.getAuthority()) {
+      final String errMsg = String
+          .format("Cannot initialize WASB file system, URI authority not recognized.");
+      throw new IllegalArgumentException(errMsg);
+    }
+    super.initialize(uri, conf);
+
+    if (store == null) {
+      store = createDefaultStore(conf);
+    }
+
+    store.initialize(uri, conf);
+    setConf(conf);
+    this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
+    this.workingDir = new Path("/user", UserGroupInformation.getCurrentUser()
+        .getShortUserName()).makeQualified(getUri(), getWorkingDirectory());
+    this.blockSize = conf.getLong(AZURE_BLOCK_SIZE_PROPERTY_NAME,
+        MAX_AZURE_BLOCK_SIZE);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NativeAzureFileSystem. Initializing.");
+      LOG.debug("  blockSize  = "
+          + conf.getLong(AZURE_BLOCK_SIZE_PROPERTY_NAME, MAX_AZURE_BLOCK_SIZE));
+    }
+
+  }
+
+  private NativeFileSystemStore createDefaultStore(Configuration conf) {
+    actualStore = new AzureNativeFileSystemStore();
+
+    if (suppressRetryPolicy) {
+      actualStore.suppressRetryPolicy();
+    }
+    return actualStore;
+  }
+
+  // Note: The logic for this method is confusing as to whether it strips the
+  // last slash or not (it adds it in the beginning, then strips it at the end).
+  // We should revisit that.
+  private String pathToKey(Path path) {
+    // Convert the path to a URI to parse the scheme, the authority, and the
+    // path from the path object.
+    URI tmpUri = path.toUri();
+    String pathUri = tmpUri.getPath();
+
+    // The scheme and authority is valid. If the path does not exist add a "/"
+    // separator to list the root of the container.
+    Path newPath = path;
+    if ("".equals(pathUri)) {
+      newPath = new Path(tmpUri.toString() + Path.SEPARATOR);
+    }
+
+    // Verify path is absolute if the path refers to a windows drive scheme.
+    if (!newPath.isAbsolute()) {
+      throw new IllegalArgumentException("Path must be absolute: " + path);
+    }
+
+    String key = null;
+    key = newPath.toUri().getPath();
+    if (key.length() == 1) {
+      return key;
+    } else {
+      return key.substring(1); // remove initial slash
+    }
+  }
+
+  private static Path keyToPath(String key) {
+    if (key.equals("/")) {
+      return new Path("/"); // container
+    }
+    return new Path("/" + key);
+  }
+
+  private Path makeAbsolute(Path path) {
+    if (path.isAbsolute()) {
+      return path;
+    }
+    return new Path(workingDir, path);
+  }
+
+  /**
+   * For unit test purposes, retrieves the AzureNativeFileSystemStore store
+   * backing this file system.
+   * 
+   * @return The store object.
+   */
+  @VisibleForTesting
+  AzureNativeFileSystemStore getStore() {
+    return actualStore;
+  }
+
+  /** This optional operation is not yet supported. */
+  @Override
+  public FSDataOutputStream append(Path f, int bufferSize, Progressable progress)
+      throws IOException {
+    throw new IOException("Not supported");
+  }
+
+  @Override
+  public FSDataOutputStream create(Path f, FsPermission permission,
+      boolean overwrite, int bufferSize, short replication, long blockSize,
+      Progressable progress) throws IOException {
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Creating file: " + f.toString());
+    }
+
+    if (containsColon(f)) {
+      throw new IOException("Cannot create file " + f
+          + " through WASB that has colons in the name");
+    }
+
+    Path absolutePath = makeAbsolute(f);
+    String key = pathToKey(absolutePath);
+
+    FileMetadata existingMetadata = store.retrieveMetadata(key);
+    if (existingMetadata != null) {
+      if (existingMetadata.isDir()) {
+        throw new IOException("Cannot create file " + f
+            + "; already exists as a directory.");
+      }
+      if (!overwrite) {
+        throw new IOException("File already exists:" + f);
+      }
+    }
+
+    Path parentFolder = absolutePath.getParent();
+    if (parentFolder != null && parentFolder.getParent() != null) { // skip root
+      // Update the parent folder last modified time if the parent folder
+      // already exists.
+      String parentKey = pathToKey(parentFolder);
+      FileMetadata parentMetadata = store.retrieveMetadata(parentKey);
+      if (parentMetadata != null
+          && parentMetadata.isDir()
+          && parentMetadata.getBlobMaterialization() == BlobMaterialization.Explicit) {
+        store.updateFolderLastModifiedTime(parentKey);
+      } else {
+        // Make sure that the parent folder exists.
+        mkdirs(parentFolder, permission);
+      }
+    }
+
+    // Open the output blob stream based on the encoded key.
+    String keyEncoded = encodeKey(key);
+
+    // Mask the permission first (with the default permission mask as well).
+    FsPermission masked = applyUMask(permission, UMaskApplyMode.NewFile);
+    PermissionStatus permissionStatus = createPermissionStatus(masked);
+
+    // First create a blob at the real key, pointing back to the temporary file
+    // This accomplishes a few things:
+    // 1. Makes sure we can create a file there.
+    // 2. Makes it visible to other concurrent threads/processes/nodes what
+    // we're
+    // doing.
+    // 3. Makes it easier to restore/cleanup data in the event of us crashing.
+    store.storeEmptyLinkFile(key, keyEncoded, permissionStatus);
+
+    // The key is encoded to point to a common container at the storage server.
+    // This reduces the number of splits on the server side when load balancing.
+    // Ingress to Azure storage can take advantage of earlier splits. We remove
+    // the root path to the key and prefix a random GUID to the tail (or leaf
+    // filename) of the key. Keys are thus broadly and randomly distributed over
+    // a single container to ease load balancing on the storage server. When the
+    // blob is committed it is renamed to its earlier key. Uncommitted blocks
+    // are not cleaned up and we leave it to Azure storage to garbage collect
+    // these
+    // blocks.
+    OutputStream bufOutStream = new NativeAzureFsOutputStream(store.storefile(
+        keyEncoded, permissionStatus), key, keyEncoded);
+
+    // Construct the data output stream from the buffered output stream.
+    FSDataOutputStream fsOut = new FSDataOutputStream(bufOutStream, statistics);
+
+    // Return data output stream to caller.
+    return fsOut;
+  }
+
+  @Override
+  @Deprecated
+  public boolean delete(Path path) throws IOException {
+    return delete(path, true);
+  }
+
+  @Override
+  public boolean delete(Path f, boolean recursive) throws IOException {
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Deleting file: " + f.toString());
+    }
+
+    Path absolutePath = makeAbsolute(f);
+    String key = pathToKey(absolutePath);
+
+    // Capture the metadata for the path.
+    //
+    FileMetadata metaFile = store.retrieveMetadata(key);
+
+    if (null == metaFile) {
+      // The path to be deleted does not exist.
+      return false;
+    }
+
+    // The path exists, determine if it is a folder containing objects,
+    // an empty folder, or a simple file and take the appropriate actions.
+    if (!metaFile.isDir()) {
+      // The path specifies a file. We need to check the parent path
+      // to make sure it's a proper materialized directory before we
+      // delete the file. Otherwise we may get into a situation where
+      // the file we were deleting was the last one in an implicit directory
+      // (e.g. the blob store only contains the blob a/b and there's no
+      // corresponding directory blob a) and that would implicitly delete
+      // the directory as well, which is not correct.
+      Path parentPath = absolutePath.getParent();
+      if (parentPath.getParent() != null) {// Not root
+        String parentKey = pathToKey(parentPath);
+        FileMetadata parentMetadata = store.retrieveMetadata(parentKey);
+        if (!parentMetadata.isDir()) {
+          // Invalid state: the parent path is actually a file. Throw.
+          throw new AzureException("File " + f + " has a parent directory "
+              + parentPath + " which is also a file. Can't resolve.");
+        }
+        if (parentMetadata.getBlobMaterialization() == BlobMaterialization.Implicit) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Found an implicit parent directory while trying to"
+                + " delete the file " + f + ". Creating the directory blob for"
+                + " it in " + parentKey + ".");
+          }
+          store.storeEmptyFolder(parentKey,
+              createPermissionStatus(FsPermission.getDefault()));
+        } else {
+          store.updateFolderLastModifiedTime(parentKey);
+        }
+      }
+      store.delete(key);
+    } else {
+      // The path specifies a folder. Recursively delete all entries under the
+      // folder.
+      Path parentPath = absolutePath.getParent();
+      if (parentPath.getParent() != null) {
+        String parentKey = pathToKey(parentPath);
+        FileMetadata parentMetadata = store.retrieveMetadata(parentKey);
+
+        if (parentMetadata.getBlobMaterialization() == BlobMaterialization.Implicit) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Found an implicit parent directory while trying to"
+                + " delete the directory " + f
+                + ". Creating the directory blob for" + " it in " + parentKey
+                + ".");
+          }
+          store.storeEmptyFolder(parentKey,
+              createPermissionStatus(FsPermission.getDefault()));
+        }
+      }
+
+      // List all the blobs in the current folder.
+      String priorLastKey = null;
+      PartialListing listing = store.listAll(key, AZURE_LIST_ALL, 1,
+          priorLastKey);
+      FileMetadata[] contents = listing.getFiles();
+      if (!recursive && contents.length > 0) {
+        // The folder is non-empty and recursive delete was not specified.
+        // Throw an exception indicating that a non-recursive delete was
+        // specified for a non-empty folder.
+        throw new IOException("Non-recursive delete of non-empty directory "
+            + f.toString());
+      }
+
+      // Delete all the files in the folder.
+      for (FileMetadata p : contents) {
+        // Tag on the directory name found as the suffix of the suffix of the
+        // parent directory to get the new absolute path.
+        String suffix = p.getKey().substring(
+            p.getKey().lastIndexOf(PATH_DELIMITER));
+        if (!p.isDir()) {
+          store.delete(key + suffix);
+        } else {
+          // Recursively delete contents of the sub-folders. Notice this also
+          // deletes the blob for the directory.
+          if (!delete(new Path(f.toString() + suffix), true)) {
+            return false;
+          }
+        }
+      }
+      store.delete(key);
+
+      // Update parent directory last modified time
+      Path parent = absolutePath.getParent();
+      if (parent != null && parent.getParent() != null) { // not root
+        String parentKey = pathToKey(parent);
+        store.updateFolderLastModifiedTime(parentKey);
+      }
+    }
+
+    // File or directory was successfully deleted.
+    return true;
+  }
+
+  @Override
+  public FileStatus getFileStatus(Path f) throws IOException {
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Getting the file status for " + f.toString());
+    }
+
+    // Capture the absolute path and the path to key.
+    Path absolutePath = makeAbsolute(f);
+    String key = pathToKey(absolutePath);
+    if (key.length() == 0) { // root always exists
+      return newDirectory(null, absolutePath);
+    }
+
+    // The path is either a folder or a file. Retrieve metadata to
+    // determine if it is a directory or file.
+    FileMetadata meta = store.retrieveMetadata(key);
+    if (meta != null) {
+      if (meta.isDir()) {
+        // The path is a folder with files in it.
+        //
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Path " + f.toString() + "is a folder.");
+        }
+
+        // Return reference to the directory object.
+        return newDirectory(meta, absolutePath);
+      }
+
+      // The path is a file.
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Found the path: " + f.toString() + " as a file.");
+      }
+
+      // Return with reference to a file object.
+      return newFile(meta, absolutePath);
+    }
+
+    // File not found. Throw exception no such file or directory.
+    // Note: Should never get to this point since the root always exists.
+    throw new FileNotFoundException(absolutePath
+        + ": No such file or directory.");
+  }
+
+  @Override
+  public URI getUri() {
+    return uri;
+  }
+
+  /**
+   * Retrieve the status of a given path if it is a file, or of all the
+   * contained files if it is a directory.
+   */
+  @Override
+  public FileStatus[] listStatus(Path f) throws IOException {
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Listing status for " + f.toString());
+    }
+
+    Path absolutePath = makeAbsolute(f);
+    String key = pathToKey(absolutePath);
+    Set<FileStatus> status = new TreeSet<FileStatus>();
+    FileMetadata meta = store.retrieveMetadata(key);
+
+    if (meta != null) {
+      if (!meta.isDir()) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Found path as a file");
+        }
+        return new FileStatus[] { newFile(meta, absolutePath) };
+      }
+      String partialKey = null;
+      PartialListing listing = store.list(key, AZURE_LIST_ALL, 1, partialKey);
+      for (FileMetadata fileMetadata : listing.getFiles()) {
+        Path subpath = keyToPath(fileMetadata.getKey());
+
+        // Test whether the metadata represents a file or directory and
+        // add the appropriate metadata object.
+        //
+        // Note: There was a very old bug here where directories were added
+        // to the status set as files flattening out recursive listings
+        // using "-lsr" down the file system hierarchy.
+        if (fileMetadata.isDir()) {
+          // Make sure we hide the temp upload folder
+          if (fileMetadata.getKey().equals(AZURE_TEMP_FOLDER)) {
+            // Don't expose that.
+            continue;
+          }
+          status.add(newDirectory(fileMetadata, subpath));
+        } else {
+          status.add(newFile(fileMetadata, subpath));
+        }
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Found path as a directory with " + status.size()
+            + " files in it.");
+      }
+    } else {
+      // There is no metadata found for the path.
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Did not find any metadata for path: " + key);
+      }
+
+      throw new FileNotFoundException("File" + f + " does not exist.");
+    }
+
+    return status.toArray(new FileStatus[0]);
+  }
+
+  private FileStatus newFile(FileMetadata meta, Path path) {
+    return new FileStatus(meta.getLength(), false, 1, blockSize,
+        meta.getLastModified(), 0, meta.getPermissionStatus().getPermission(),
+        meta.getPermissionStatus().getUserName(), meta.getPermissionStatus()
+            .getGroupName(),
+        path.makeQualified(getUri(), getWorkingDirectory()));
+  }
+
+  private FileStatus newDirectory(FileMetadata meta, Path path) {
+    return new FileStatus(0, true, 1, blockSize, meta == null ? 0
+        : meta.getLastModified(), 0, meta == null ? FsPermission.getDefault()
+        : meta.getPermissionStatus().getPermission(), meta == null ? "" : meta
+        .getPermissionStatus().getUserName(), meta == null ? "" : meta
+        .getPermissionStatus().getGroupName(), path.makeQualified(getUri(),
+        getWorkingDirectory()));
+  }
+
+  private static enum UMaskApplyMode {
+    NewFile, NewDirectory, ChangeExistingFile, ChangeExistingDirectory,
+  }
+
+  /**
+   * Applies the applicable UMASK's on the given permission.
+   * 
+   * @param permission
+   *          The permission to mask.
+   * @param applyDefaultUmask
+   *          Whether to also apply the default umask.
+   * @return The masked persmission.
+   */
+  private FsPermission applyUMask(final FsPermission permission,
+      final UMaskApplyMode applyMode) {
+    FsPermission newPermission = new FsPermission(permission);
+    // Apply the default umask - this applies for new files or directories.
+    if (applyMode == UMaskApplyMode.NewFile
+        || applyMode == UMaskApplyMode.NewDirectory) {
+      newPermission = newPermission
+          .applyUMask(FsPermission.getUMask(getConf()));
+    }
+    return newPermission;
+  }
+
+  /**
+   * Creates the PermissionStatus object to use for the given permission, based
+   * on the current user in context.
+   * 
+   * @param permission
+   *          The permission for the file.
+   * @return The permission status object to use.
+   * @throws IOException
+   *           If login fails in getCurrentUser
+   */
+  private PermissionStatus createPermissionStatus(FsPermission permission)
+      throws IOException {
+    // Create the permission status for this file based on current user
+    return new PermissionStatus(UserGroupInformation.getCurrentUser()
+        .getShortUserName(), getConf().get(AZURE_DEFAULT_GROUP_PROPERTY_NAME,
+        AZURE_DEFAULT_GROUP_DEFAULT), permission);
+  }
+
+  @Override
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Creating directory: " + f.toString());
+    }
+
+    if (containsColon(f)) {
+      throw new IOException("Cannot create directory " + f
+          + " through WASB that has colons in the name");
+    }
+
+    Path absolutePath = makeAbsolute(f);
+    PermissionStatus permissionStatus = createPermissionStatus(applyUMask(
+        permission, UMaskApplyMode.NewDirectory));
+
+    ArrayList<String> keysToCreateAsFolder = new ArrayList<String>();
+    ArrayList<String> keysToUpdateAsFolder = new ArrayList<String>();
+    boolean childCreated = false;
+    // Check that there is no file in the parent chain of the given path.
+    // Stop when you get to the root
+    for (Path current = absolutePath, parent = current.getParent(); parent != null; current = parent, parent = current
+        .getParent()) {
+      String currentKey = pathToKey(current);
+      FileMetadata currentMetadata = store.retrieveMetadata(currentKey);
+      if (currentMetadata != null && !currentMetadata.isDir()) {
+        throw new IOException("Cannot create directory " + f + " because "
+            + current + " is an existing file.");
+      } else if (currentMetadata == null
+          || (currentMetadata.isDir() && currentMetadata
+              .getBlobMaterialization() == BlobMaterialization.Implicit)) {
+        keysToCreateAsFolder.add(currentKey);
+        childCreated = true;
+      } else {
+        // The directory already exists. Its last modified time need to be
+        // updated if there is a child directory created under it.
+        if (childCreated) {
+          keysToUpdateAsFolder.add(currentKey);
+        }
+        childCreated = false;
+      }
+    }
+
+    for (String currentKey : keysToCreateAsFolder) {
+      store.storeEmptyFolder(currentKey, permissionStatus);
+    }
+
+    // Take the time after finishing mkdirs as the modified time, and update all
+    // the existing directories' modified time to it uniformly.
+    final Calendar lastModifiedCalendar = Calendar
+        .getInstance(Utility.LOCALE_US);
+    lastModifiedCalendar.setTimeZone(Utility.UTC_ZONE);
+    Date lastModified = lastModifiedCalendar.getTime();
+    for (String key : keysToUpdateAsFolder) {
+      store.updateFolderLastModifiedTime(key, lastModified);
+    }
+
+    // otherwise throws exception
+    return true;
+  }
+
+  @Override
+  public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Opening file: " + f.toString());
+    }
+
+    Path absolutePath = makeAbsolute(f);
+    String key = pathToKey(absolutePath);
+    FileMetadata meta = store.retrieveMetadata(key);
+    if (meta == null) {
+      throw new FileNotFoundException(f.toString());
+    }
+    if (meta.isDir()) {
+      throw new FileNotFoundException(f.toString()
+          + " is a directory not a file.");
+    }
+
+    return new FSDataInputStream(new BufferedFSInputStream(
+        new NativeAzureFsInputStream(store.retrieve(key), key), bufferSize));
+  }
+
+  @Override
+  public boolean rename(Path src, Path dst) throws IOException {
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Moving " + src + " to " + dst);
+    }
+
+    if (containsColon(dst)) {
+      throw new IOException("Cannot rename to file " + dst
+          + " through WASB that has colons in the name");
+    }
+
+    String srcKey = pathToKey(makeAbsolute(src));
+
+    if (srcKey.length() == 0) {
+      // Cannot rename root of file system
+      return false;
+    }
+
+    FileMetadata srcMetadata = store.retrieveMetadata(srcKey);
+    if (srcMetadata == null) {
+      // Source doesn't exist
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Source " + src + " doesn't exist, failing the rename.");
+      }
+      return false;
+    }
+
+    // Figure out the final destination
+    Path absoluteDst = makeAbsolute(dst);
+    String dstKey = pathToKey(absoluteDst);
+    FileMetadata dstMetadata = store.retrieveMetadata(dstKey);
+
+    // directory rename validations
+    if (srcMetadata.isDir()) {
+
+      // rename dir to self is an error
+      if (srcKey.equals(dstKey)) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Renaming directory to itself is disallowed. path=" + src);
+        }
+        return false;
+      }
+
+      // rename dir to (sub-)child of self is an error. see
+      // FileSystemContractBaseTest.testRenameChildDirForbidden
+      if (dstKey.startsWith(srcKey + PATH_DELIMITER)) {
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Renaming directory to a itself is disallowed. src=" + src
+              + " dest=" + dst);
+        }
+        return false;
+      }
+    }
+
+    // file rename early checks
+    if (!srcMetadata.isDir()) {
+      if (srcKey.equals(dstKey)) {
+        // rename file to self is OK
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Renaming file to itself. This is allowed and is treated as no-op. path="
+              + src);
+        }
+        return true;
+      }
+    }
+
+    // More validations..
+    // If target is dir but target already exists, alter the dst to be a
+    // subfolder.
+    // eg move("/a/file.txt", "/b") where "/b" already exists causes the target
+    // to be "/c/file.txt
+    if (dstMetadata != null && dstMetadata.isDir()) {
+      dstKey = pathToKey(makeAbsolute(new Path(dst, src.getName())));
+      // Best would be to update dstMetadata, but it is not used further, so set
+      // it to null and skip the additional cost
+      dstMetadata = null;
+      // dstMetadata = store.retrieveMetadata(dstKey);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Destination " + dst
+            + " is a directory, adjusted the destination to be " + dstKey);
+      }
+
+      // rename dir to self is an error
+      if (srcKey.equals(dstKey)) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Renaming directory to itself is disallowed. path=" + src);
+        }
+        return false;
+      }
+
+    } else if (dstMetadata != null) {
+      // Otherwise, attempting to overwrite a file is error
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Destination " + dst
+            + " is an already existing file, failing the rename.");
+      }
+      return false;
+    } else {
+      // Either dir or file and target doesn't exist.. Check that the parent
+      // directory exists.
+      FileMetadata parentOfDestMetadata = store
+          .retrieveMetadata(pathToKey(absoluteDst.getParent()));
+      if (parentOfDestMetadata == null) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Parent of the destination " + dst
+              + " doesn't exist, failing the rename.");
+        }
+        return false;
+      } else if (!parentOfDestMetadata.isDir()) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Parent of the destination " + dst
+              + " is a file, failing the rename.");
+        }
+        return false;
+      }
+    }
+
+    // Validations complete, do the move.
+    if (!srcMetadata.isDir()) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Source " + src + " found as a file, renaming.");
+      }
+      store.rename(srcKey, dstKey);
+    } else {
+      // Move everything inside the folder.
+      String priorLastKey = null;
+
+      // Calculate the index of the part of the string to be moved. That
+      // is everything on the path up to the folder name.
+      do {
+        // List all blobs rooted at the source folder.
+        PartialListing listing = store.listAll(srcKey, AZURE_LIST_ALL,
+            AZURE_UNBOUNDED_DEPTH, priorLastKey);
+
+        // Rename all the files in the folder.
+        for (FileMetadata file : listing.getFiles()) {
+          // Rename all materialized entries under the folder to point to the
+          // final destination.
+          if (file.getBlobMaterialization() == BlobMaterialization.Explicit) {
+            String srcName = file.getKey();
+            String suffix = srcName.substring(srcKey.length());
+            String dstName = dstKey + suffix;
+            store.rename(srcName, dstName);
+          }
+        }
+        priorLastKey = listing.getPriorLastKey();
+      } while (priorLastKey != null);
+      // Rename the top level empty blob for the folder.
+      if (srcMetadata.getBlobMaterialization() == BlobMaterialization.Explicit) {
+        store.rename(srcKey, dstKey);
+      }
+    }
+
+    // Update both source and destination parent folder last modified time.
+    Path srcParent = makeAbsolute(keyToPath(srcKey)).getParent();
+    if (srcParent != null && srcParent.getParent() != null) { // not root
+      String srcParentKey = pathToKey(srcParent);
+
+      // ensure the srcParent is a materialized folder
+      FileMetadata srcParentMetadata = store.retrieveMetadata(srcParentKey);
+      if (srcParentMetadata.isDir()
+          && srcParentMetadata.getBlobMaterialization() == BlobMaterialization.Implicit) {
+        store.storeEmptyFolder(srcParentKey,
+            createPermissionStatus(FsPermission.getDefault()));
+      }
+
+      store.updateFolderLastModifiedTime(srcParentKey);
+    }
+
+    Path destParent = makeAbsolute(keyToPath(dstKey)).getParent();
+    if (destParent != null && destParent.getParent() != null) { // not root
+      String dstParentKey = pathToKey(destParent);
+
+      // ensure the dstParent is a materialized folder
+      FileMetadata dstParentMetadata = store.retrieveMetadata(dstParentKey);
+      if (dstParentMetadata.isDir()
+          && dstParentMetadata.getBlobMaterialization() == BlobMaterialization.Implicit) {
+        store.storeEmptyFolder(dstParentKey,
+            createPermissionStatus(FsPermission.getDefault()));
+      }
+
+      store.updateFolderLastModifiedTime(dstParentKey);
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Renamed " + src + " to " + dst + " successfully.");
+    }
+    return true;
+  }
+
+  /**
+   * Return an array containing hostnames, offset and size of portions of the
+   * given file. For WASB we'll just lie and give fake hosts to make sure we get
+   * many splits in MR jobs.
+   */
+  @Override
+  public BlockLocation[] getFileBlockLocations(FileStatus file, long start,
+      long len) throws IOException {
+    if (file == null) {
+      return null;
+    }
+
+    if ((start < 0) || (len < 0)) {
+      throw new IllegalArgumentException("Invalid start or len parameter");
+    }
+
+    if (file.getLen() < start) {
+      return new BlockLocation[0];
+    }
+    final String blobLocationHost = getConf().get(
+        AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME,
+        AZURE_BLOCK_LOCATION_HOST_DEFAULT);
+    final String[] name = { blobLocationHost };
+    final String[] host = { blobLocationHost };
+    long blockSize = file.getBlockSize();
+    if (blockSize <= 0) {
+      throw new IllegalArgumentException(
+          "The block size for the given file is not a positive number: "
+              + blockSize);
+    }
+    int numberOfLocations = (int) (len / blockSize)
+        + ((len % blockSize == 0) ? 0 : 1);
+    BlockLocation[] locations = new BlockLocation[numberOfLocations];
+    for (int i = 0; i < locations.length; i++) {
+      long currentOffset = start + (i * blockSize);
+      long currentLength = Math.min(blockSize, start + len - currentOffset);
+      locations[i] = new BlockLocation(name, host, currentOffset, currentLength);
+    }
+    return locations;
+  }
+
+  /**
+   * Set the working directory to the given directory.
+   */
+  @Override
+  public void setWorkingDirectory(Path newDir) {
+    workingDir = makeAbsolute(newDir);
+  }
+
+  @Override
+  public Path getWorkingDirectory() {
+    return workingDir;
+  }
+
+  @Override
+  public void setPermission(Path p, FsPermission permission) throws IOException {
+    Path absolutePath = makeAbsolute(p);
+    String key = pathToKey(absolutePath);
+    FileMetadata metadata = store.retrieveMetadata(key);
+    if (metadata == null) {
+      throw new FileNotFoundException("File doesn't exist: " + p);
+    }
+    permission = applyUMask(permission,
+        metadata.isDir() ? UMaskApplyMode.ChangeExistingDirectory
+            : UMaskApplyMode.ChangeExistingFile);
+    if (metadata.getBlobMaterialization() == BlobMaterialization.Implicit) {
+      // It's an implicit folder, need to materialize it.
+      store.storeEmptyFolder(key, createPermissionStatus(permission));
+    } else if (!metadata.getPermissionStatus().getPermission()
+        .equals(permission)) {
+      store.changePermissionStatus(key, new PermissionStatus(metadata
+          .getPermissionStatus().getUserName(), metadata.getPermissionStatus()
+          .getGroupName(), permission));
+    }
+  }
+
+  @Override
+  public void setOwner(Path p, String username, String groupname)
+      throws IOException {
+    Path absolutePath = makeAbsolute(p);
+    String key = pathToKey(absolutePath);
+    FileMetadata metadata = store.retrieveMetadata(key);
+    if (metadata == null) {
+      throw new FileNotFoundException("File doesn't exist: " + p);
+    }
+    PermissionStatus newPermissionStatus = new PermissionStatus(
+        username == null ? metadata.getPermissionStatus().getUserName()
+            : username, groupname == null ? metadata.getPermissionStatus()
+            .getGroupName() : groupname, metadata.getPermissionStatus()
+            .getPermission());
+    if (metadata.getBlobMaterialization() == BlobMaterialization.Implicit) {
+      // It's an implicit folder, need to materialize it.
+      store.storeEmptyFolder(key, newPermissionStatus);
+    } else {
+      store.changePermissionStatus(key, newPermissionStatus);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    // Call the base close() to close any resources there.
+    super.close();
+    // Close the store
+    store.close();
+  }
+
+  /**
+   * A handler that defines what to do with blobs whose upload was interrupted.
+   */
+  private abstract class DanglingFileHandler {
+    abstract void handleFile(FileMetadata file, FileMetadata tempFile)
+        throws IOException;
+  }
+
+  /**
+   * Handler implementation for just deleting dangling files and cleaning them
+   * up.
+   */
+  private class DanglingFileDeleter extends DanglingFileHandler {
+    @Override
+    void handleFile(FileMetadata file, FileMetadata tempFile)
+        throws IOException {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Deleting dangling file " + file.getKey());
+      }
+      store.delete(file.getKey());
+      store.delete(tempFile.getKey());
+    }
+  }
+
+  /**
+   * Handler implementation for just moving dangling files to recovery location
+   * (/lost+found).
+   */
+  private class DanglingFileRecoverer extends DanglingFileHandler {
+    private final Path destination;
+
+    DanglingFileRecoverer(Path destination) {
+      this.destination = destination;
+    }
+
+    @Override
+    void handleFile(FileMetadata file, FileMetadata tempFile)
+        throws IOException {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Recovering " + file.getKey());
+      }
+      // Move to the final destination
+      String finalDestinationKey = pathToKey(new Path(destination,
+          file.getKey()));
+      store.rename(tempFile.getKey(), finalDestinationKey);
+      if (!finalDestinationKey.equals(file.getKey())) {
+        // Delete the empty link file now that we've restored it.
+        store.delete(file.getKey());
+      }
+    }
+  }
+
+  /**
+   * Check if a path has colons in its name
+   */
+  private boolean containsColon(Path p) {
+    return p.toUri().getPath().toString().contains(":");
+  }
+
+  /**
+   * Implements recover and delete (-move and -delete) behaviors for handling
+   * dangling files (blobs whose upload was interrupted).
+   * 
+   * @param root
+   *          The root path to check from.
+   * @param handler
+   *          The handler that deals with dangling files.
+   */
+  private void handleFilesWithDanglingTempData(Path root,
+      DanglingFileHandler handler) throws IOException {
+    // Calculate the cut-off for when to consider a blob to be dangling.
+    long cutoffForDangling = new Date().getTime()
+        - getConf().getInt(AZURE_TEMP_EXPIRY_PROPERTY_NAME,
+            AZURE_TEMP_EXPIRY_DEFAULT) * 1000;
+    // Go over all the blobs under the given root and look for blobs to
+    // recover.
+    String priorLastKey = null;
+    do {
+      PartialListing listing = store.listAll(pathToKey(root), AZURE_LIST_ALL,
+          AZURE_UNBOUNDED_DEPTH, priorLastKey);
+
+      for (FileMetadata file : listing.getFiles()) {
+        if (!file.isDir()) { // We don't recover directory blobs
+          // See if this blob has a link in it (meaning it's a place-holder
+          // blob for when the upload to the temp blob is complete).
+          String link = store.getLinkInFileMetadata(file.getKey());
+          if (link != null) {
+            // It has a link, see if the temp blob it is pointing to is
+            // existent and old enough to be considered dangling.
+            FileMetadata linkMetadata = store.retrieveMetadata(link);
+            if (linkMetadata != null
+                && linkMetadata.getLastModified() >= cutoffForDangling) {
+              // Found one!
+              handler.handleFile(file, linkMetadata);
+            }
+          }
+        }
+      }
+      priorLastKey = listing.getPriorLastKey();
+    } while (priorLastKey != null);
+  }
+
+  /**
+   * Looks under the given root path for any blob that are left "dangling",
+   * meaning that they are place-holder blobs that we created while we upload
+   * the data to a temporary blob, but for some reason we crashed in the middle
+   * of the upload and left them there. If any are found, we move them to the
+   * destination given.
+   * 
+   * @param root
+   *          The root path to consider.
+   * @param destination
+   *          The destination path to move any recovered files to.
+   * @throws IOException
+   */
+  public void recoverFilesWithDanglingTempData(Path root, Path destination)
+      throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Recovering files with dangling temp data in " + root);
+    }
+    handleFilesWithDanglingTempData(root,
+        new DanglingFileRecoverer(destination));
+  }
+
+  /**
+   * Looks under the given root path for any blob that are left "dangling",
+   * meaning that they are place-holder blobs that we created while we upload
+   * the data to a temporary blob, but for some reason we crashed in the middle
+   * of the upload and left them there. If any are found, we delete them.
+   * 
+   * @param root
+   *          The root path to consider.
+   * @throws IOException
+   */
+  public void deleteFilesWithDanglingTempData(Path root) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Deleting files with dangling temp data in " + root);
+    }
+    handleFilesWithDanglingTempData(root, new DanglingFileDeleter());
+  }
+
+  /**
+   * Encode the key with a random prefix for load balancing in Azure storage.
+   * Upload data to a random temporary file then do storage side renaming to
+   * recover the original key.
+   * 
+   * @param aKey
+   * @param numBuckets
+   * @return Encoded version of the original key.
+   */
+  private static String encodeKey(String aKey) {
+    // Get the tail end of the key name.
+    //
+    String fileName = aKey.substring(aKey.lastIndexOf(Path.SEPARATOR) + 1,
+        aKey.length());
+
+    // Construct the randomized prefix of the file name. The prefix ensures the
+    // file always drops into the same folder but with a varying tail key name.
+    String filePrefix = AZURE_TEMP_FOLDER + Path.SEPARATOR
+        + UUID.randomUUID().toString();
+
+    // Concatenate the randomized prefix with the tail of the key name.
+    String randomizedKey = filePrefix + fileName;
+
+    // Return to the caller with the randomized key.
+    return randomizedKey;
+  }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
new file mode 100644
index 0000000..0fb3c22
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
@@ -0,0 +1,97 @@
+/**
+ * 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.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.net.URI;
+import java.util.Date;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * <p>
+ * An abstraction for a key-based {@link File} store.
+ * </p>
+ */
+@InterfaceAudience.Private
+interface NativeFileSystemStore {
+
+  void initialize(URI uri, Configuration conf) throws IOException;
+
+  void storeEmptyFolder(String key, PermissionStatus permissionStatus)
+      throws AzureException;
+
+  FileMetadata retrieveMetadata(String key) throws IOException;
+
+  DataInputStream retrieve(String key) throws IOException;
+
+  DataInputStream retrieve(String key, long byteRangeStart) throws IOException;
+
+  DataOutputStream storefile(String key, PermissionStatus permissionStatus)
+      throws AzureException;
+
+  void storeEmptyLinkFile(String key, String tempBlobKey,
+      PermissionStatus permissionStatus) throws AzureException;
+
+  String getLinkInFileMetadata(String key) throws AzureException;
+
+  PartialListing list(String prefix, final int maxListingCount,
+      final int maxListingDepth) throws IOException;
+
+  PartialListing list(String prefix, final int maxListingCount,
+      final int maxListingDepth, String priorLastKey) throws IOException;
+
+  PartialListing listAll(String prefix, final int maxListingCount,
+      final int maxListingDepth, String priorLastKey) throws IOException;
+
+  void changePermissionStatus(String key, PermissionStatus newPermission)
+      throws AzureException;
+
+  void delete(String key) throws IOException;
+
+  void rename(String srcKey, String dstKey) throws IOException;
+
+  /**
+   * Delete all keys with the given prefix. Used for testing.
+   * 
+   * @throws IOException
+   */
+  @VisibleForTesting
+  void purge(String prefix) throws IOException;
+
+  /**
+   * Diagnostic method to dump state to the console.
+   * 
+   * @throws IOException
+   */
+  void dump() throws IOException;
+
+  void close();
+
+  void updateFolderLastModifiedTime(String key) throws AzureException;
+
+  void updateFolderLastModifiedTime(String key, Date lastModified)
+      throws AzureException;
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PartialListing.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PartialListing.java
new file mode 100644
index 0000000..9e49de8
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/PartialListing.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
+ *
+ *     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.classification.InterfaceAudience;
+
+/**
+ * <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
+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;
+  }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfThrottlingIntercept.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfThrottlingIntercept.java
new file mode 100644
index 0000000..25f2883
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SelfThrottlingIntercept.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 java.net.HttpURLConnection;
+import java.util.Date;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import com.microsoft.windowsazure.storage.OperationContext;
+import com.microsoft.windowsazure.storage.RequestResult;
+import com.microsoft.windowsazure.storage.ResponseReceivedEvent;
+import com.microsoft.windowsazure.storage.SendingRequestEvent;
+import com.microsoft.windowsazure.storage.StorageEvent;
+
+/*
+ * Self throttling is implemented by hooking into send & response callbacks 
+ * One instance of this class is created per operationContext so each blobUpload/blobDownload/etc.
+ * 
+ * Self throttling only applies to 2nd and subsequent packets of an operation.  This is a simple way to 
+ * ensure it only affects bulk transfers and not every tiny request.
+ * 
+ * A blobDownload will involve sequential packet transmissions and so there are no concurrency concerns
+ * A blobUpload will generally involve concurrent upload worker threads that share one operationContext and one throttling instance.
+ *   -- we do not track the latencies for each worker thread as they are doing similar work and will rarely collide in practice.  
+ *   -- concurrent access to lastE2Edelay must be protected.  
+ *       -- volatile is necessary and should be sufficient to protect simple access to primitive values (java 1.5 onwards) 
+ *       -- synchronized{} blocks are also used to be conservative and for easier maintenance.
+ *   
+ * If an operation were to perform concurrent GETs and PUTs there is the possibility of getting confused regarding
+ * whether lastE2Edelay was a read or write measurement.  This scenario does not occur.
+ *
+ * readFactor  = target read throughput as factor of unrestricted throughput.
+ * writeFactor = target write throughput as factor of unrestricted throughput.
+ * 
+ * As we introduce delays it is important to only measure the actual E2E latency and not the augmented latency
+ * To achieve this, we fiddle the 'startDate' of the transfer tracking object.
+ */
+
+
+/**
+ * 
+ * Introduces delays in our Azure traffic to prevent overrunning the server-side throttling limits.
+ *
+ */
+@InterfaceAudience.Private
+public class SelfThrottlingIntercept {
+  public static final Log LOG = LogFactory
+      .getLog(SelfThrottlingIntercept.class);
+
+  private final float readFactor;
+  private final float writeFactor;
+
+  // Concurrency: access to non-final members must be thread-safe
+  private long lastE2Elatency;
+
+  public SelfThrottlingIntercept(OperationContext operationContext,
+      float readFactor, float writeFactor) {
+    this.readFactor = readFactor;
+    this.writeFactor = writeFactor;
+  }
+
+  public static void hook(OperationContext operationContext, float readFactor,
+      float writeFactor) {
+
+    SelfThrottlingIntercept throttler = new SelfThrottlingIntercept(
+        operationContext, readFactor, writeFactor);
+    ResponseReceivedListener responseListener = throttler.new ResponseReceivedListener();
+    SendingRequestListener sendingListener = throttler.new SendingRequestListener();
+
+    operationContext.getResponseReceivedEventHandler().addListener(
+        responseListener);
+    operationContext.getSendingRequestEventHandler().addListener(
+        sendingListener);
+  }
+
+  public void responseReceived(ResponseReceivedEvent event) {
+    RequestResult result = event.getRequestResult();
+    Date startDate = result.getStartDate();
+    Date stopDate = result.getStopDate();
+    long elapsed = stopDate.getTime() - startDate.getTime();
+
+    synchronized (this) {
+      this.lastE2Elatency = elapsed;
+    }
+
+    if (LOG.isDebugEnabled()) {
+      int statusCode = result.getStatusCode();
+      String etag = result.getEtag();
+      HttpURLConnection urlConnection = (HttpURLConnection) event
+          .getConnectionObject();
+      int contentLength = urlConnection.getContentLength();
+      String requestMethod = urlConnection.getRequestMethod();
+      long threadId = Thread.currentThread().getId();
+      LOG.debug(String
+          .format(
+              "SelfThrottlingIntercept:: ResponseReceived: threadId=%d, Status=%d, Elapsed(ms)=%d, ETAG=%s, contentLength=%d, requestMethod=%s",
+              threadId, statusCode, elapsed, etag, contentLength, requestMethod));
+    }
+  }
+
+  public void sendingRequest(SendingRequestEvent sendEvent) {
+    long lastLatency;
+    boolean operationIsRead; // for logging
+    synchronized (this) {
+
+      lastLatency = this.lastE2Elatency;
+    }
+
+    float sleepMultiple;
+    HttpURLConnection urlConnection = (HttpURLConnection) sendEvent
+        .getConnectionObject();
+
+    // Azure REST API never uses POST, so PUT is a sufficient test for an
+    // upload.
+    if (urlConnection.getRequestMethod().equalsIgnoreCase("PUT")) {
+      operationIsRead = false;
+      sleepMultiple = (1 / writeFactor) - 1;
+    } else {
+      operationIsRead = true;
+      sleepMultiple = (1 / readFactor) - 1;
+    }
+
+    long sleepDuration = (long) (sleepMultiple * lastLatency);
+    if (sleepDuration < 0) {
+      sleepDuration = 0;
+    }
+
+    if (sleepDuration > 0) {
+      try {
+        // Thread.sleep() is not exact but it seems sufficiently accurate for
+        // our needs. If needed this could become a loop of small waits that
+        // tracks actual
+        // elapsed time.
+        Thread.sleep(sleepDuration);
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+      }
+
+      // reset to avoid counting the sleep against request latency
+      sendEvent.getRequestResult().setStartDate(new Date());
+    }
+
+    if (LOG.isDebugEnabled()) {
+      boolean isFirstRequest = (lastLatency == 0);
+      long threadId = Thread.currentThread().getId();
+      LOG.debug(String
+          .format(
+              " SelfThrottlingIntercept:: SendingRequest:   threadId=%d, requestType=%s, isFirstRequest=%b, sleepDuration=%d",
+              threadId, operationIsRead ? "read " : "write", isFirstRequest,
+              sleepDuration));
+    }
+  }
+
+  // simply forwards back to the main class.
+  // this is necessary as our main class cannot implement two base-classes.
+  @InterfaceAudience.Private
+  class SendingRequestListener extends StorageEvent<SendingRequestEvent> {
+
+    @Override
+    public void eventOccurred(SendingRequestEvent event) {
+      sendingRequest(event);
+    }
+  }
+
+  // simply forwards back to the main class.
+  // this is necessary as our main class cannot implement two base-classes.
+  @InterfaceAudience.Private
+  class ResponseReceivedListener extends StorageEvent<ResponseReceivedEvent> {
+
+    @Override
+    public void eventOccurred(ResponseReceivedEvent event) {
+      responseReceived(event);
+    }
+  }
+}
\ No newline at end of file
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SendRequestIntercept.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SendRequestIntercept.java
new file mode 100644
index 0000000..18f173e
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SendRequestIntercept.java
@@ -0,0 +1,166 @@
+/**
+ * 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.HttpURLConnection;
+import java.security.InvalidKeyException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import com.microsoft.windowsazure.storage.Constants.HeaderConstants;
+import com.microsoft.windowsazure.storage.OperationContext;
+import com.microsoft.windowsazure.storage.SendingRequestEvent;
+import com.microsoft.windowsazure.storage.StorageCredentials;
+import com.microsoft.windowsazure.storage.StorageEvent;
+import com.microsoft.windowsazure.storage.StorageException;
+
+/**
+ * Manages the lifetime of binding on the operation contexts to intercept send
+ * request events to Azure storage.
+ */
+@InterfaceAudience.Private
+public final class SendRequestIntercept extends StorageEvent<SendingRequestEvent> {
+
+  public static final Log LOG = LogFactory.getLog(SendRequestIntercept.class);
+
+  private static final String ALLOW_ALL_REQUEST_PRECONDITIONS = "*";
+  private final StorageCredentials storageCreds;
+  private final boolean allowConcurrentOOBIo;
+  private final OperationContext opContext;
+
+  /**
+   * Getter returning the storage account credentials.
+   * 
+   * @return storageCreds - account storage credentials.
+   */
+  private StorageCredentials getCredentials() {
+    return storageCreds;
+  }
+
+  /**
+   * Query if out-of-band I/Os are allowed.
+   * 
+   * return allowConcurrentOOBIo - true if OOB I/O is allowed, and false
+   * otherwise.
+   */
+  private boolean isOutOfBandIoAllowed() {
+    return allowConcurrentOOBIo;
+  }
+
+  /**
+   * Getter returning the operation context.
+   * 
+   * @return storageCreds - account storage credentials.
+   */
+  private OperationContext getOperationContext() {
+    return opContext;
+  }
+
+  /**
+   * Constructor for SendRequestThrottle.
+   * 
+   * @param storageCreds
+   *          - storage account credentials for signing packets.
+   * 
+   */
+  private SendRequestIntercept(StorageCredentials storageCreds,
+      boolean allowConcurrentOOBIo, OperationContext opContext) {
+    // Capture the send delay callback interface.
+    this.storageCreds = storageCreds;
+    this.allowConcurrentOOBIo = allowConcurrentOOBIo;
+    this.opContext = opContext;
+  }
+
+  /**
+   * Binds a new lister to the operation context so the WASB file system can
+   * appropriately intercept sends. By allowing concurrent OOB I/Os, we bypass
+   * the blob immutability check when reading streams.
+   * 
+   * @param opContext
+   *          The operation context to bind to listener.
+   * 
+   * @param allowConcurrentOOBIo
+   *          True if reads are allowed with concurrent OOB writes.
+   */
+  public static void bind(StorageCredentials storageCreds,
+      OperationContext opContext, boolean allowConcurrentOOBIo) {
+    SendRequestIntercept sendListener = new SendRequestIntercept(storageCreds,
+        allowConcurrentOOBIo, opContext);
+    opContext.getSendingRequestEventHandler().addListener(sendListener);
+  }
+
+  /**
+   * Handler which processes the sending request event from Azure SDK. The
+   * handler simply sets reset the conditional header to make all read requests
+   * unconditional if reads with concurrent OOB writes are allowed.
+   * 
+   * @param sendEvent
+   *          - send event context from Windows Azure SDK.
+   */
+  @Override
+  public void eventOccurred(SendingRequestEvent sendEvent) {
+
+    if (!(sendEvent.getConnectionObject() instanceof HttpURLConnection)) {
+      // Pass if there is no HTTP connection associated with this send
+      // request.
+      return;
+    }
+
+    // Capture the HTTP URL connection object and get size of the payload for
+    // the request.
+    HttpURLConnection urlConnection = (HttpURLConnection) sendEvent
+        .getConnectionObject();
+
+    // Determine whether this is a download request by checking that the request
+    // method
+    // is a "GET" operation.
+    if (urlConnection.getRequestMethod().equalsIgnoreCase("GET")
+        && isOutOfBandIoAllowed()) {
+      // If concurrent reads on OOB writes are allowed, reset the if-match
+      // condition on the conditional header.
+      urlConnection.setRequestProperty(HeaderConstants.IF_MATCH,
+          ALLOW_ALL_REQUEST_PRECONDITIONS);
+
+      // In the Java AzureSDK the packet is signed before firing the
+      // SendRequest. Setting
+      // the conditional packet header property changes the contents of the
+      // packet, therefore the packet has to be re-signed.
+      try {
+        // Sign the request. GET's have no payload so the content length is
+        // zero.
+        getCredentials().signBlobAndQueueRequest(urlConnection, -1L, getOperationContext());
+      } catch (InvalidKeyException e) {
+        // Log invalid key exception to track signing error before the send
+        // fails.
+        String errString = String.format(
+            "Received invalid key exception when attempting sign packet."
+                + " Cause: %s", e.getCause().toString());
+        LOG.error(errString);
+      } catch (StorageException e) {
+        // Log storage exception to track signing error before the call fails.
+        String errString = String.format(
+            "Received storage exception when attempting to sign packet."
+                + " Cause: %s", e.getCause().toString());
+        LOG.error(errString);
+      }
+    }
+  }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/ShellDecryptionKeyProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/ShellDecryptionKeyProvider.java
new file mode 100644
index 0000000..2ce8ebd
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/ShellDecryptionKeyProvider.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
+ *
+ *     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.util.Arrays;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Shell;
+
+/**
+ * Shell decryption key provider which invokes an external script that will
+ * perform the key decryption.
+ */
+@InterfaceAudience.Private
+public class ShellDecryptionKeyProvider extends SimpleKeyProvider {
+  static final String KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT = "fs.azure.shellkeyprovider.script";
+
+  @Override
+  public String getStorageAccountKey(String accountName, Configuration conf)
+      throws KeyProviderException {
+    String envelope = super.getStorageAccountKey(accountName, conf);
+
+    final String command = conf.get(KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT);
+    if (command == null) {
+      throw new KeyProviderException(
+          "Script path is not specified via fs.azure.shellkeyprovider.script");
+    }
+
+    String[] cmd = command.split(" ");
+    String[] cmdWithEnvelope = Arrays.copyOf(cmd, cmd.length + 1);
+    cmdWithEnvelope[cmdWithEnvelope.length - 1] = envelope;
+
+    String decryptedKey = null;
+    try {
+      decryptedKey = Shell.execCommand(cmdWithEnvelope);
+    } catch (IOException ex) {
+      throw new KeyProviderException(ex);
+    }
+
+    // trim any whitespace
+    return decryptedKey.trim();
+  }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SimpleKeyProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SimpleKeyProvider.java
new file mode 100644
index 0000000..ef44a85
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SimpleKeyProvider.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Key provider that simply returns the storage account key from the
+ * configuration as plaintext.
+ */
+@InterfaceAudience.Private
+public class SimpleKeyProvider implements KeyProvider {
+
+  protected static final String KEY_ACCOUNT_KEY_PREFIX = "fs.azure.account.key.";
+
+  @Override
+  public String getStorageAccountKey(String accountName, Configuration conf)
+      throws KeyProviderException {
+    return conf.get(getStorageAccountKeyName(accountName));
+  }
+
+  protected String getStorageAccountKeyName(String accountName) {
+    return KEY_ACCOUNT_KEY_PREFIX + accountName;
+  }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
new file mode 100644
index 0000000..87cef86
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
@@ -0,0 +1,566 @@
+/**
+ * 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.URI;
+import java.net.URISyntaxException;
+import java.util.EnumSet;
+import java.util.HashMap;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import com.microsoft.windowsazure.storage.CloudStorageAccount;
+import com.microsoft.windowsazure.storage.OperationContext;
+import com.microsoft.windowsazure.storage.RetryPolicyFactory;
+import com.microsoft.windowsazure.storage.StorageCredentials;
+import com.microsoft.windowsazure.storage.StorageException;
+import com.microsoft.windowsazure.storage.blob.BlobListingDetails;
+import com.microsoft.windowsazure.storage.blob.BlobProperties;
+import com.microsoft.windowsazure.storage.blob.BlobRequestOptions;
+import com.microsoft.windowsazure.storage.blob.CopyState;
+import com.microsoft.windowsazure.storage.blob.ListBlobItem;
+
+/**
+ * This is a very thin layer over the methods exposed by the Windows Azure
+ * Storage SDK that we need for WASB implementation. This base class has a real
+ * implementation that just simply redirects to the SDK, and a memory-backed one
+ * that's used for unit tests.
+ * 
+ * IMPORTANT: all the methods here must remain very simple redirects since code
+ * written here can't be properly unit tested.
+ */
+@InterfaceAudience.Private
+abstract class StorageInterface {
+
+  /**
+   * Sets the timeout to use when making requests to the storage service.
+   * <p>
+   * The server timeout interval begins at the time that the complete request
+   * has been received by the service, and the server begins processing the
+   * response. If the timeout interval elapses before the response is returned
+   * to the client, the operation times out. The timeout interval resets with
+   * each retry, if the request is retried.
+   * 
+   * The default timeout interval for a request made via the service client is
+   * 90 seconds. You can change this value on the service client by setting this
+   * property, so that all subsequent requests made via the service client will
+   * use the new timeout interval. You can also change this value for an
+   * individual request, by setting the
+   * {@link RequestOptions#timeoutIntervalInMs} property.
+   * 
+   * If you are downloading a large blob, you should increase the value of the
+   * timeout beyond the default value.
+   * 
+   * @param timeoutInMs
+   *          The timeout, in milliseconds, to use when making requests to the
+   *          storage service.
+   */
+  public abstract void setTimeoutInMs(int timeoutInMs);
+
+  /**
+   * Sets the RetryPolicyFactory object to use when making service requests.
+   * 
+   * @param retryPolicyFactory
+   *          the RetryPolicyFactory object to use when making service requests.
+   */
+  public abstract void setRetryPolicyFactory(
+      final RetryPolicyFactory retryPolicyFactory);
+
+  /**
+   * Creates a new Blob service client.
+   * 
+   */
+  public abstract void createBlobClient(CloudStorageAccount account);
+
+  /**
+   * Creates an instance of the <code>CloudBlobClient</code> class using the
+   * specified Blob service endpoint.
+   * 
+   * @param baseUri
+   *          A <code>java.net.URI</code> object that represents the Blob
+   *          service endpoint used to create the client.
+   */
+  public abstract void createBlobClient(URI baseUri);
+
+  /**
+   * Creates an instance of the <code>CloudBlobClient</code> class using the
+   * specified Blob service endpoint and account credentials.
+   * 
+   * @param baseUri
+   *          A <code>java.net.URI</code> object that represents the Blob
+   *          service endpoint used to create the client.
+   * @param credentials
+   *          A {@link StorageCredentials} object that represents the account
+   *          credentials.
+   */
+  public abstract void createBlobClient(URI baseUri,
+      StorageCredentials credentials);
+
+  /**
+   * Returns the credentials for the Blob service, as configured for the storage
+   * account.
+   * 
+   * @return A {@link StorageCredentials} object that represents the credentials
+   *         for this storage account.
+   */
+  public abstract StorageCredentials getCredentials();
+
+  /**
+   * Returns a reference to a {@link CloudBlobContainerWrapper} object that
+   * represents the cloud blob container for the specified address.
+   * 
+   * @param name
+   *          A <code>String</code> that represents the name of the container.
+   * @return A {@link CloudBlobContainerWrapper} object that represents a
+   *         reference to the cloud blob container.
+   * 
+   * @throws URISyntaxException
+   *           If the resource URI is invalid.
+   * @throws StorageException
+   *           If a storage service error occurred.
+   */
+  public abstract CloudBlobContainerWrapper getContainerReference(String name)
+      throws URISyntaxException, StorageException;
+
+  /**
+   * A thin wrapper over the {@link CloudBlobDirectory} class that simply
+   * redirects calls to the real object except in unit tests.
+   */
+  @InterfaceAudience.Private
+  public abstract static class CloudBlobDirectoryWrapper implements
+      ListBlobItem {
+    /**
+     * Returns the URI for this directory.
+     * 
+     * @return A <code>java.net.URI</code> object that represents the URI for
+     *         this directory.
+     */
+    public abstract URI getUri();
+
+    /**
+     * Returns an enumerable collection of blob items whose names begin with the
+     * specified prefix, using the specified flat or hierarchical option,
+     * listing details options, request options, and operation context.
+     * 
+     * @param prefix
+     *          A <code>String</code> that represents the prefix of the blob
+     *          name.
+     * @param useFlatBlobListing
+     *          <code>true</code> to indicate that the returned list will be
+     *          flat; <code>false</code> to indicate that the returned list will
+     *          be hierarchical.
+     * @param listingDetails
+     *          A <code>java.util.EnumSet</code> object that contains
+     *          {@link BlobListingDetails} values that indicate whether
+     *          snapshots, metadata, and/or uncommitted blocks are returned.
+     *          Committed blocks are always returned.
+     * @param options
+     *          A {@link BlobRequestOptions} object that specifies any
+     *          additional options for the request. Specifying <code>null</code>
+     *          will use the default request options from the associated service
+     *          client ( {@link CloudBlobClient}).
+     * @param opContext
+     *          An {@link OperationContext} object that represents the context
+     *          for the current operation. This object is used to track requests
+     *          to the storage service, and to provide additional runtime
+     *          information about the operation.
+     * 
+     * @return An enumerable collection of {@link ListBlobItem} objects that
+     *         represent the block items whose names begin with the specified
+     *         prefix in this directory.
+     * 
+     * @throws StorageException
+     *           If a storage service error occurred.
+     * @throws URISyntaxException
+     *           If the resource URI is invalid.
+     */
+    public abstract Iterable<ListBlobItem> listBlobs(String prefix,
+        boolean useFlatBlobListing, EnumSet<BlobListingDetails> listingDetails,
+        BlobRequestOptions options, OperationContext opContext)
+        throws URISyntaxException, StorageException;
+  }
+
+  /**
+   * A thin wrapper over the {@link CloudBlobContainer} class that simply
+   * redirects calls to the real object except in unit tests.
+   */
+  @InterfaceAudience.Private
+  public abstract static class CloudBlobContainerWrapper {
+    /**
+     * Returns the name of the container.
+     * 
+     * @return A <code>String</code> that represents the name of the container.
+     */
+    public abstract String getName();
+
+    /**
+     * Returns a value that indicates whether the container exists, using the
+     * specified operation context.
+     * 
+     * @param opContext
+     *          An {@link OperationContext} object that represents the context
+     *          for the current operation. This object is used to track requests
+     *          to the storage service, and to provide additional runtime
+     *          information about the operation.
+     * 
+     * @return <code>true</code> if the container exists, otherwise
+     *         <code>false</code>.
+     * 
+     * @throws StorageException
+     *           If a storage service error occurred.
+     */
+    public abstract boolean exists(OperationContext opContext)
+        throws StorageException;
+
+    /**
+     * Returns the metadata for the container.
+     * 
+     * @return A <code>java.util.HashMap</code> object that represents the
+     *         metadata for the container.
+     */
+    public abstract HashMap<String, String> getMetadata();
+
+    /**
+     * Sets the metadata for the container.
+     * 
+     * @param metadata
+     *          A <code>java.util.HashMap</code> object that represents the
+     *          metadata being assigned to the container.
+     */
+    public abstract void setMetadata(HashMap<String, String> metadata);
+
+    /**
+     * Downloads the container's attributes, which consist of metadata and
+     * properties, using the specified operation context.
+     * 
+     * @param opContext
+     *          An {@link OperationContext} object that represents the context
+     *          for the current operation. This object is used to track requests
+     *          to the storage service, and to provide additional runtime
+     *          information about the operation.
+     * 
+     * @throws StorageException
+     *           If a storage service error occurred.
+     */
+    public abstract void downloadAttributes(OperationContext opContext)
+        throws StorageException;
+
+    /**
+     * Uploads the container's metadata using the specified operation context.
+     * 
+     * @param opContext
+     *          An {@link OperationContext} object that represents the context
+     *          for the current operation. This object is used to track requests
+     *          to the storage service, and to provide additional runtime
+     *          information about the operation.
+     * 
+     * @throws StorageException
+     *           If a storage service error occurred.
+     */
+    public abstract void uploadMetadata(OperationContext opContext)
+        throws StorageException;
+
+    /**
+     * Creates the container using the specified operation context.
+     * 
+     * @param opContext
+     *          An {@link OperationContext} object that represents the context
+     *          for the current operation. This object is used to track requests
+     *          to the storage service, and to provide additional runtime
+     *          information about the operation.
+     * 
+     * @throws StorageException
+     *           If a storage service error occurred.
+     */
+    public abstract void create(OperationContext opContext)
+        throws StorageException;
+
+    /**
+     * Returns a wrapper for a CloudBlobDirectory.
+     * 
+     * @param relativePath
+     *          A <code>String</code> that represents the name of the directory,
+     *          relative to the container
+     * 
+     * @throws StorageException
+     *           If a storage service error occurred.
+     * 
+     * @throws URISyntaxException
+     *           If URI syntax exception occurred.
+     */
+    public abstract CloudBlobDirectoryWrapper getDirectoryReference(
+        String relativePath) throws URISyntaxException, StorageException;
+
+    /**
+     * Returns a wrapper for a CloudBlockBlob.
+     * 
+     * @param relativePath
+     *          A <code>String</code> that represents the name of the blob,
+     *          relative to the container
+     * 
+     * @throws StorageException
+     *           If a storage service error occurred.
+     * 
+     * @throws URISyntaxException
+     *           If URI syntax exception occurred.
+     */
+    public abstract CloudBlockBlobWrapper getBlockBlobReference(
+        String relativePath) throws URISyntaxException, StorageException;
+  }
+
+  /**
+   * A thin wrapper over the {@link CloudBlockBlob} class that simply redirects
+   * calls to the real object except in unit tests.
+   */
+  @InterfaceAudience.Private
+  public abstract static class CloudBlockBlobWrapper implements ListBlobItem {
+    /**
+     * Returns the URI for this blob.
+     * 
+     * @return A <code>java.net.URI</code> object that represents the URI for
+     *         the blob.
+     */
+    public abstract URI getUri();
+
+    /**
+     * Returns the metadata for the blob.
+     * 
+     * @return A <code>java.util.HashMap</code> object that represents the
+     *         metadata for the blob.
+     */
+    public abstract HashMap<String, String> getMetadata();
+
+    /**
+     * Sets the metadata for the blob.
+     * 
+     * @param metadata
+     *          A <code>java.util.HashMap</code> object that contains the
+     *          metadata being assigned to the blob.
+     */
+    public abstract void setMetadata(HashMap<String, String> metadata);
+
+    /**
+     * Copies an existing blob's contents, properties, and metadata to this
+     * instance of the <code>CloudBlob</code> class, using the specified
+     * operation context.
+     * 
+     * @param sourceBlob
+     *          A <code>CloudBlob</code> object that represents the source blob
+     *          to copy.
+     * @param opContext
+     *          An {@link OperationContext} object that represents the context
+     *          for the current operation. This object is used to track requests
+     *          to the storage service, and to provide additional runtime
+     *          information about the operation.
+     * 
+     * @throws StorageException
+     *           If a storage service error occurred.
+     * @throws URISyntaxException
+     * 
+     */
+    public abstract void startCopyFromBlob(CloudBlockBlobWrapper sourceBlob,
+        OperationContext opContext) throws StorageException, URISyntaxException;
+
+    /**
+     * Returns the blob's copy state.
+     * 
+     * @return A {@link CopyState} object that represents the copy state of the
+     *         blob.
+     */
+    public abstract CopyState getCopyState();
+
+    /**
+     * Deletes the blob using the specified operation context.
+     * <p>
+     * A blob that has snapshots cannot be deleted unless the snapshots are also
+     * deleted. If a blob has snapshots, use the
+     * {@link DeleteSnapshotsOption#DELETE_SNAPSHOTS_ONLY} or
+     * {@link DeleteSnapshotsOption#INCLUDE_SNAPSHOTS} value in the
+     * <code>deleteSnapshotsOption</code> parameter to specify how the snapshots
+     * should be handled when the blob is deleted.
+     * 
+     * @param opContext
+     *          An {@link OperationContext} object that represents the context
+     *          for the current operation. This object is used to track requests
+     *          to the storage service, and to provide additional runtime
+     *          information about the operation.
+     * 
+     * @throws StorageException
+     *           If a storage service error occurred.
+     */
+    public abstract void delete(OperationContext opContext)
+        throws StorageException;
+
+    /**
+     * Checks to see if the blob exists, using the specified operation context.
+     * 
+     * @param opContext
+     *          An {@link OperationContext} object that represents the context
+     *          for the current operation. This object is used to track requests
+     *          to the storage service, and to provide additional runtime
+     *          information about the operation.
+     * 
+     * @return <code>true</code> if the blob exists, other wise
+     *         <code>false</code>.
+     * 
+     * @throws StorageException
+     *           f a storage service error occurred.
+     */
+    public abstract boolean exists(OperationContext opContext)
+        throws StorageException;
+
+    /**
+     * Populates a blob's properties and metadata using the specified operation
+     * context.
+     * <p>
+     * This method populates the blob's system properties and user-defined
+     * metadata. Before reading a blob's properties or metadata, call this
+     * method or its overload to retrieve the latest values for the blob's
+     * properties and metadata from the Windows Azure storage service.
+     * 
+     * @param opContext
+     *          An {@link OperationContext} object that represents the context
+     *          for the current operation. This object is used to track requests
+     *          to the storage service, and to provide additional runtime
+     *          information about the operation.
+     * 
+     * @throws StorageException
+     *           If a storage service error occurred.
+     */
+    public abstract void downloadAttributes(OperationContext opContext)
+        throws StorageException;
+
+    /**
+     * Returns the blob's properties.
+     * 
+     * @return A {@link BlobProperties} object that represents the properties of
+     *         the blob.
+     */
+    public abstract BlobProperties getProperties();
+
+    /**
+     * Opens a blob input stream to download the blob using the specified
+     * operation context.
+     * <p>
+     * Use {@link CloudBlobClient#setStreamMinimumReadSizeInBytes} to configure
+     * the read size.
+     * 
+     * @param opContext
+     *          An {@link OperationContext} object that represents the context
+     *          for the current operation. This object is used to track requests
+     *          to the storage service, and to provide additional runtime
+     *          information about the operation.
+     * 
+     * @return An <code>InputStream</code> object that represents the stream to
+     *         use for reading from the blob.
+     * 
+     * @throws StorageException
+     *           If a storage service error occurred.
+     */
+    public abstract InputStream openInputStream(BlobRequestOptions options,
+        OperationContext opContext) throws StorageException;
+
+    /**
+     * Creates and opens an output stream to write data to the block blob using
+     * the specified operation context.
+     * 
+     * @param opContext
+     *          An {@link OperationContext} object that represents the context
+     *          for the current operation. This object is used to track requests
+     *          to the storage service, and to provide additional runtime
+     *          information about the operation.
+     * 
+     * @return A {@link BlobOutputStream} object used to write data to the blob.
+     * 
+     * @throws StorageException
+     *           If a storage service error occurred.
+     */
+    public abstract OutputStream openOutputStream(BlobRequestOptions options,
+        OperationContext opContext) throws StorageException;
+
+    /**
+     * Uploads the source stream data to the blob, using the specified operation
+     * context.
+     * 
+     * @param sourceStream
+     *          An <code>InputStream</code> object that represents the input
+     *          stream to write to the block blob.
+     * @param opContext
+     *          An {@link OperationContext} object that represents the context
+     *          for the current operation. This object is used to track requests
+     *          to the storage service, and to provide additional runtime
+     *          information about the operation.
+     * 
+     * @throws IOException
+     *           If an I/O error occurred.
+     * @throws StorageException
+     *           If a storage service error occurred.
+     */
+    public abstract void upload(InputStream sourceStream,
+        OperationContext opContext) throws StorageException, IOException;
+
+    /**
+     * Uploads the blob's metadata to the storage service using the specified
+     * lease ID, request options, and operation context.
+     * 
+     * @param opContext
+     *          An {@link OperationContext} object that represents the context
+     *          for the current operation. This object is used to track requests
+     *          to the storage service, and to provide additional runtime
+     *          information about the operation.
+     * 
+     * @throws StorageException
+     *           If a storage service error occurred.
+     */
+    public abstract void uploadMetadata(OperationContext opContext)
+        throws StorageException;
+
+    public abstract void uploadProperties(OperationContext opContext)
+        throws StorageException;
+
+    /**
+     * Sets the minimum read block size to use with this Blob.
+     * 
+     * @param minimumReadSizeBytes
+     *          The maximum block size, in bytes, for reading from a block blob
+     *          while using a {@link BlobInputStream} object, ranging from 512
+     *          bytes to 64 MB, inclusive.
+     */
+    public abstract void setStreamMinimumReadSizeInBytes(
+        int minimumReadSizeBytes);
+
+    /**
+     * Sets the write block size to use with this Blob.
+     * 
+     * @param writeBlockSizeBytes
+     *          The maximum block size, in bytes, for writing to a block blob
+     *          while using a {@link BlobOutputStream} object, ranging from 1 MB
+     *          to 4 MB, inclusive.
+     * 
+     * @throws IllegalArgumentException
+     *           If <code>writeBlockSizeInBytes</code> is less than 1 MB or
+     *           greater than 4 MB.
+     */
+    public abstract void setWriteBlockSizeInBytes(int writeBlockSizeBytes);
+
+  }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
new file mode 100644
index 0000000..935bf71
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
@@ -0,0 +1,372 @@
+/**
+ * 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.URI;
+import java.net.URISyntaxException;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Iterator;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import com.microsoft.windowsazure.storage.AccessCondition;
+import com.microsoft.windowsazure.storage.CloudStorageAccount;
+import com.microsoft.windowsazure.storage.OperationContext;
+import com.microsoft.windowsazure.storage.RetryPolicyFactory;
+import com.microsoft.windowsazure.storage.StorageCredentials;
+import com.microsoft.windowsazure.storage.StorageException;
+import com.microsoft.windowsazure.storage.StorageUri;
+import com.microsoft.windowsazure.storage.blob.BlobListingDetails;
+import com.microsoft.windowsazure.storage.blob.BlobProperties;
+import com.microsoft.windowsazure.storage.blob.BlobRequestOptions;
+import com.microsoft.windowsazure.storage.blob.CloudBlobClient;
+import com.microsoft.windowsazure.storage.blob.CloudBlobContainer;
+import com.microsoft.windowsazure.storage.blob.CloudBlobDirectory;
+import com.microsoft.windowsazure.storage.blob.CloudBlockBlob;
+import com.microsoft.windowsazure.storage.blob.CopyState;
+import com.microsoft.windowsazure.storage.blob.DeleteSnapshotsOption;
+import com.microsoft.windowsazure.storage.blob.ListBlobItem;
+
+/**
+ * A real implementation of the Azure interaction layer that just redirects
+ * calls to the Windows Azure storage SDK.
+ */
+@InterfaceAudience.Private
+class StorageInterfaceImpl extends StorageInterface {
+  private CloudBlobClient serviceClient;
+
+  @Override
+  public void setRetryPolicyFactory(final RetryPolicyFactory retryPolicyFactory) {
+    serviceClient.setRetryPolicyFactory(retryPolicyFactory);
+  }
+
+  @Override
+  public void setTimeoutInMs(int timeoutInMs) {
+    serviceClient.setTimeoutInMs(timeoutInMs);
+  }
+
+  @Override
+  public void createBlobClient(CloudStorageAccount account) {
+    serviceClient = account.createCloudBlobClient();
+  }
+
+  @Override
+  public void createBlobClient(URI baseUri) {
+    serviceClient = new CloudBlobClient(baseUri);
+  }
+
+  @Override
+  public void createBlobClient(URI baseUri, StorageCredentials credentials) {
+    serviceClient = new CloudBlobClient(baseUri, credentials);
+  }
+
+  @Override
+  public StorageCredentials getCredentials() {
+    return serviceClient.getCredentials();
+  }
+
+  @Override
+  public CloudBlobContainerWrapper getContainerReference(String uri)
+      throws URISyntaxException, StorageException {
+    return new CloudBlobContainerWrapperImpl(
+        serviceClient.getContainerReference(uri));
+  }
+
+  //
+  // WrappingIterator
+  //
+
+  /**
+   * This iterator wraps every ListBlobItem as they come from the listBlobs()
+   * calls to their proper wrapping objects.
+   */
+  private static class WrappingIterator implements Iterator<ListBlobItem> {
+    private final Iterator<ListBlobItem> present;
+
+    public WrappingIterator(Iterator<ListBlobItem> present) {
+      this.present = present;
+    }
+
+    public static Iterable<ListBlobItem> wrap(
+        final Iterable<ListBlobItem> present) {
+      return new Iterable<ListBlobItem>() {
+        @Override
+        public Iterator<ListBlobItem> iterator() {
+          return new WrappingIterator(present.iterator());
+        }
+      };
+    }
+
+    @Override
+    public boolean hasNext() {
+      return present.hasNext();
+    }
+
+    @Override
+    public ListBlobItem next() {
+      ListBlobItem unwrapped = present.next();
+      if (unwrapped instanceof CloudBlobDirectory) {
+        return new CloudBlobDirectoryWrapperImpl((CloudBlobDirectory) unwrapped);
+      } else if (unwrapped instanceof CloudBlockBlob) {
+        return new CloudBlockBlobWrapperImpl((CloudBlockBlob) unwrapped);
+      } else {
+        return unwrapped;
+      }
+    }
+
+    @Override
+    public void remove() {
+      present.remove();
+    }
+  }
+
+  //
+  // CloudBlobDirectoryWrapperImpl
+  //
+  @InterfaceAudience.Private
+  static class CloudBlobDirectoryWrapperImpl extends CloudBlobDirectoryWrapper {
+    private final CloudBlobDirectory directory;
+
+    public CloudBlobDirectoryWrapperImpl(CloudBlobDirectory directory) {
+      this.directory = directory;
+    }
+
+    @Override
+    public URI getUri() {
+      return directory.getUri();
+    }
+
+    @Override
+    public Iterable<ListBlobItem> listBlobs(String prefix,
+        boolean useFlatBlobListing, EnumSet<BlobListingDetails> listingDetails,
+        BlobRequestOptions options, OperationContext opContext)
+        throws URISyntaxException, StorageException {
+      return WrappingIterator.wrap(directory.listBlobs(prefix,
+          useFlatBlobListing, listingDetails, options, opContext));
+    }
+
+    @Override
+    public CloudBlobContainer getContainer() throws URISyntaxException,
+        StorageException {
+      return directory.getContainer();
+    }
+
+    @Override
+    public CloudBlobDirectory getParent() throws URISyntaxException,
+        StorageException {
+      return directory.getParent();
+    }
+
+    @Override
+    public StorageUri getStorageUri() {
+      return directory.getStorageUri();
+    }
+
+  }
+
+  //
+  // CloudBlobContainerWrapperImpl
+  //
+  @InterfaceAudience.Private
+  static class CloudBlobContainerWrapperImpl extends CloudBlobContainerWrapper {
+    private final CloudBlobContainer container;
+
+    public CloudBlobContainerWrapperImpl(CloudBlobContainer container) {
+      this.container = container;
+    }
+
+    @Override
+    public String getName() {
+      return container.getName();
+    }
+
+    @Override
+    public boolean exists(OperationContext opContext) throws StorageException {
+      return container.exists(AccessCondition.generateEmptyCondition(), null,
+          opContext);
+    }
+
+    @Override
+    public void create(OperationContext opContext) throws StorageException {
+      container.create(null, opContext);
+    }
+
+    @Override
+    public HashMap<String, String> getMetadata() {
+      return container.getMetadata();
+    }
+
+    @Override
+    public void setMetadata(HashMap<String, String> metadata) {
+      container.setMetadata(metadata);
+    }
+
+    @Override
+    public void downloadAttributes(OperationContext opContext)
+        throws StorageException {
+      container.downloadAttributes(AccessCondition.generateEmptyCondition(),
+          null, opContext);
+    }
+
+    @Override
+    public void uploadMetadata(OperationContext opContext)
+        throws StorageException {
+      container.uploadMetadata(AccessCondition.generateEmptyCondition(), null,
+          opContext);
+    }
+
+    @Override
+    public CloudBlobDirectoryWrapper getDirectoryReference(String relativePath)
+        throws URISyntaxException, StorageException {
+
+      CloudBlobDirectory dir = container.getDirectoryReference(relativePath);
+      return new CloudBlobDirectoryWrapperImpl(dir);
+    }
+
+    @Override
+    public CloudBlockBlobWrapper getBlockBlobReference(String relativePath)
+        throws URISyntaxException, StorageException {
+
+      return new CloudBlockBlobWrapperImpl(
+          container.getBlockBlobReference(relativePath));
+    }
+  }
+
+  //
+  // CloudBlockBlobWrapperImpl
+  //
+  @InterfaceAudience.Private
+  static class CloudBlockBlobWrapperImpl extends CloudBlockBlobWrapper {
+    private final CloudBlockBlob blob;
+
+    public URI getUri() {
+      return blob.getUri();
+    }
+
+    public CloudBlockBlobWrapperImpl(CloudBlockBlob blob) {
+      this.blob = blob;
+    }
+
+    @Override
+    public HashMap<String, String> getMetadata() {
+      return blob.getMetadata();
+    }
+
+    @Override
+    public void startCopyFromBlob(CloudBlockBlobWrapper sourceBlob,
+        OperationContext opContext) throws StorageException, URISyntaxException {
+
+      blob.startCopyFromBlob(((CloudBlockBlobWrapperImpl) sourceBlob).blob,
+          null, null, null, opContext);
+
+    }
+
+    @Override
+    public void delete(OperationContext opContext) throws StorageException {
+      blob.delete(DeleteSnapshotsOption.NONE, null, null, opContext);
+    }
+
+    @Override
+    public boolean exists(OperationContext opContext) throws StorageException {
+      return blob.exists(null, null, opContext);
+    }
+
+    @Override
+    public void downloadAttributes(OperationContext opContext)
+        throws StorageException {
+      blob.downloadAttributes(null, null, opContext);
+    }
+
+    @Override
+    public BlobProperties getProperties() {
+      return blob.getProperties();
+    }
+
+    @Override
+    public void setMetadata(HashMap<String, String> metadata) {
+      blob.setMetadata(metadata);
+    }
+
+    @Override
+    public InputStream openInputStream(BlobRequestOptions options,
+        OperationContext opContext) throws StorageException {
+      return blob.openInputStream(null, options, opContext);
+    }
+
+    @Override
+    public OutputStream openOutputStream(BlobRequestOptions options,
+        OperationContext opContext) throws StorageException {
+      return blob.openOutputStream(null, options, opContext);
+    }
+
+    @Override
+    public void upload(InputStream sourceStream, OperationContext opContext)
+        throws StorageException, IOException {
+      blob.upload(sourceStream, 0, null, null, opContext);
+    }
+
+    @Override
+    public CloudBlobContainer getContainer() throws URISyntaxException,
+        StorageException {
+      return blob.getContainer();
+    }
+
+    @Override
+    public CloudBlobDirectory getParent() throws URISyntaxException,
+        StorageException {
+      return blob.getParent();
+    }
+
+    @Override
+    public void uploadMetadata(OperationContext opContext)
+        throws StorageException {
+      blob.uploadMetadata(null, null, opContext);
+    }
+
+    @Override
+    public void uploadProperties(OperationContext opContext)
+        throws StorageException {
+      blob.uploadProperties(null, null, opContext);
+    }
+
+    @Override
+    public void setStreamMinimumReadSizeInBytes(int minimumReadSizeBytes) {
+      blob.setStreamMinimumReadSizeInBytes(minimumReadSizeBytes);
+    }
+
+    @Override
+    public void setWriteBlockSizeInBytes(int writeBlockSizeBytes) {
+      blob.setStreamWriteSizeInBytes(writeBlockSizeBytes);
+    }
+
+    @Override
+    public StorageUri getStorageUri() {
+      return blob.getStorageUri();
+    }
+
+    @Override
+    public CopyState getCopyState() {
+      return blob.getCopyState();
+    }
+  }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/Wasb.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/Wasb.java
new file mode 100644
index 0000000..e098cef
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/Wasb.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegateToFileSystem;
+
+
+/**
+ * WASB implementation of AbstractFileSystem.
+ * This impl delegates to the old FileSystem
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class Wasb extends DelegateToFileSystem {
+
+  Wasb(final URI theUri, final Configuration conf) throws IOException,
+      URISyntaxException {
+    super(theUri, new NativeAzureFileSystem(), conf, "wasb", false);
+  }
+
+  @Override
+  public int getUriDefaultPort() {
+    return -1;
+  }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/WasbFsck.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/WasbFsck.java
new file mode 100644
index 0000000..d311550
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/WasbFsck.java
@@ -0,0 +1,196 @@
+/**
+ * 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.util.Arrays;
+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.conf.Configured;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * An fsck tool implementation for WASB that does various admin/cleanup/recovery
+ * tasks on the WASB file system.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class WasbFsck extends Configured implements Tool {
+  private FileSystem mockFileSystemForTesting = null;
+  private static final String LOST_AND_FOUND_PATH = "/lost+found";
+  private boolean pathNameWarning = false;
+
+  public WasbFsck(Configuration conf) {
+    super(conf);
+  }
+
+  /**
+   * For testing purposes, set the file system to use here instead of relying on
+   * getting it from the FileSystem class based on the URI.
+   * 
+   * @param fileSystem
+   *          The file system to use.
+   */
+  @VisibleForTesting
+  public void setMockFileSystemForTesting(FileSystem fileSystem) {
+    this.mockFileSystemForTesting = fileSystem;
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    if (doPrintUsage(Arrays.asList(args))) {
+      printUsage();
+      return -1;
+    }
+    Path pathToCheck = null;
+    boolean doRecover = false;
+    boolean doDelete = false;
+    for (String arg : args) {
+      if (!arg.startsWith("-")) {
+        if (pathToCheck != null) {
+          System.err
+              .println("Can't specify multiple paths to check on the command-line");
+          return 1;
+        }
+        pathToCheck = new Path(arg);
+      } else if (arg.equals("-move")) {
+        doRecover = true;
+      } else if (arg.equals("-delete")) {
+        doDelete = true;
+      }
+    }
+    if (doRecover && doDelete) {
+      System.err
+          .println("Conflicting options: can't specify both -move and -delete.");
+      return 1;
+    }
+    if (pathToCheck == null) {
+      pathToCheck = new Path("/"); // Check everything.
+    }
+    FileSystem fs;
+    if (mockFileSystemForTesting == null) {
+      fs = FileSystem.get(pathToCheck.toUri(), getConf());
+    } else {
+      fs = mockFileSystemForTesting;
+    }
+
+    if (!recursiveCheckChildPathName(fs, fs.makeQualified(pathToCheck))) {
+      pathNameWarning = true;
+    }
+
+    if (!(fs instanceof NativeAzureFileSystem)) {
+      System.err
+          .println("Can only check WASB file system. Instead I'm asked to"
+              + " check: " + fs.getUri());
+      return 2;
+    }
+    NativeAzureFileSystem wasbFs = (NativeAzureFileSystem) fs;
+    if (doRecover) {
+      System.out.println("Recovering files with dangling data under: "
+          + pathToCheck);
+      wasbFs.recoverFilesWithDanglingTempData(pathToCheck, new Path(
+          LOST_AND_FOUND_PATH));
+    } else if (doDelete) {
+      System.out.println("Deleting temp files with dangling data under: "
+          + pathToCheck);
+      wasbFs.deleteFilesWithDanglingTempData(pathToCheck);
+    } else {
+      System.out.println("Please specify -move or -delete");
+    }
+    return 0;
+  }
+
+  public boolean getPathNameWarning() {
+    return pathNameWarning;
+  }
+
+  /**
+   * Recursively check if a given path and its child paths have colons in their
+   * names. It returns true if none of them has a colon or this path does not
+   * exist, and false otherwise.
+   */
+  private boolean recursiveCheckChildPathName(FileSystem fs, Path p)
+      throws IOException {
+    if (p == null) {
+      return true;
+    }
+    if (!fs.exists(p)) {
+      System.out.println("Path " + p + " does not exist!");
+      return true;
+    }
+
+    if (fs.isFile(p)) {
+      if (containsColon(p)) {
+        System.out.println("Warning: file " + p + " has a colon in its name.");
+        return false;
+      } else {
+        return true;
+      }
+    } else {
+      boolean flag;
+      if (containsColon(p)) {
+        System.out.println("Warning: directory " + p
+            + " has a colon in its name.");
+        flag = false;
+      } else {
+        flag = true;
+      }
+      FileStatus[] listed = fs.listStatus(p);
+      for (FileStatus l : listed) {
+        if (!recursiveCheckChildPathName(fs, l.getPath())) {
+          flag = false;
+        }
+      }
+      return flag;
+    }
+  }
+
+  private boolean containsColon(Path p) {
+    return p.toUri().getPath().toString().contains(":");
+  }
+
+  private static void printUsage() {
+    System.out.println("Usage: WasbFSck [<path>] [-move | -delete]");
+    System.out.println("\t<path>\tstart checking from this path");
+    System.out.println("\t-move\tmove any files whose upload was interrupted"
+        + " mid-stream to " + LOST_AND_FOUND_PATH);
+    System.out
+        .println("\t-delete\tdelete any files whose upload was interrupted"
+            + " mid-stream");
+    ToolRunner.printGenericCommandUsage(System.out);
+  }
+
+  private boolean doPrintUsage(List<String> args) {
+    return args.contains("-H");
+  }
+
+  public static void main(String[] args) throws Exception {
+    int res = ToolRunner.run(new WasbFsck(new Configuration()), args);
+    System.exit(res);
+  }
+}
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/package.html b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/package.html
new file mode 100644
index 0000000..de01683
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/package.html
@@ -0,0 +1,31 @@
+<html>
+
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<body>
+
+<p>
+A distributed implementation of {@link
+org.apache.hadoop.fs.FileSystem} for reading and writing files on
+<a href="http://store.azure.com">Azure Block Storage</a>.
+This implementation is blob-based and stores files on Azure in their native form for
+interoperability with other Azure tools.
+</p>
+
+</body>
+</html>
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
new file mode 100644
index 0000000..8133954
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/AzureBlobStorageTestAccount.java
@@ -0,0 +1,726 @@
+/**
+ * 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.DEFAULT_STORAGE_EMULATOR_ACCOUNT_NAME;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.EnumSet;
+import java.util.GregorianCalendar;
+import java.util.TimeZone;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+import com.microsoft.windowsazure.storage.AccessCondition;
+import com.microsoft.windowsazure.storage.CloudStorageAccount;
+import com.microsoft.windowsazure.storage.StorageCredentials;
+import com.microsoft.windowsazure.storage.StorageCredentialsAccountAndKey;
+import com.microsoft.windowsazure.storage.StorageCredentialsAnonymous;
+import com.microsoft.windowsazure.storage.blob.BlobContainerPermissions;
+import com.microsoft.windowsazure.storage.blob.BlobContainerPublicAccessType;
+import com.microsoft.windowsazure.storage.blob.BlobOutputStream;
+import com.microsoft.windowsazure.storage.blob.CloudBlobClient;
+import com.microsoft.windowsazure.storage.blob.CloudBlobContainer;
+import com.microsoft.windowsazure.storage.blob.CloudBlockBlob;
+import com.microsoft.windowsazure.storage.blob.SharedAccessBlobPermissions;
+import com.microsoft.windowsazure.storage.blob.SharedAccessBlobPolicy;
+import com.microsoft.windowsazure.storage.core.Base64;
+
+/**
+ * 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.
+ */
+public final class AzureBlobStorageTestAccount {
+
+  private static final String ACCOUNT_KEY_PROPERTY_NAME = "fs.azure.account.key.";
+  private static final String SAS_PROPERTY_NAME = "fs.azure.sas.";
+  private static final String TEST_CONFIGURATION_FILE_NAME = "azure-test.xml";
+  private static final String TEST_ACCOUNT_NAME_PROPERTY_NAME = "fs.azure.test.account.name";
+  public static final String MOCK_ACCOUNT_NAME = "mockAccount.blob.core.windows.net";
+  public static final String MOCK_CONTAINER_NAME = "mockContainer";
+  public static final String WASB_AUTHORITY_DELIMITER = "@";
+  public static final String WASB_SCHEME = "wasb";
+  public static final String PATH_DELIMITER = "/";
+  public static final String AZURE_ROOT_CONTAINER = "$root";
+  public static final String MOCK_WASB_URI = "wasb://" + MOCK_CONTAINER_NAME
+      + WASB_AUTHORITY_DELIMITER + MOCK_ACCOUNT_NAME + "/";
+  private static final String USE_EMULATOR_PROPERTY_NAME = "fs.azure.test.emulator";
+
+  private static final String KEY_DISABLE_THROTTLING = "fs.azure.disable.bandwidth.throttling";
+  private static final String KEY_READ_TOLERATE_CONCURRENT_APPEND = "fs.azure.io.read.tolerate.concurrent.append";
+
+  private CloudStorageAccount account;
+  private CloudBlobContainer container;
+  private CloudBlockBlob blob;
+  private NativeAzureFileSystem fs;
+  private AzureNativeFileSystemStore storage;
+  private MockStorageInterface mockStorage;
+
+  private AzureBlobStorageTestAccount(NativeAzureFileSystem fs,
+      CloudStorageAccount account, CloudBlobContainer container) {
+    this.account = account;
+    this.container = container;
+    this.fs = fs;
+  }
+
+  /**
+   * Create a test account with an initialized storage reference.
+   * 
+   * @param storage
+   *          -- store to be accessed by the account
+   * @param account
+   *          -- Windows Azure account object
+   * @param container
+   *          -- Windows Azure container object
+   */
+  private AzureBlobStorageTestAccount(AzureNativeFileSystemStore storage,
+      CloudStorageAccount account, CloudBlobContainer container) {
+    this.account = account;
+    this.container = container;
+    this.storage = storage;
+  }
+
+  /**
+   * Create a test account sessions with the default root container.
+   * 
+   * @param fs
+   *          - file system, namely WASB file system
+   * @param account
+   *          - Windows Azure account object
+   * @param blob
+   *          - block blob reference
+   */
+  private AzureBlobStorageTestAccount(NativeAzureFileSystem fs,
+      CloudStorageAccount account, CloudBlockBlob blob) {
+
+    this.account = account;
+    this.blob = blob;
+    this.fs = fs;
+  }
+
+  private AzureBlobStorageTestAccount(NativeAzureFileSystem fs,
+      MockStorageInterface mockStorage) {
+    this.fs = fs;
+    this.mockStorage = mockStorage;
+  }
+
+  public static String getMockContainerUri() {
+    return String.format("http://%s/%s",
+        AzureBlobStorageTestAccount.MOCK_ACCOUNT_NAME,
+        AzureBlobStorageTestAccount.MOCK_CONTAINER_NAME);
+  }
+
+  public static String toMockUri(String path) {
+    return String.format("http://%s/%s/%s",
+        AzureBlobStorageTestAccount.MOCK_ACCOUNT_NAME,
+        AzureBlobStorageTestAccount.MOCK_CONTAINER_NAME, path);
+  }
+
+  public static String toMockUri(Path path) {
+    // Remove the first SEPARATOR
+    return toMockUri(path.toUri().getRawPath().substring(1)); 
+  }
+
+  /**
+   * Gets the blob reference to the given blob key.
+   * 
+   * @param blobKey
+   *          The blob key (no initial slash).
+   * @return The blob reference.
+   */
+  public CloudBlockBlob getBlobReference(String blobKey) throws Exception {
+    return container.getBlockBlobReference(String.format(blobKey));
+  }
+
+  /**
+   * Acquires a short lease on the given blob in this test account.
+   * 
+   * @param blobKey
+   *          The key to the blob (no initial slash).
+   * @return The lease ID.
+   */
+  public String acquireShortLease(String blobKey) throws Exception {
+    return getBlobReference(blobKey).acquireLease(60, null);
+  }
+
+  /**
+   * Releases the lease on the container.
+   * 
+   * @param leaseID
+   *          The lease ID.
+   */
+  public void releaseLease(String leaseID, String blobKey) throws Exception {
+    AccessCondition accessCondition = new AccessCondition();
+    accessCondition.setLeaseID(leaseID);
+    getBlobReference(blobKey).releaseLease(accessCondition);
+  }
+
+  public static AzureBlobStorageTestAccount createMock() throws Exception {
+    return createMock(new Configuration());
+  }
+
+  public static AzureBlobStorageTestAccount createMock(Configuration conf)
+      throws Exception {
+    AzureNativeFileSystemStore store = new AzureNativeFileSystemStore();
+    MockStorageInterface mockStorage = new MockStorageInterface();
+    store.setAzureStorageInteractionLayer(mockStorage);
+    NativeAzureFileSystem fs = new NativeAzureFileSystem(store);
+    addWasbToConfiguration(conf);
+    setMockAccountKey(conf);
+    // register the fs provider.
+
+    fs.initialize(new URI(MOCK_WASB_URI), conf);
+    AzureBlobStorageTestAccount testAcct = new AzureBlobStorageTestAccount(fs,
+        mockStorage);
+    return testAcct;
+  }
+
+  /**
+   * Creates a test account that goes against the storage emulator.
+   * 
+   * @return The test account, or null if the emulator isn't setup.
+   */
+  public static AzureBlobStorageTestAccount createForEmulator()
+      throws Exception {
+    NativeAzureFileSystem fs = null;
+    CloudBlobContainer container = null;
+    Configuration conf = createTestConfiguration();
+    if (!conf.getBoolean(USE_EMULATOR_PROPERTY_NAME, false)) {
+      // Not configured to test against the storage emulator.
+      System.out.println("Skipping emulator Azure test because configuration "
+          + "doesn't indicate that it's running."
+          + " Please see README.txt for guidance.");
+      return null;
+    }
+    CloudStorageAccount account = CloudStorageAccount
+        .getDevelopmentStorageAccount();
+    fs = new NativeAzureFileSystem();
+    String containerName = String.format("wasbtests-%s-%tQ",
+        System.getProperty("user.name"), new Date());
+    container = account.createCloudBlobClient().getContainerReference(
+        containerName);
+    container.create();
+
+    // Set account URI and initialize Azure file system.
+    URI accountUri = createAccountUri(DEFAULT_STORAGE_EMULATOR_ACCOUNT_NAME,
+        containerName);
+    fs.initialize(accountUri, conf);
+
+    // Create test account initializing the appropriate member variables.
+    AzureBlobStorageTestAccount testAcct = new AzureBlobStorageTestAccount(fs,
+        account, container);
+
+    return testAcct;
+  }
+
+  public static AzureBlobStorageTestAccount createOutOfBandStore(
+      int uploadBlockSize, int downloadBlockSize) throws Exception {
+
+    CloudBlobContainer container = null;
+    Configuration conf = createTestConfiguration();
+    CloudStorageAccount account = createTestAccount(conf);
+    if (null == account) {
+      return null;
+    }
+
+    String containerName = String.format("wasbtests-%s-%tQ",
+        System.getProperty("user.name"), new Date());
+
+    // Create the container.
+    container = account.createCloudBlobClient().getContainerReference(
+        containerName);
+    container.create();
+
+    String accountName = conf.get(TEST_ACCOUNT_NAME_PROPERTY_NAME);
+
+    // Ensure that custom throttling is disabled and tolerate concurrent
+    // out-of-band appends.
+    conf.setBoolean(KEY_DISABLE_THROTTLING, true);
+    conf.setBoolean(KEY_READ_TOLERATE_CONCURRENT_APPEND, true);
+
+    // Set account URI and initialize Azure file system.
+    URI accountUri = createAccountUri(accountName, containerName);
+
+    // Create a new AzureNativeFileSystemStore object.
+    AzureNativeFileSystemStore testStorage = new AzureNativeFileSystemStore();
+
+    // Initialize the store with the throttling feedback interfaces.
+    testStorage.initialize(accountUri, conf);
+
+    // Create test account initializing the appropriate member variables.
+    AzureBlobStorageTestAccount testAcct = new AzureBlobStorageTestAccount(
+        testStorage, account, container);
+
+    return testAcct;
+  }
+
+  /**
+   * Sets the mock account key in the given configuration.
+   * 
+   * @param conf
+   *          The configuration.
+   */
+  public static void setMockAccountKey(Configuration conf) {
+    setMockAccountKey(conf, MOCK_ACCOUNT_NAME);
+  }
+
+  /**
+   * Sets the mock account key in the given configuration.
+   * 
+   * @param conf
+   *          The configuration.
+   */
+  public static void setMockAccountKey(Configuration conf, String accountName) {
+    conf.set(ACCOUNT_KEY_PROPERTY_NAME + accountName,
+        Base64.encode(new byte[] { 1, 2, 3 }));  
+  }
+
+  private static URI createAccountUri(String accountName)
+      throws URISyntaxException {
+    return new URI(WASB_SCHEME + ":" + PATH_DELIMITER + PATH_DELIMITER
+        + accountName);
+  }
+
+  private static URI createAccountUri(String accountName, String containerName)
+      throws URISyntaxException {
+    return new URI(WASB_SCHEME + ":" + PATH_DELIMITER + PATH_DELIMITER
+        + containerName + WASB_AUTHORITY_DELIMITER + accountName);
+  }
+
+  public static AzureBlobStorageTestAccount create() throws Exception {
+    return create("");
+  }
+
+  public static AzureBlobStorageTestAccount create(String containerNameSuffix)
+      throws Exception {
+    return create(containerNameSuffix,
+        EnumSet.of(CreateOptions.CreateContainer));
+  }
+
+  // Create a test account which uses throttling.
+  public static AzureBlobStorageTestAccount createThrottled() throws Exception {
+    return create("",
+        EnumSet.of(CreateOptions.useThrottling, CreateOptions.CreateContainer));
+  }
+
+  public static AzureBlobStorageTestAccount create(Configuration conf)
+      throws Exception {
+    return create("", EnumSet.of(CreateOptions.CreateContainer), conf);
+  }
+
+  static CloudStorageAccount createStorageAccount(String accountName,
+      Configuration conf, boolean allowAnonymous) throws URISyntaxException,
+      KeyProviderException {
+    String accountKey = AzureNativeFileSystemStore
+        .getAccountKeyFromConfiguration(accountName, conf);
+    StorageCredentials credentials;
+    if (accountKey == null && allowAnonymous) {
+      credentials = StorageCredentialsAnonymous.ANONYMOUS;
+    } else {
+      credentials = new StorageCredentialsAccountAndKey(
+          accountName.split("\\.")[0], accountKey);
+    }
+    if (credentials == null) {
+      return null;
+    } else {
+      return new CloudStorageAccount(credentials);
+    }
+  }
+
+  private static Configuration createTestConfiguration() {
+    return createTestConfiguration(null);
+  }
+
+  protected static Configuration createTestConfiguration(Configuration conf) {
+    if (conf == null) {
+      conf = new Configuration();
+    }
+
+    conf.addResource(TEST_CONFIGURATION_FILE_NAME);
+    return conf;
+  }
+
+  // for programmatic setting of the wasb configuration.
+  // note that tests can also get the
+  public static void addWasbToConfiguration(Configuration conf) {
+    conf.set("fs.wasb.impl", "org.apache.hadoop.fs.azure.NativeAzureFileSystem");
+    conf.set("fs.wasbs.impl",
+        "org.apache.hadoop.fs.azure.NativeAzureFileSystem");
+  }
+
+  static CloudStorageAccount createTestAccount() throws URISyntaxException,
+      KeyProviderException {
+    return createTestAccount(createTestConfiguration());
+  }
+
+  static CloudStorageAccount createTestAccount(Configuration conf)
+      throws URISyntaxException, KeyProviderException {
+    String testAccountName = conf.get(TEST_ACCOUNT_NAME_PROPERTY_NAME);
+    if (testAccountName == null) {
+      System.out
+          .println("Skipping live Azure test because of missing test account."
+              + " Please see README.txt for guidance.");
+      return null;
+    }
+    return createStorageAccount(testAccountName, conf, false);
+  }
+
+  public static enum CreateOptions {
+    UseSas, Readonly, CreateContainer, useThrottling
+  }
+
+  public static AzureBlobStorageTestAccount create(String containerNameSuffix,
+      EnumSet<CreateOptions> createOptions) throws Exception {
+    return create(containerNameSuffix, createOptions, null);
+  }
+
+  public static AzureBlobStorageTestAccount create(String containerNameSuffix,
+      EnumSet<CreateOptions> createOptions, Configuration initialConfiguration)
+      throws Exception {
+    NativeAzureFileSystem fs = null;
+    CloudBlobContainer container = null;
+    Configuration conf = createTestConfiguration(initialConfiguration);
+    CloudStorageAccount account = createTestAccount(conf);
+    if (account == null) {
+      return null;
+    }
+    fs = new NativeAzureFileSystem();
+    String containerName = String.format("wasbtests-%s-%tQ%s",
+        System.getProperty("user.name"), new Date(), containerNameSuffix);
+    container = account.createCloudBlobClient().getContainerReference(
+        containerName);
+    if (createOptions.contains(CreateOptions.CreateContainer)) {
+      container.create();
+    }
+    String accountName = conf.get(TEST_ACCOUNT_NAME_PROPERTY_NAME);
+    if (createOptions.contains(CreateOptions.UseSas)) {
+      String sas = generateSAS(container,
+          createOptions.contains(CreateOptions.Readonly));
+      if (!createOptions.contains(CreateOptions.CreateContainer)) {
+        // The caller doesn't want the container to be pre-created,
+        // so delete it now that we have generated the SAS.
+        container.delete();
+      }
+      // Remove the account key from the configuration to make sure we don't
+      // cheat and use that.
+      conf.set(ACCOUNT_KEY_PROPERTY_NAME + accountName, "");
+      // Set the SAS key.
+      conf.set(SAS_PROPERTY_NAME + containerName + "." + accountName, sas);
+    }
+
+    // Check if throttling is turned on and set throttling parameters
+    // appropriately.
+    if (createOptions.contains(CreateOptions.useThrottling)) {
+      conf.setBoolean(KEY_DISABLE_THROTTLING, false);
+    } else {
+      conf.setBoolean(KEY_DISABLE_THROTTLING, true);
+    }
+
+    // Set account URI and initialize Azure file system.
+    URI accountUri = createAccountUri(accountName, containerName);
+    fs.initialize(accountUri, conf);
+
+    // Create test account initializing the appropriate member variables.
+    AzureBlobStorageTestAccount testAcct = new AzureBlobStorageTestAccount(fs,
+        account, container);
+
+    return testAcct;
+  }
+
+  private static String generateContainerName() throws Exception {
+    String containerName = String.format("wasbtests-%s-%tQ",
+        System.getProperty("user.name"), new Date());
+    return containerName;
+  }
+
+  private static String generateSAS(CloudBlobContainer container,
+      boolean readonly) throws Exception {
+
+    // Create a container if it does not exist.
+    container.createIfNotExists();
+
+    // Create a new shared access policy.
+    SharedAccessBlobPolicy sasPolicy = new SharedAccessBlobPolicy();
+
+    // Create a UTC Gregorian calendar value.
+    GregorianCalendar calendar = new GregorianCalendar(
+        TimeZone.getTimeZone("UTC"));
+
+    // Specify the current time as the start time for the shared access
+    // signature.
+    //
+    calendar.setTime(new Date());
+    sasPolicy.setSharedAccessStartTime(calendar.getTime());
+
+    // Use the start time delta one hour as the end time for the shared
+    // access signature.
+    calendar.add(Calendar.HOUR, 10);
+    sasPolicy.setSharedAccessExpiryTime(calendar.getTime());
+
+    if (readonly) {
+      // Set READ permissions
+      sasPolicy.setPermissions(EnumSet.of(SharedAccessBlobPermissions.READ,
+          SharedAccessBlobPermissions.LIST));
+    } else {
+      // Set READ and WRITE permissions.
+      sasPolicy.setPermissions(EnumSet.of(SharedAccessBlobPermissions.READ,
+          SharedAccessBlobPermissions.WRITE, SharedAccessBlobPermissions.LIST));
+    }
+
+    // Create the container permissions.
+    BlobContainerPermissions containerPermissions = new BlobContainerPermissions();
+
+    // Turn public access to the container off.
+    containerPermissions.setPublicAccess(BlobContainerPublicAccessType.OFF);
+
+    container.uploadPermissions(containerPermissions);
+
+    // Create a shared access signature for the container.
+    String sas = container.generateSharedAccessSignature(sasPolicy, null);
+    // HACK: when the just generated SAS is used straight away, we get an
+    // authorization error intermittently. Sleeping for 1.5 seconds fixes that
+    // on my box.
+    Thread.sleep(1500);
+
+    // Return to caller with the shared access signature.
+    return sas;
+  }
+
+  public static void primePublicContainer(CloudBlobClient blobClient,
+      String accountName, String containerName, String blobName, int fileSize)
+      throws Exception {
+
+    // Create a container if it does not exist. The container name
+    // must be lower case.
+    CloudBlobContainer container = blobClient
+        .getContainerReference(containerName);
+
+    container.createIfNotExists();
+
+    // Create a new shared access policy.
+    SharedAccessBlobPolicy sasPolicy = new SharedAccessBlobPolicy();
+
+    // Set READ and WRITE permissions.
+    sasPolicy.setPermissions(EnumSet.of(SharedAccessBlobPermissions.READ,
+        SharedAccessBlobPermissions.WRITE, SharedAccessBlobPermissions.LIST,
+        SharedAccessBlobPermissions.DELETE));
+
+    // Create the container permissions.
+    BlobContainerPermissions containerPermissions = new BlobContainerPermissions();
+
+    // Turn public access to the container off.
+    containerPermissions
+        .setPublicAccess(BlobContainerPublicAccessType.CONTAINER);
+
+    // Set the policy using the values set above.
+    containerPermissions.getSharedAccessPolicies().put("testwasbpolicy",
+        sasPolicy);
+    container.uploadPermissions(containerPermissions);
+
+    // Create a blob output stream.
+    CloudBlockBlob blob = container.getBlockBlobReference(blobName);
+    BlobOutputStream outputStream = blob.openOutputStream();
+
+    outputStream.write(new byte[fileSize]);
+    outputStream.close();
+  }
+
+  public static AzureBlobStorageTestAccount createAnonymous(
+      final String blobName, final int fileSize) throws Exception {
+
+    NativeAzureFileSystem fs = null;
+    CloudBlobContainer container = null;
+    Configuration conf = createTestConfiguration(), noTestAccountConf = new Configuration();
+
+    // Set up a session with the cloud blob client to generate SAS and check the
+    // existence of a container and capture the container object.
+    CloudStorageAccount account = createTestAccount(conf);
+    if (account == null) {
+      return null;
+    }
+    CloudBlobClient blobClient = account.createCloudBlobClient();
+
+    // Capture the account URL and the account name.
+    String accountName = conf.get(TEST_ACCOUNT_NAME_PROPERTY_NAME);
+
+    // Generate a container name and create a shared access signature string for
+    // it.
+    //
+    String containerName = generateContainerName();
+
+    // Set up public container with the specified blob name.
+    primePublicContainer(blobClient, accountName, containerName, blobName,
+        fileSize);
+
+    // Capture the blob container object. It should exist after generating the
+    // shared access signature.
+    container = blobClient.getContainerReference(containerName);
+    if (null == container || !container.exists()) {
+      final String errMsg = String
+          .format("Container '%s' expected but not found while creating SAS account.");
+      throw new Exception(errMsg);
+    }
+
+    // Set the account URI.
+    URI accountUri = createAccountUri(accountName, containerName);
+
+    // Initialize the Native Azure file system with anonymous credentials.
+    fs = new NativeAzureFileSystem();
+    fs.initialize(accountUri, noTestAccountConf);
+
+    // Create test account initializing the appropriate member variables.
+    AzureBlobStorageTestAccount testAcct = new AzureBlobStorageTestAccount(fs,
+        account, container);
+
+    // Return to caller with test account.
+    return testAcct;
+  }
+
+  private static CloudBlockBlob primeRootContainer(CloudBlobClient blobClient,
+      String accountName, String blobName, int fileSize) throws Exception {
+
+    // Create a container if it does not exist. The container name
+    // must be lower case.
+    CloudBlobContainer container = blobClient.getContainerReference("https://"
+        + accountName + "/" + "$root");
+    container.createIfNotExists();
+
+    // Create a blob output stream.
+    CloudBlockBlob blob = container.getBlockBlobReference(blobName);
+    BlobOutputStream outputStream = blob.openOutputStream();
+
+    outputStream.write(new byte[fileSize]);
+    outputStream.close();
+
+    // Return a reference to the block blob object.
+    return blob;
+  }
+
+  public static AzureBlobStorageTestAccount createRoot(final String blobName,
+      final int fileSize) throws Exception {
+
+    NativeAzureFileSystem fs = null;
+    CloudBlobContainer container = null;
+    Configuration conf = createTestConfiguration();
+
+    // Set up a session with the cloud blob client to generate SAS and check the
+    // existence of a container and capture the container object.
+    CloudStorageAccount account = createTestAccount(conf);
+    if (account == null) {
+      return null;
+    }
+    CloudBlobClient blobClient = account.createCloudBlobClient();
+
+    // Capture the account URL and the account name.
+    String accountName = conf.get(TEST_ACCOUNT_NAME_PROPERTY_NAME);
+
+    // Set up public container with the specified blob name.
+    CloudBlockBlob blobRoot = primeRootContainer(blobClient, accountName,
+        blobName, fileSize);
+
+    // Capture the blob container object. It should exist after generating the
+    // shared access signature.
+    container = blobClient.getContainerReference(AZURE_ROOT_CONTAINER);
+    if (null == container || !container.exists()) {
+      final String errMsg = String
+          .format("Container '%s' expected but not found while creating SAS account.");
+      throw new Exception(errMsg);
+    }
+
+    // Set the account URI without a container name.
+    URI accountUri = createAccountUri(accountName);
+
+    // Initialize the Native Azure file system with anonymous credentials.
+    fs = new NativeAzureFileSystem();
+    fs.initialize(accountUri, conf);
+
+    // Create test account initializing the appropriate member variables.
+    // Set the container value to null for the default root container.
+    AzureBlobStorageTestAccount testAcct = new AzureBlobStorageTestAccount(fs,
+        account, blobRoot);
+
+    // Return to caller with test account.
+    return testAcct;
+  }
+
+  public void closeFileSystem() throws Exception {
+    if (fs != null) {
+      fs.close();
+    }
+  }
+
+  public void cleanup() throws Exception {
+    if (fs != null) {
+      fs.close();
+      fs = null;
+    }
+    if (container != null) {
+      container.deleteIfExists();
+      container = null;
+    }
+    if (blob != null) {
+      // The blob member variable is set for blobs under root containers.
+      // Delete blob objects created for root container tests when cleaning
+      // up the test account.
+      blob.delete();
+      blob = null;
+    }
+  }
+
+  public NativeAzureFileSystem getFileSystem() {
+    return fs;
+  }
+
+  public AzureNativeFileSystemStore getStore() {
+    return this.storage;
+  }
+
+  /**
+   * Gets the real blob container backing this account if it's not a mock.
+   * 
+   * @return A container, or null if it's a mock.
+   */
+  public CloudBlobContainer getRealContainer() {
+    return container;
+  }
+
+  /**
+   * Gets the real blob account backing this account if it's not a mock.
+   * 
+   * @return An account, or null if it's a mock.
+   */
+  public CloudStorageAccount getRealAccount() {
+    return account;
+  }
+
+  /**
+   * Gets the mock storage interface if this account is backed by a mock.
+   * 
+   * @return The mock storage, or null if it's backed by a real account.
+   */
+  public MockStorageInterface getMockStorage() {
+    return mockStorage;
+  }
+ 
+}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/InMemoryBlockBlobStore.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/InMemoryBlockBlobStore.java
new file mode 100644
index 0000000..ab35961
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/InMemoryBlockBlobStore.java
@@ -0,0 +1,146 @@
+/**
+ * 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.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A simple memory key-value store to help mock the Windows Azure Storage
+ * implementation for unit testing.
+ */
+public class InMemoryBlockBlobStore {
+  private final HashMap<String, Entry> blobs = new HashMap<String, Entry>();
+  private HashMap<String, String> containerMetadata;
+
+  public synchronized Iterable<String> getKeys() {
+    return new ArrayList<String>(blobs.keySet());
+  }
+
+  public static class ListBlobEntry {
+    private final String key;
+    private final HashMap<String, String> metadata;
+    private final int contentLength;
+
+    ListBlobEntry(String key, HashMap<String, String> metadata,
+        int contentLength) {
+      this.key = key;
+      this.metadata = metadata;
+      this.contentLength = contentLength;
+    }
+
+    public String getKey() {
+      return key;
+    }
+
+    public HashMap<String, String> getMetadata() {
+      return metadata;
+    }
+
+    public int getContentLength() {
+      return contentLength;
+    }
+  }
+
+  /**
+   * List all the blobs whose key starts with the given prefix.
+   * 
+   * @param prefix
+   *          The prefix to check.
+   * @param includeMetadata
+   *          If set, the metadata in the returned listing will be populated;
+   *          otherwise it'll be null.
+   * @return The listing.
+   */
+  public synchronized Iterable<ListBlobEntry> listBlobs(String prefix,
+      boolean includeMetadata) {
+    ArrayList<ListBlobEntry> list = new ArrayList<ListBlobEntry>();
+    for (Map.Entry<String, Entry> entry : blobs.entrySet()) {
+      if (entry.getKey().startsWith(prefix)) {
+        list.add(new ListBlobEntry(entry.getKey(),
+            includeMetadata ? new HashMap<String, String>(
+                entry.getValue().metadata) : null,
+            entry.getValue().content.length));
+      }
+    }
+    return list;
+  }
+
+  public synchronized byte[] getContent(String key) {
+    return blobs.get(key).content;
+  }
+
+  @SuppressWarnings("unchecked")
+  public synchronized void setContent(String key, byte[] value,
+      HashMap<String, String> metadata) {
+    blobs
+        .put(key, new Entry(value, (HashMap<String, String>) metadata.clone()));
+  }
+
+  public OutputStream upload(final String key,
+      final HashMap<String, String> metadata) {
+    setContent(key, new byte[0], metadata);
+    return new ByteArrayOutputStream() {
+      @Override
+      public void flush() throws IOException {
+        super.flush();
+        setContent(key, toByteArray(), metadata);
+      }
+    };
+  }
+
+  public synchronized void copy(String sourceKey, String destKey) {
+    blobs.put(destKey, blobs.get(sourceKey));
+  }
+
+  public synchronized void delete(String key) {
+    blobs.remove(key);
+  }
+
+  public synchronized boolean exists(String key) {
+    return blobs.containsKey(key);
+  }
+
+  @SuppressWarnings("unchecked")
+  public synchronized HashMap<String, String> getMetadata(String key) {
+    return (HashMap<String, String>) blobs.get(key).metadata.clone();
+  }
+
+  public synchronized HashMap<String, String> getContainerMetadata() {
+    return containerMetadata;
+  }
+
+  public synchronized void setContainerMetadata(HashMap<String, String> metadata) {
+    containerMetadata = metadata;
+  }
+
+  private static class Entry {
+    private byte[] content;
+    private HashMap<String, String> metadata;
+
+    public Entry(byte[] content, HashMap<String, String> metadata) {
+      this.content = content;
+      this.metadata = metadata;
+    }
+  }
+}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
new file mode 100644
index 0000000..b03997c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
@@ -0,0 +1,433 @@
+/**
+ * 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.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.TimeZone;
+
+import org.apache.commons.httpclient.URIException;
+import org.apache.commons.httpclient.util.URIUtil;
+import org.apache.commons.io.output.ByteArrayOutputStream;
+
+import com.microsoft.windowsazure.storage.CloudStorageAccount;
+import com.microsoft.windowsazure.storage.OperationContext;
+import com.microsoft.windowsazure.storage.RetryPolicyFactory;
+import com.microsoft.windowsazure.storage.StorageCredentials;
+import com.microsoft.windowsazure.storage.StorageException;
+import com.microsoft.windowsazure.storage.StorageUri;
+import com.microsoft.windowsazure.storage.blob.BlobListingDetails;
+import com.microsoft.windowsazure.storage.blob.BlobProperties;
+import com.microsoft.windowsazure.storage.blob.BlobRequestOptions;
+import com.microsoft.windowsazure.storage.blob.CloudBlobContainer;
+import com.microsoft.windowsazure.storage.blob.CloudBlobDirectory;
+import com.microsoft.windowsazure.storage.blob.CopyState;
+import com.microsoft.windowsazure.storage.blob.ListBlobItem;
+
+/**
+ * A mock implementation of the Azure Storage interaction layer for unit tests.
+ * Just does in-memory storage.
+ */
+public class MockStorageInterface extends StorageInterface {
+  private InMemoryBlockBlobStore backingStore;
+  private final ArrayList<PreExistingContainer> preExistingContainers = new ArrayList<MockStorageInterface.PreExistingContainer>();
+  private String baseUriString;
+
+  public InMemoryBlockBlobStore getBackingStore() {
+    return backingStore;
+  }
+
+  /**
+   * Mocks the situation where a container already exists before WASB comes in,
+   * i.e. the situation where a user creates a container then mounts WASB on the
+   * pre-existing container.
+   * 
+   * @param uri
+   *          The URI of the container.
+   * @param metadata
+   *          The metadata on the container.
+   */
+  public void addPreExistingContainer(String uri,
+      HashMap<String, String> metadata) {
+    preExistingContainers.add(new PreExistingContainer(uri, metadata));
+  }
+
+  @Override
+  public void setRetryPolicyFactory(final RetryPolicyFactory retryPolicyFactory) {
+  }
+
+  @Override
+  public void setTimeoutInMs(int timeoutInMs) {
+  }
+
+  @Override
+  public void createBlobClient(CloudStorageAccount account) {
+    backingStore = new InMemoryBlockBlobStore();
+  }
+
+  @Override
+  public void createBlobClient(URI baseUri) {
+    backingStore = new InMemoryBlockBlobStore();
+  }
+
+  @Override
+  public void createBlobClient(URI baseUri, StorageCredentials credentials) {
+    this.baseUriString = baseUri.toString();
+    backingStore = new InMemoryBlockBlobStore();
+  }
+
+  @Override
+  public StorageCredentials getCredentials() {
+    // Not implemented for mock interface.
+    return null;
+  }
+
+  @Override
+  public CloudBlobContainerWrapper getContainerReference(String name)
+      throws URISyntaxException, StorageException {
+    String fullUri;
+    try {
+      fullUri = baseUriString + "/" + URIUtil.encodePath(name);
+    } catch (URIException e) {
+      throw new RuntimeException("problem encoding fullUri", e);
+    }
+
+    MockCloudBlobContainerWrapper container = new MockCloudBlobContainerWrapper(
+        fullUri, name);
+    // Check if we have a pre-existing container with that name, and prime
+    // the wrapper with that knowledge if it's found.
+    for (PreExistingContainer existing : preExistingContainers) {
+      if (fullUri.equalsIgnoreCase(existing.containerUri)) {
+        // We have a pre-existing container. Mark the wrapper as created and
+        // make sure we use the metadata for it.
+        container.created = true;
+        backingStore.setContainerMetadata(existing.containerMetadata);
+        break;
+      }
+    }
+    return container;
+  }
+
+  class MockCloudBlobContainerWrapper extends CloudBlobContainerWrapper {
+    private boolean created = false;
+    private HashMap<String, String> metadata;
+    private final String baseUri;
+    private final String name;
+
+    public MockCloudBlobContainerWrapper(String baseUri, String name) {
+      this.baseUri = baseUri;
+      this.name = name;
+    }
+
+    @Override
+    public String getName() {
+      return name;
+    }
+
+    @Override
+    public boolean exists(OperationContext opContext) throws StorageException {
+      return created;
+    }
+
+    @Override
+    public void create(OperationContext opContext) throws StorageException {
+      created = true;
+      backingStore.setContainerMetadata(metadata);
+    }
+
+    @Override
+    public HashMap<String, String> getMetadata() {
+      return metadata;
+    }
+
+    @Override
+    public void setMetadata(HashMap<String, String> metadata) {
+      this.metadata = metadata;
+    }
+
+    @Override
+    public void downloadAttributes(OperationContext opContext)
+        throws StorageException {
+      metadata = backingStore.getContainerMetadata();
+    }
+
+    @Override
+    public void uploadMetadata(OperationContext opContext)
+        throws StorageException {
+      backingStore.setContainerMetadata(metadata);
+    }
+
+    @Override
+    public CloudBlobDirectoryWrapper getDirectoryReference(String relativePath)
+        throws URISyntaxException, StorageException {
+      return new MockCloudBlobDirectoryWrapper(new URI(fullUriString(
+          relativePath, true)));
+    }
+
+    @Override
+    public CloudBlockBlobWrapper getBlockBlobReference(String relativePath)
+        throws URISyntaxException, StorageException {
+      return new MockCloudBlockBlobWrapper(new URI(fullUriString(relativePath,
+          false)), null, 0);
+    }
+
+    // helper to create full URIs for directory and blob.
+    // use withTrailingSlash=true to get a good path for a directory.
+    private String fullUriString(String relativePath, boolean withTrailingSlash) {
+      String fullUri;
+
+      String baseUri = this.baseUri;
+      if (!baseUri.endsWith("/")) {
+        baseUri += "/";
+      }
+      if (withTrailingSlash && !relativePath.equals("")
+          && !relativePath.endsWith("/")) {
+        relativePath += "/";
+      }
+
+      try {
+        fullUri = baseUri + URIUtil.encodePath(relativePath);
+      } catch (URIException e) {
+        throw new RuntimeException("problem encoding fullUri", e);
+      }
+
+      return fullUri;
+    }
+  }
+
+  private static class PreExistingContainer {
+    final String containerUri;
+    final HashMap<String, String> containerMetadata;
+
+    public PreExistingContainer(String uri, HashMap<String, String> metadata) {
+      this.containerUri = uri;
+      this.containerMetadata = metadata;
+    }
+  }
+
+  class MockCloudBlobDirectoryWrapper extends CloudBlobDirectoryWrapper {
+    private URI uri;
+
+    public MockCloudBlobDirectoryWrapper(URI uri) {
+      this.uri = uri;
+    }
+
+    @Override
+    public CloudBlobContainer getContainer() throws URISyntaxException,
+        StorageException {
+      return null;
+    }
+
+    @Override
+    public CloudBlobDirectory getParent() throws URISyntaxException,
+        StorageException {
+      return null;
+    }
+
+    @Override
+    public URI getUri() {
+      return uri;
+    }
+
+    @Override
+    public Iterable<ListBlobItem> listBlobs(String prefix,
+        boolean useFlatBlobListing, EnumSet<BlobListingDetails> listingDetails,
+        BlobRequestOptions options, OperationContext opContext)
+        throws URISyntaxException, StorageException {
+      ArrayList<ListBlobItem> ret = new ArrayList<ListBlobItem>();
+      String fullPrefix = prefix == null ? uri.toString() : new URI(
+          uri.getScheme(), uri.getAuthority(), uri.getPath() + prefix,
+          uri.getQuery(), uri.getFragment()).toString();
+      boolean includeMetadata = listingDetails
+          .contains(BlobListingDetails.METADATA);
+      HashSet<String> addedDirectories = new HashSet<String>();
+      for (InMemoryBlockBlobStore.ListBlobEntry current : backingStore
+          .listBlobs(fullPrefix, includeMetadata)) {
+        int indexOfSlash = current.getKey().indexOf('/', fullPrefix.length());
+        if (useFlatBlobListing || indexOfSlash < 0) {
+          ret.add(new MockCloudBlockBlobWrapper(new URI(current.getKey()),
+              current.getMetadata(), current.getContentLength()));
+        } else {
+          String directoryName = current.getKey().substring(0, indexOfSlash);
+          if (!addedDirectories.contains(directoryName)) {
+            addedDirectories.add(current.getKey());
+            ret.add(new MockCloudBlobDirectoryWrapper(new URI(directoryName
+                + "/")));
+          }
+        }
+      }
+      return ret;
+    }
+
+    @Override
+    public StorageUri getStorageUri() {
+      throw new UnsupportedOperationException();
+    }
+
+  }
+
+  class MockCloudBlockBlobWrapper extends CloudBlockBlobWrapper {
+    private URI uri;
+    private HashMap<String, String> metadata = new HashMap<String, String>();
+    private BlobProperties properties;
+
+    public MockCloudBlockBlobWrapper(URI uri, HashMap<String, String> metadata,
+        int length) {
+      this.uri = uri;
+      this.metadata = metadata;
+      this.properties = new BlobProperties();
+      this.properties.setLength(length);
+      this.properties.setLastModified(Calendar.getInstance(
+          TimeZone.getTimeZone("UTC")).getTime());
+    }
+
+    private void refreshProperties(boolean getMetadata) {
+      if (backingStore.exists(uri.toString())) {
+        byte[] content = backingStore.getContent(uri.toString());
+        properties = new BlobProperties();
+        properties.setLength(content.length);
+        properties.setLastModified(Calendar.getInstance(
+            TimeZone.getTimeZone("UTC")).getTime());
+        if (getMetadata) {
+          metadata = backingStore.getMetadata(uri.toString());
+        }
+      }
+    }
+
+    @Override
+    public CloudBlobContainer getContainer() throws URISyntaxException,
+        StorageException {
+      return null;
+    }
+
+    @Override
+    public CloudBlobDirectory getParent() throws URISyntaxException,
+        StorageException {
+      return null;
+    }
+
+    @Override
+    public URI getUri() {
+      return uri;
+    }
+
+    @Override
+    public HashMap<String, String> getMetadata() {
+      return metadata;
+    }
+
+    @Override
+    public void setMetadata(HashMap<String, String> metadata) {
+      this.metadata = metadata;
+    }
+
+    @Override
+    public void startCopyFromBlob(CloudBlockBlobWrapper sourceBlob,
+        OperationContext opContext) throws StorageException, URISyntaxException {
+      backingStore.copy(sourceBlob.getUri().toString(), uri.toString());
+      // it would be best if backingStore.properties.CopyState were tracked
+      // If implemented, update azureNativeFileSystemStore.waitForCopyToComplete
+    }
+
+    @Override
+    public CopyState getCopyState() {
+      return this.properties.getCopyState();
+    }
+
+    @Override
+    public void delete(OperationContext opContext) throws StorageException {
+      backingStore.delete(uri.toString());
+    }
+
+    @Override
+    public boolean exists(OperationContext opContext) throws StorageException {
+      return backingStore.exists(uri.toString());
+    }
+
+    @Override
+    public void downloadAttributes(OperationContext opContext)
+        throws StorageException {
+      refreshProperties(true);
+    }
+
+    @Override
+    public BlobProperties getProperties() {
+      return properties;
+    }
+
+    @Override
+    public InputStream openInputStream(BlobRequestOptions options,
+        OperationContext opContext) throws StorageException {
+      return new ByteArrayInputStream(backingStore.getContent(uri.toString()));
+    }
+
+    @Override
+    public OutputStream openOutputStream(BlobRequestOptions options,
+        OperationContext opContext) throws StorageException {
+      return backingStore.upload(uri.toString(), metadata);
+    }
+
+    @Override
+    public void upload(InputStream sourceStream, OperationContext opContext)
+        throws StorageException, IOException {
+      ByteArrayOutputStream allContent = new ByteArrayOutputStream();
+      allContent.write(sourceStream);
+      backingStore.setContent(uri.toString(), allContent.toByteArray(),
+          metadata);
+      refreshProperties(false);
+      allContent.close();
+    }
+
+    @Override
+    public void uploadMetadata(OperationContext opContext)
+        throws StorageException {
+      backingStore.setContent(uri.toString(),
+          backingStore.getContent(uri.toString()), metadata);
+    }
+
+    @Override
+    public void uploadProperties(OperationContext opContext)
+        throws StorageException {
+      refreshProperties(false);
+    }
+
+    @Override
+    public void setStreamMinimumReadSizeInBytes(int minimumReadSize) {
+    }
+
+    @Override
+    public void setWriteBlockSizeInBytes(int writeBlockSizeInBytes) {
+    }
+
+    @Override
+    public StorageUri getStorageUri() {
+      throw new UnsupportedOperationException();
+    }
+
+  }
+}
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
new file mode 100644
index 0000000..bc7e344
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemBaseTest.java
@@ -0,0 +1,584 @@
+/**
+ * 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.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeNotNull;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.TimeZone;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/*
+ * Tests the Native Azure file system (WASB) against an actual blob store if
+ * provided in the environment.
+ * Subclasses implement createTestAccount() to hit local&mock storage with the same test code.
+ * 
+ * For hand-testing: remove "abstract" keyword and copy in an implementation of createTestAccount
+ * from one of the subclasses
+ */
+public abstract class NativeAzureFileSystemBaseTest {
+
+  private FileSystem fs;
+  private AzureBlobStorageTestAccount testAccount;
+  private final long modifiedTimeErrorMargin = 5 * 1000; // Give it +/-5 seconds
+
+  protected abstract AzureBlobStorageTestAccount createTestAccount()
+      throws Exception;
+
+  @Before
+  public void setUp() throws Exception {
+    testAccount = createTestAccount();
+    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 testCheckingNonExistentOneLetterFile() throws Exception {
+    assertFalse(fs.exists(new Path("/a")));
+  }
+
+  @Test
+  public void testStoreRetrieveFile() throws Exception {
+    Path testFile = new Path("unit-test-file");
+    writeString(testFile, "Testing");
+    assertTrue(fs.exists(testFile));
+    FileStatus status = fs.getFileStatus(testFile);
+    assertNotNull(status);
+    // By default, files should be have masked permissions
+    // that grant RW to user, and R to group/other
+    assertEquals(new FsPermission((short) 0644), status.getPermission());
+    assertEquals("Testing", readString(testFile));
+    fs.delete(testFile, true);
+  }
+
+  @Test
+  public void testStoreDeleteFolder() throws Exception {
+    Path testFolder = new Path("storeDeleteFolder");
+    assertFalse(fs.exists(testFolder));
+    assertTrue(fs.mkdirs(testFolder));
+    assertTrue(fs.exists(testFolder));
+    FileStatus status = fs.getFileStatus(testFolder);
+    assertNotNull(status);
+    assertTrue(status.isDirectory());
+    // By default, directories should be have masked permissions
+    // that grant RWX to user, and RX to group/other
+    assertEquals(new FsPermission((short) 0755), status.getPermission());
+    Path innerFile = new Path(testFolder, "innerFile");
+    assertTrue(fs.createNewFile(innerFile));
+    assertTrue(fs.exists(innerFile));
+    assertTrue(fs.delete(testFolder, true));
+    assertFalse(fs.exists(innerFile));
+    assertFalse(fs.exists(testFolder));
+  }
+
+  @Test
+  public void testFileOwnership() throws Exception {
+    Path testFile = new Path("ownershipTestFile");
+    writeString(testFile, "Testing");
+    testOwnership(testFile);
+  }
+
+  @Test
+  public void testFolderOwnership() throws Exception {
+    Path testFolder = new Path("ownershipTestFolder");
+    fs.mkdirs(testFolder);
+    testOwnership(testFolder);
+  }
+
+  private void testOwnership(Path pathUnderTest) throws IOException {
+    FileStatus ret = fs.getFileStatus(pathUnderTest);
+    UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
+    assertEquals(ret.getOwner(), currentUser.getShortUserName());
+    fs.delete(pathUnderTest, true);
+  }
+
+  private static FsPermission ignoreStickyBit(FsPermission original) {
+    return new FsPermission(original.getUserAction(),
+        original.getGroupAction(), original.getOtherAction());
+  }
+
+  // When FsPermission applies a UMask, it loses sticky bit information.
+  // And since we always apply UMask, we should ignore whether the sticky
+  // bit is equal or not.
+  private static void assertEqualsIgnoreStickyBit(FsPermission expected,
+      FsPermission actual) {
+    assertEquals(ignoreStickyBit(expected), ignoreStickyBit(actual));
+  }
+
+  @Test
+  public void testFilePermissions() throws Exception {
+    Path testFile = new Path("permissionTestFile");
+    FsPermission permission = FsPermission.createImmutable((short) 644);
+    createEmptyFile(testFile, permission);
+    FileStatus ret = fs.getFileStatus(testFile);
+    assertEqualsIgnoreStickyBit(permission, ret.getPermission());
+    fs.delete(testFile, true);
+  }
+
+  @Test
+  public void testFolderPermissions() throws Exception {
+    Path testFolder = new Path("permissionTestFolder");
+    FsPermission permission = FsPermission.createImmutable((short) 644);
+    fs.mkdirs(testFolder, permission);
+    FileStatus ret = fs.getFileStatus(testFolder);
+    assertEqualsIgnoreStickyBit(permission, ret.getPermission());
+    fs.delete(testFolder, true);
+  }
+
+  @Test
+  public void testDeepFileCreation() throws Exception {
+    Path testFile = new Path("deep/file/creation/test");
+    FsPermission permission = FsPermission.createImmutable((short) 644);
+    createEmptyFile(testFile, permission);
+    assertTrue(fs.exists(testFile));
+    assertTrue(fs.exists(new Path("deep")));
+    assertTrue(fs.exists(new Path("deep/file/creation")));
+    FileStatus ret = fs.getFileStatus(new Path("deep/file"));
+    assertTrue(ret.isDirectory());
+    assertEqualsIgnoreStickyBit(permission, ret.getPermission());
+    assertTrue(fs.delete(new Path("deep"), true));
+    assertFalse(fs.exists(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
+    // doesn't actually work as expected right now.
+  }
+
+  private static enum RenameVariation {
+    NormalFileName, SourceInAFolder, SourceWithSpace, SourceWithPlusAndPercent
+  }
+
+  @Test
+  public void testRename() throws Exception {
+    for (RenameVariation variation : RenameVariation.values()) {
+      System.out.printf("Rename variation: %s\n", variation);
+      Path originalFile;
+      switch (variation) {
+      case NormalFileName:
+        originalFile = new Path("fileToRename");
+        break;
+      case SourceInAFolder:
+        originalFile = new Path("file/to/rename");
+        break;
+      case SourceWithSpace:
+        originalFile = new Path("file to rename");
+        break;
+      case SourceWithPlusAndPercent:
+        originalFile = new Path("file+to%rename");
+        break;
+      default:
+        throw new Exception("Unknown variation");
+      }
+      Path destinationFile = new Path("file/resting/destination");
+      assertTrue(fs.createNewFile(originalFile));
+      assertTrue(fs.exists(originalFile));
+      assertFalse(fs.rename(originalFile, destinationFile)); // Parent directory
+      // doesn't exist
+      assertTrue(fs.mkdirs(destinationFile.getParent()));
+      assertTrue(fs.rename(originalFile, destinationFile));
+      assertTrue(fs.exists(destinationFile));
+      assertFalse(fs.exists(originalFile));
+      fs.delete(destinationFile.getParent(), true);
+    }
+  }
+
+  @Test
+  public void testRenameImplicitFolder() throws Exception {
+    Path testFile = new Path("deep/file/rename/test");
+    FsPermission permission = FsPermission.createImmutable((short) 644);
+    createEmptyFile(testFile, permission);
+    assertTrue(fs.rename(new Path("deep/file"), new Path("deep/renamed")));
+    assertFalse(fs.exists(testFile));
+    FileStatus newStatus = fs
+        .getFileStatus(new Path("deep/renamed/rename/test"));
+    assertNotNull(newStatus);
+    assertEqualsIgnoreStickyBit(permission, newStatus.getPermission());
+    assertTrue(fs.delete(new Path("deep"), true));
+  }
+
+  private static enum RenameFolderVariation {
+    CreateFolderAndInnerFile, CreateJustInnerFile, CreateJustFolder
+  }
+
+  @Test
+  public void testRenameFolder() throws Exception {
+    for (RenameFolderVariation variation : RenameFolderVariation.values()) {
+      Path originalFolder = new Path("folderToRename");
+      if (variation != RenameFolderVariation.CreateJustInnerFile){
+        assertTrue(fs.mkdirs(originalFolder));
+      }
+      Path innerFile = new Path(originalFolder, "innerFile");
+      if (variation != RenameFolderVariation.CreateJustFolder){
+        assertTrue(fs.createNewFile(innerFile));
+      }
+      Path destination = new Path("renamedFolder");
+      assertTrue(fs.rename(originalFolder, destination));
+      assertTrue(fs.exists(destination));
+      if (variation != RenameFolderVariation.CreateJustFolder){
+        assertTrue(fs.exists(new Path(destination, innerFile.getName())));
+      }
+      assertFalse(fs.exists(originalFolder));
+      assertFalse(fs.exists(innerFile));
+      fs.delete(destination, true);
+    }
+  }
+
+  @Test
+  public void testCopyFromLocalFileSystem() throws Exception {
+    Path localFilePath = new Path(System.getProperty("test.build.data",
+        "azure_test"));
+    FileSystem localFs = FileSystem.get(new Configuration());
+    localFs.delete(localFilePath, true);
+    try {
+      writeString(localFs, localFilePath, "Testing");
+      Path dstPath = new Path("copiedFromLocal");
+      assertTrue(FileUtil.copy(localFs, localFilePath, fs, dstPath, false,
+          fs.getConf()));
+      assertTrue(fs.exists(dstPath));
+      assertEquals("Testing", readString(fs, dstPath));
+      fs.delete(dstPath, true);
+    } finally {
+      localFs.delete(localFilePath, true);
+    }
+  }
+
+  @Test
+  public void testListDirectory() throws Exception {
+    Path rootFolder = new Path("testingList");
+    assertTrue(fs.mkdirs(rootFolder));
+    FileStatus[] listed = fs.listStatus(rootFolder);
+    assertEquals(0, listed.length);
+    Path innerFolder = new Path(rootFolder, "inner");
+    assertTrue(fs.mkdirs(innerFolder));
+    listed = fs.listStatus(rootFolder);
+    assertEquals(1, listed.length);
+    assertTrue(listed[0].isDirectory());
+    Path innerFile = new Path(innerFolder, "innerFile");
+    writeString(innerFile, "testing");
+    listed = fs.listStatus(rootFolder);
+    assertEquals(1, listed.length);
+    assertTrue(listed[0].isDirectory());
+    listed = fs.listStatus(innerFolder);
+    assertEquals(1, listed.length);
+    assertFalse(listed[0].isDirectory());
+    assertTrue(fs.delete(rootFolder, true));
+  }
+
+  @Test
+  public void testStatistics() throws Exception {
+    FileSystem.clearStatistics();
+    FileSystem.Statistics stats = FileSystem.getStatistics("wasb",
+        NativeAzureFileSystem.class);
+    assertEquals(0, stats.getBytesRead());
+    assertEquals(0, stats.getBytesWritten());
+    Path newFile = new Path("testStats");
+    writeString(newFile, "12345678");
+    assertEquals(8, stats.getBytesWritten());
+    assertEquals(0, stats.getBytesRead());
+    String readBack = readString(newFile);
+    assertEquals("12345678", readBack);
+    assertEquals(8, stats.getBytesRead());
+    assertEquals(8, stats.getBytesWritten());
+    assertTrue(fs.delete(newFile, true));
+    assertEquals(8, stats.getBytesRead());
+    assertEquals(8, stats.getBytesWritten());
+  }
+
+  @Test
+  public void testUriEncoding() throws Exception {
+    fs.create(new Path("p/t%5Fe")).close();
+    FileStatus[] listing = fs.listStatus(new Path("p"));
+    assertEquals(1, listing.length);
+    assertEquals("t%5Fe", listing[0].getPath().getName());
+    assertTrue(fs.rename(new Path("p"), new Path("q")));
+    assertTrue(fs.delete(new Path("q"), true));
+  }
+
+  @Test
+  public void testUriEncodingMoreComplexCharacters() throws Exception {
+    // Create a file name with URI reserved characters, plus the percent
+    String fileName = "!#$'()*;=[]%";
+    String directoryName = "*;=[]%!#$'()";
+    fs.create(new Path(directoryName, fileName)).close();
+    FileStatus[] listing = fs.listStatus(new Path(directoryName));
+    assertEquals(1, listing.length);
+    assertEquals(fileName, listing[0].getPath().getName());
+    FileStatus status = fs.getFileStatus(new Path(directoryName, fileName));
+    assertEquals(fileName, status.getPath().getName());
+    InputStream stream = fs.open(new Path(directoryName, fileName));
+    assertNotNull(stream);
+    stream.close();
+    assertTrue(fs.delete(new Path(directoryName, fileName), true));
+    assertTrue(fs.delete(new Path(directoryName), true));
+  }
+
+  @Test
+  public void testReadingDirectoryAsFile() throws Exception {
+    Path dir = new Path("/x");
+    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());
+    }
+  }
+
+  @Test
+  public void testCreatingFileOverDirectory() throws Exception {
+    Path dir = new Path("/x");
+    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());
+    }
+  }
+
+  @Test
+  public void testSetPermissionOnFile() throws Exception {
+    Path newFile = new Path("testPermission");
+    OutputStream output = fs.create(newFile);
+    output.write(13);
+    output.close();
+    FsPermission newPermission = new FsPermission((short) 0700);
+    fs.setPermission(newFile, newPermission);
+    FileStatus newStatus = fs.getFileStatus(newFile);
+    assertNotNull(newStatus);
+    assertEquals(newPermission, newStatus.getPermission());
+    assertEquals("supergroup", newStatus.getGroup());
+    assertEquals(UserGroupInformation.getCurrentUser().getShortUserName(),
+        newStatus.getOwner());
+    assertEquals(1, newStatus.getLen());
+  }
+
+  @Test
+  public void testSetPermissionOnFolder() throws Exception {
+    Path newFolder = new Path("testPermission");
+    assertTrue(fs.mkdirs(newFolder));
+    FsPermission newPermission = new FsPermission((short) 0600);
+    fs.setPermission(newFolder, newPermission);
+    FileStatus newStatus = fs.getFileStatus(newFolder);
+    assertNotNull(newStatus);
+    assertEquals(newPermission, newStatus.getPermission());
+    assertTrue(newStatus.isDirectory());
+  }
+
+  @Test
+  public void testSetOwnerOnFile() throws Exception {
+    Path newFile = new Path("testOwner");
+    OutputStream output = fs.create(newFile);
+    output.write(13);
+    output.close();
+    fs.setOwner(newFile, "newUser", null);
+    FileStatus newStatus = fs.getFileStatus(newFile);
+    assertNotNull(newStatus);
+    assertEquals("newUser", newStatus.getOwner());
+    assertEquals("supergroup", newStatus.getGroup());
+    assertEquals(1, newStatus.getLen());
+    fs.setOwner(newFile, null, "newGroup");
+    newStatus = fs.getFileStatus(newFile);
+    assertNotNull(newStatus);
+    assertEquals("newUser", newStatus.getOwner());
+    assertEquals("newGroup", newStatus.getGroup());
+  }
+
+  @Test
+  public void testSetOwnerOnFolder() throws Exception {
+    Path newFolder = new Path("testOwner");
+    assertTrue(fs.mkdirs(newFolder));
+    fs.setOwner(newFolder, "newUser", null);
+    FileStatus newStatus = fs.getFileStatus(newFolder);
+    assertNotNull(newStatus);
+    assertEquals("newUser", newStatus.getOwner());
+    assertTrue(newStatus.isDirectory());
+  }
+
+  @Test
+  public void testModifiedTimeForFile() throws Exception {
+    Path testFile = new Path("testFile");
+    fs.create(testFile).close();
+    testModifiedTime(testFile);
+  }
+
+  @Test
+  public void testModifiedTimeForFolder() throws Exception {
+    Path testFolder = new Path("testFolder");
+    assertTrue(fs.mkdirs(testFolder));
+    testModifiedTime(testFolder);
+  }
+
+  @Test
+  public void testFolderLastModifiedTime() throws Exception {
+    Path parentFolder = new Path("testFolder");
+    Path innerFile = new Path(parentFolder, "innerfile");
+    assertTrue(fs.mkdirs(parentFolder));
+
+    // Create file
+    long lastModifiedTime = fs.getFileStatus(parentFolder)
+        .getModificationTime();
+    // Wait at least the error margin
+    Thread.sleep(modifiedTimeErrorMargin + 1);
+    assertTrue(fs.createNewFile(innerFile));
+    // The parent folder last modified time should have changed because we
+    // create an inner file.
+    assertFalse(testModifiedTime(parentFolder, lastModifiedTime));
+    testModifiedTime(parentFolder);
+
+    // Rename file
+    lastModifiedTime = fs.getFileStatus(parentFolder).getModificationTime();
+    Path destFolder = new Path("testDestFolder");
+    assertTrue(fs.mkdirs(destFolder));
+    long destLastModifiedTime = fs.getFileStatus(destFolder)
+        .getModificationTime();
+    Thread.sleep(modifiedTimeErrorMargin + 1);
+    Path destFile = new Path(destFolder, "innerfile");
+    assertTrue(fs.rename(innerFile, destFile));
+    // Both source and destination folder last modified time should have changed
+    // because of renaming.
+    assertFalse(testModifiedTime(parentFolder, lastModifiedTime));
+    assertFalse(testModifiedTime(destFolder, destLastModifiedTime));
+    testModifiedTime(parentFolder);
+    testModifiedTime(destFolder);
+
+    // Delete file
+    destLastModifiedTime = fs.getFileStatus(destFolder).getModificationTime();
+    // Wait at least the error margin
+    Thread.sleep(modifiedTimeErrorMargin + 1);
+    fs.delete(destFile, false);
+    // The parent folder last modified time should have changed because we
+    // delete an inner file.
+    assertFalse(testModifiedTime(destFolder, destLastModifiedTime));
+    testModifiedTime(destFolder);
+  }
+
+  @Test
+  public void testListSlash() throws Exception {
+    Path testFolder = new Path("/testFolder");
+    Path testFile = new Path(testFolder, "testFile");
+    assertTrue(fs.mkdirs(testFolder));
+    assertTrue(fs.createNewFile(testFile));
+    FileStatus status = fs.getFileStatus(new Path("/testFolder/."));
+    assertNotNull(status);
+  }
+
+  private boolean testModifiedTime(Path testPath, long time) throws Exception {
+    FileStatus fileStatus = fs.getFileStatus(testPath);
+    final long errorMargin = modifiedTimeErrorMargin;
+    long lastModified = fileStatus.getModificationTime();
+    return (lastModified > (time - errorMargin) && lastModified < (time + errorMargin));
+  }
+
+  private void testModifiedTime(Path testPath) throws Exception {
+    Calendar utc = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
+    long currentUtcTime = utc.getTime().getTime();
+    FileStatus fileStatus = fs.getFileStatus(testPath);
+    assertTrue("Modification time "
+        + new Date(fileStatus.getModificationTime()) + " is not close to now: "
+        + utc.getTime(), testModifiedTime(testPath, currentUtcTime));
+  }
+
+   private void createEmptyFile(Path testFile, FsPermission permission)
+      throws IOException {
+    FSDataOutputStream outputStream = fs.create(testFile, permission, true,
+        4096, (short) 1, 1024, null);
+    outputStream.close();
+  }
+
+  private String readString(Path testFile) throws IOException {
+    return readString(fs, testFile);
+  }
+
+  private String readString(FileSystem fs, Path testFile) throws IOException {
+    FSDataInputStream inputStream = fs.open(testFile);
+    String ret = readString(inputStream);
+    inputStream.close();
+    return ret;
+  }
+
+  private String readString(FSDataInputStream inputStream) throws IOException {
+    BufferedReader reader = new BufferedReader(new InputStreamReader(
+        inputStream));
+    final int BUFFER_SIZE = 1024;
+    char[] buffer = new char[BUFFER_SIZE];
+    int count = reader.read(buffer, 0, BUFFER_SIZE);
+    if (count >= BUFFER_SIZE) {
+      throw new IOException("Exceeded buffer size");
+    }
+    inputStream.close();
+    return new String(buffer, 0, count);
+  }
+
+  private void writeString(Path path, String value) throws IOException {
+    writeString(fs, path, value);
+  }
+
+  private void writeString(FileSystem fs, Path path, String value)
+      throws IOException {
+    FSDataOutputStream outputStream = fs.create(path, true);
+    writeString(outputStream, value);
+    outputStream.close();
+  }
+
+  private void writeString(FSDataOutputStream outputStream, String value)
+      throws IOException {
+    BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(
+        outputStream));
+    writer.write(value);
+    writer.close();
+  }
+}
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
new file mode 100644
index 0000000..c10ac0f
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureConcurrentOutOfBandIo.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.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeNotNull;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+
+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.
+  private static final int NUMBER_OF_BLOCKS = 2;
+
+  private 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];
+
+      DataOutputStream 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()));
+
+          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.
+    DataOutputStream outputStream = testAccount.getStore().storefile(
+        "WASB_String.txt",
+        new PermissionStatus("", "", FsPermission.getDefault()));
+    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;
+    DataInputStream inputStream = null;
+
+    for (int i = 0; i < 5; i++) {
+      try {
+        inputStream = testAccount.getStore().retrieve("WASB_String.txt", 0);
+        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
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
new file mode 100644
index 0000000..88d976c
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestAzureFileSystemErrorConditions.java
@@ -0,0 +1,257 @@
+/**
+ * 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 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.FSDataInputStream;
+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.fs.permission.FsPermission;
+import org.junit.Test;
+
+import com.microsoft.windowsazure.storage.OperationContext;
+import com.microsoft.windowsazure.storage.SendingRequestEvent;
+import com.microsoft.windowsazure.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 {
+    Configuration conf = new Configuration();
+    AzureBlobStorageTestAccount.addWasbToConfiguration(conf);
+    Path noAccessPath = new Path(
+        "wasb://nonExistentContainer@hopefullyNonExistentAccount/someFile");
+    NativeAzureFileSystem.suppressRetryPolicy();
+    try {
+      FileSystem.get(noAccessPath.toUri(), conf).open(noAccessPath);
+      assertTrue("Should've thrown.", false);
+    } catch (AzureException ex) {
+      assertTrue("Unexpected message in exception " + ex, ex.getMessage()
+          .contains(
+              "Unable to access container nonExistentContainer in account"
+                  + " hopefullyNonExistentAccount"));
+    } 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().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();
+    }
+  }
+  
+  // Tests an error during stream creation (in this case in the seek() implementation
+  // to verify the close-stream-on-error logic.
+  @Test (expected=AzureException.class)
+  public void testErrorDuringRetrieve() throws Exception {
+    NativeAzureFileSystem fs = AzureBlobStorageTestAccount.createMock().getFileSystem();
+    Path testFile = new Path("/testErrorDuringRetrieve");
+    writeAllThreeFile(fs, testFile);
+
+    FSDataInputStream stream = fs.open(testFile);
+    stream.seek(Integer.MAX_VALUE);
+  }
+}
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
new file mode 100644
index 0000000..b585c56
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobDataValidation.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 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.assertNotNull;
+import static org.junit.Assert.assertNull;
+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.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.windowsazure.storage.Constants;
+import com.microsoft.windowsazure.storage.OperationContext;
+import com.microsoft.windowsazure.storage.ResponseReceivedEvent;
+import com.microsoft.windowsazure.storage.StorageErrorCodeStrings;
+import com.microsoft.windowsazure.storage.StorageEvent;
+import com.microsoft.windowsazure.storage.StorageException;
+import com.microsoft.windowsazure.storage.blob.BlockEntry;
+import com.microsoft.windowsazure.storage.blob.BlockSearchMode;
+import com.microsoft.windowsazure.storage.blob.CloudBlockBlob;
+import com.microsoft.windowsazure.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);
+      assertTrue("Unexpected cause: " + cause,
+          cause.getErrorCode().equals(StorageErrorCodeStrings.INVALID_MD5));
+    }
+  }
+
+  /**
+   * 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().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();
+  }
+}
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
new file mode 100644
index 0000000..b75fc38
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestBlobMetadata.java
@@ -0,0 +1,265 @@
+/**
+ * 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.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.URI;
+import java.util.HashMap;
+
+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.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Tests that we put the correct metadata on blobs created through WASB.
+ */
+public class TestBlobMetadata {
+  private AzureBlobStorageTestAccount testAccount;
+  private FileSystem fs;
+  private InMemoryBlockBlobStore backingStore;
+
+  @Before
+  public void setUp() throws Exception {
+    testAccount = AzureBlobStorageTestAccount.createMock();
+    fs = testAccount.getFileSystem();
+    backingStore = testAccount.getMockStorage().getBackingStore();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    testAccount.cleanup();
+    fs = null;
+    backingStore = null;
+  }
+
+  private static String getExpectedOwner() throws Exception {
+    return UserGroupInformation.getCurrentUser().getShortUserName();
+  }
+
+  private static String getExpectedPermissionString(String permissionString)
+      throws Exception {
+    return String.format(
+        "{\"owner\":\"%s\",\"group\":\"%s\",\"permissions\":\"%s\"}",
+        getExpectedOwner(), NativeAzureFileSystem.AZURE_DEFAULT_GROUP_DEFAULT,
+        permissionString);
+  }
+
+  /**
+   * Tests that WASB stamped the version in the container metadata.
+   */
+  @Test
+  public void testContainerVersionMetadata() throws Exception {
+    // Do a write operation to trigger version stamp
+    fs.createNewFile(new Path("/foo"));
+    HashMap<String, String> containerMetadata = backingStore
+        .getContainerMetadata();
+    assertNotNull(containerMetadata);
+    assertEquals(AzureNativeFileSystemStore.CURRENT_WASB_VERSION,
+        containerMetadata.get(AzureNativeFileSystemStore.VERSION_METADATA_KEY));
+  }
+
+  private static final class FsWithPreExistingContainer implements Closeable {
+    private final MockStorageInterface mockStorage;
+    private final NativeAzureFileSystem fs;
+
+    private FsWithPreExistingContainer(MockStorageInterface mockStorage,
+        NativeAzureFileSystem fs) {
+      this.mockStorage = mockStorage;
+      this.fs = fs;
+    }
+
+    public NativeAzureFileSystem getFs() {
+      return fs;
+    }
+
+    public HashMap<String, String> getContainerMetadata() {
+      return mockStorage.getBackingStore().getContainerMetadata();
+    }
+
+    public static FsWithPreExistingContainer create() throws Exception {
+      return create(null);
+    }
+
+    public static FsWithPreExistingContainer create(
+        HashMap<String, String> containerMetadata) throws Exception {
+      AzureNativeFileSystemStore store = new AzureNativeFileSystemStore();
+      MockStorageInterface mockStorage = new MockStorageInterface();
+      store.setAzureStorageInteractionLayer(mockStorage);
+      NativeAzureFileSystem fs = new NativeAzureFileSystem(store);
+      Configuration conf = new Configuration();
+      AzureBlobStorageTestAccount.setMockAccountKey(conf);
+      mockStorage.addPreExistingContainer(
+          AzureBlobStorageTestAccount.getMockContainerUri(), containerMetadata);
+      fs.initialize(new URI(AzureBlobStorageTestAccount.MOCK_WASB_URI), conf);
+      return new FsWithPreExistingContainer(mockStorage, fs);
+    }
+
+    @Override
+    public void close() throws IOException {
+      fs.close();
+    }
+  }
+
+  /**
+   * Tests that WASB stamped the version in the container metadata if it does a
+   * write operation to a pre-existing container.
+   */
+  @Test
+  public void testPreExistingContainerVersionMetadata() throws Exception {
+    // Create a mock storage with a pre-existing container that has no
+    // WASB version metadata on it.
+    FsWithPreExistingContainer fsWithContainer = FsWithPreExistingContainer
+        .create();
+
+    // Now, do some read operations (should touch the metadata)
+    assertFalse(fsWithContainer.getFs().exists(new Path("/IDontExist")));
+    assertEquals(0, fsWithContainer.getFs().listStatus(new Path("/")).length);
+
+    // Check that no container metadata exists yet
+    assertNull(fsWithContainer.getContainerMetadata());
+
+    // Now do a write operation - should stamp the version
+    fsWithContainer.getFs().mkdirs(new Path("/dir"));
+
+    // Check that now we have the version stamp
+    assertNotNull(fsWithContainer.getContainerMetadata());
+    assertEquals(
+        AzureNativeFileSystemStore.CURRENT_WASB_VERSION,
+        fsWithContainer.getContainerMetadata().get(
+            AzureNativeFileSystemStore.VERSION_METADATA_KEY));
+    fsWithContainer.close();
+  }
+
+  /**
+   * Tests that WASB works well with an older version container with ASV-era
+   * version and metadata.
+   */
+  @Test
+  public void testFirstContainerVersionMetadata() throws Exception {
+    // Create a mock storage with a pre-existing container that has
+    // ASV version metadata on it.
+    HashMap<String, String> containerMetadata = new HashMap<String, String>();
+    containerMetadata.put(AzureNativeFileSystemStore.OLD_VERSION_METADATA_KEY,
+        AzureNativeFileSystemStore.FIRST_WASB_VERSION);
+    FsWithPreExistingContainer fsWithContainer = FsWithPreExistingContainer
+        .create(containerMetadata);
+
+    // Now, do some read operations (should touch the metadata)
+    assertFalse(fsWithContainer.getFs().exists(new Path("/IDontExist")));
+    assertEquals(0, fsWithContainer.getFs().listStatus(new Path("/")).length);
+
+    // Check that no container metadata exists yet
+    assertEquals(
+        AzureNativeFileSystemStore.FIRST_WASB_VERSION,
+        fsWithContainer.getContainerMetadata().get(
+            AzureNativeFileSystemStore.OLD_VERSION_METADATA_KEY));
+    assertNull(fsWithContainer.getContainerMetadata().get(
+        AzureNativeFileSystemStore.VERSION_METADATA_KEY));
+
+    // Now do a write operation - should stamp the version
+    fsWithContainer.getFs().mkdirs(new Path("/dir"));
+
+    // Check that now we have the version stamp
+    assertEquals(
+        AzureNativeFileSystemStore.CURRENT_WASB_VERSION,
+        fsWithContainer.getContainerMetadata().get(
+            AzureNativeFileSystemStore.VERSION_METADATA_KEY));
+    assertNull(fsWithContainer.getContainerMetadata().get(
+        AzureNativeFileSystemStore.OLD_VERSION_METADATA_KEY));
+    fsWithContainer.close();
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test
+  public void testPermissionMetadata() throws Exception {
+    FsPermission justMe = new FsPermission(FsAction.READ_WRITE, FsAction.NONE,
+        FsAction.NONE);
+    Path selfishFile = new Path("/noOneElse");
+    fs.create(selfishFile, justMe, true, 4096, fs.getDefaultReplication(),
+        fs.getDefaultBlockSize(), null).close();
+    HashMap<String, String> metadata = backingStore
+        .getMetadata(AzureBlobStorageTestAccount.toMockUri(selfishFile));
+    assertNotNull(metadata);
+    String storedPermission = metadata.get("hdi_permission");
+    assertEquals(getExpectedPermissionString("rw-------"), storedPermission);
+    FileStatus retrievedStatus = fs.getFileStatus(selfishFile);
+    assertNotNull(retrievedStatus);
+    assertEquals(justMe, retrievedStatus.getPermission());
+    assertEquals(getExpectedOwner(), retrievedStatus.getOwner());
+    assertEquals(NativeAzureFileSystem.AZURE_DEFAULT_GROUP_DEFAULT,
+        retrievedStatus.getGroup());
+  }
+
+  /**
+   * Tests that WASB understands the old-style ASV metadata and changes it when
+   * it gets the chance.
+   */
+  @Test
+  public void testOldPermissionMetadata() throws Exception {
+    Path selfishFile = new Path("/noOneElse");
+    HashMap<String, String> metadata = new HashMap<String, String>();
+    metadata.put("asv_permission", getExpectedPermissionString("rw-------"));
+    backingStore.setContent(AzureBlobStorageTestAccount.toMockUri(selfishFile),
+        new byte[] {}, metadata);
+    FsPermission justMe = new FsPermission(FsAction.READ_WRITE, FsAction.NONE,
+        FsAction.NONE);
+    FileStatus retrievedStatus = fs.getFileStatus(selfishFile);
+    assertNotNull(retrievedStatus);
+    assertEquals(justMe, retrievedStatus.getPermission());
+    assertEquals(getExpectedOwner(), retrievedStatus.getOwner());
+    assertEquals(NativeAzureFileSystem.AZURE_DEFAULT_GROUP_DEFAULT,
+        retrievedStatus.getGroup());
+    FsPermission meAndYou = new FsPermission(FsAction.READ_WRITE,
+        FsAction.READ_WRITE, FsAction.NONE);
+    fs.setPermission(selfishFile, meAndYou);
+    metadata = backingStore.getMetadata(AzureBlobStorageTestAccount
+        .toMockUri(selfishFile));
+    assertNotNull(metadata);
+    String storedPermission = metadata.get("hdi_permission");
+    assertEquals(getExpectedPermissionString("rw-rw----"), storedPermission);
+    assertNull(metadata.get("asv_permission"));
+  }
+
+  @Test
+  public void testFolderMetadata() throws Exception {
+    Path folder = new Path("/folder");
+    FsPermission justRead = new FsPermission(FsAction.READ, FsAction.READ,
+        FsAction.READ);
+    fs.mkdirs(folder, justRead);
+    HashMap<String, String> metadata = backingStore
+        .getMetadata(AzureBlobStorageTestAccount.toMockUri(folder));
+    assertNotNull(metadata);
+    assertEquals("true", metadata.get("hdi_isfolder"));
+    assertEquals(getExpectedPermissionString("r--r--r--"),
+        metadata.get("hdi_permission"));
+  }
+}
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
new file mode 100644
index 0000000..727f540
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestContainerChecks.java
@@ -0,0 +1,175 @@
+/**
+ * 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.Test;
+
+import com.microsoft.windowsazure.storage.blob.BlobOutputStream;
+import com.microsoft.windowsazure.storage.blob.CloudBlobContainer;
+import com.microsoft.windowsazure.storage.blob.CloudBlockBlob;
+
+/**
+ * Tests that WASB creates containers only if needed.
+ */
+public class TestContainerChecks {
+  private AzureBlobStorageTestAccount testAccount;
+
+  @After
+  public void tearDown() throws Exception {
+    if (testAccount != null) {
+      testAccount.cleanup();
+      testAccount = null;
+    }
+  }
+
+  @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 {
+    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());
+  }
+}
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
new file mode 100644
index 0000000..b2660bb
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemBlockLocations.java
@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.OutputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+public class TestNativeAzureFileSystemBlockLocations {
+  @Test
+  public void testNumberOfBlocks() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(NativeAzureFileSystem.AZURE_BLOCK_SIZE_PROPERTY_NAME, "500");
+    AzureBlobStorageTestAccount testAccount = AzureBlobStorageTestAccount
+        .createMock(conf);
+    FileSystem fs = testAccount.getFileSystem();
+    Path testFile = createTestFile(fs, 1200);
+    FileStatus stat = fs.getFileStatus(testFile);
+    assertEquals(500, stat.getBlockSize());
+    testAccount.cleanup();
+  }
+
+  @Test
+  public void testBlockLocationsTypical() throws Exception {
+    BlockLocation[] locations = getBlockLocationsOutput(210, 50, 0, 210);
+    assertEquals(5, locations.length);
+    assertEquals("localhost", locations[0].getHosts()[0]);
+    assertEquals(50, locations[0].getLength());
+    assertEquals(10, locations[4].getLength());
+    assertEquals(100, locations[2].getOffset());
+  }
+
+  @Test
+  public void testBlockLocationsEmptyFile() throws Exception {
+    BlockLocation[] locations = getBlockLocationsOutput(0, 50, 0, 0);
+    assertEquals(0, locations.length);
+  }
+
+  @Test
+  public void testBlockLocationsSmallFile() throws Exception {
+    BlockLocation[] locations = getBlockLocationsOutput(1, 50, 0, 1);
+    assertEquals(1, locations.length);
+    assertEquals(1, locations[0].getLength());
+  }
+
+  @Test
+  public void testBlockLocationsExactBlockSizeMultiple() throws Exception {
+    BlockLocation[] locations = getBlockLocationsOutput(200, 50, 0, 200);
+    assertEquals(4, locations.length);
+    assertEquals(150, locations[3].getOffset());
+    assertEquals(50, locations[3].getLength());
+  }
+
+  @Test
+  public void testBlockLocationsSubsetOfFile() throws Exception {
+    BlockLocation[] locations = getBlockLocationsOutput(205, 10, 15, 35);
+    assertEquals(4, locations.length);
+    assertEquals(10, locations[0].getLength());
+    assertEquals(15, locations[0].getOffset());
+    assertEquals(5, locations[3].getLength());
+    assertEquals(45, locations[3].getOffset());
+  }
+
+  @Test
+  public void testBlockLocationsOutOfRangeSubsetOfFile() throws Exception {
+    BlockLocation[] locations = getBlockLocationsOutput(205, 10, 300, 10);
+    assertEquals(0, locations.length);
+  }
+
+  @Test
+  public void testBlockLocationsEmptySubsetOfFile() throws Exception {
+    BlockLocation[] locations = getBlockLocationsOutput(205, 10, 0, 0);
+    assertEquals(0, locations.length);
+  }
+
+  @Test
+  public void testBlockLocationsDifferentLocationHost() throws Exception {
+    BlockLocation[] locations = getBlockLocationsOutput(100, 10, 0, 100,
+        "myblobhost");
+    assertEquals(10, locations.length);
+    assertEquals("myblobhost", locations[0].getHosts()[0]);
+  }
+
+  private static BlockLocation[] getBlockLocationsOutput(int fileSize,
+      int blockSize, long start, long len) throws Exception {
+    return getBlockLocationsOutput(fileSize, blockSize, start, len, null);
+  }
+
+  private static BlockLocation[] getBlockLocationsOutput(int fileSize,
+      int blockSize, long start, long len, String blockLocationHost)
+      throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(NativeAzureFileSystem.AZURE_BLOCK_SIZE_PROPERTY_NAME, ""
+        + blockSize);
+    if (blockLocationHost != null) {
+      conf.set(NativeAzureFileSystem.AZURE_BLOCK_LOCATION_HOST_PROPERTY_NAME,
+          blockLocationHost);
+    }
+    AzureBlobStorageTestAccount testAccount = AzureBlobStorageTestAccount
+        .createMock(conf);
+    FileSystem fs = testAccount.getFileSystem();
+    Path testFile = createTestFile(fs, fileSize);
+    FileStatus stat = fs.getFileStatus(testFile);
+    BlockLocation[] locations = fs.getFileBlockLocations(stat, start, len);
+    testAccount.cleanup();
+    return locations;
+  }
+
+  private static Path createTestFile(FileSystem fs, int size) throws Exception {
+    Path testFile = new Path("/testFile");
+    OutputStream outputStream = fs.create(testFile);
+    outputStream.write(new byte[size]);
+    outputStream.close();
+    return testFile;
+  }
+}
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
new file mode 100644
index 0000000..5583fec
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemConcurrency.java
@@ -0,0 +1,186 @@
+/**
+ * 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.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;
+import java.util.HashMap;
+import java.util.Iterator;
+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;
+  private InMemoryBlockBlobStore backingStore;
+
+  @Before
+  public void setUp() throws Exception {
+    testAccount = AzureBlobStorageTestAccount.createMock();
+    fs = testAccount.getFileSystem();
+    backingStore = testAccount.getMockStorage().getBackingStore();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    testAccount.cleanup();
+    fs = null;
+    backingStore = null;
+  }
+
+  @Test
+  public void testLinkBlobs() throws Exception {
+    Path filePath = new Path("/inProgress");
+    FSDataOutputStream outputStream = fs.create(filePath);
+    // Since the stream is still open, we should see an empty link
+    // blob in the backing store linking to the temporary file.
+    HashMap<String, String> metadata = backingStore
+        .getMetadata(AzureBlobStorageTestAccount.toMockUri(filePath));
+    assertNotNull(metadata);
+    String linkValue = metadata
+        .get(AzureNativeFileSystemStore.LINK_BACK_TO_UPLOAD_IN_PROGRESS_METADATA_KEY);
+    assertNotNull(linkValue);
+    assertTrue(backingStore.exists(AzureBlobStorageTestAccount
+        .toMockUri(linkValue)));
+    // Also, WASB should say the file exists now even before we close the
+    // stream.
+    assertTrue(fs.exists(filePath));
+    outputStream.close();
+    // Now there should be no link metadata on the final file.
+    metadata = backingStore.getMetadata(AzureBlobStorageTestAccount
+        .toMockUri(filePath));
+    assertNull(metadata
+        .get(AzureNativeFileSystemStore.LINK_BACK_TO_UPLOAD_IN_PROGRESS_METADATA_KEY));
+  }
+
+  private static String toString(FileStatus[] list) {
+    String[] asStrings = new String[list.length];
+    for (int i = 0; i < list.length; i++) {
+      asStrings[i] = list[i].getPath().toString();
+    }
+    return StringUtils.join(",", asStrings);
+  }
+
+  /**
+   * Test to make sure that we don't expose the temporary upload folder when
+   * listing at the root.
+   */
+  @Test
+  public void testNoTempBlobsVisible() throws Exception {
+    Path filePath = new Path("/inProgress");
+    FSDataOutputStream outputStream = fs.create(filePath);
+    // Make sure I can't see the temporary blob if I ask for a listing
+    FileStatus[] listOfRoot = fs.listStatus(new Path("/"));
+    assertEquals("Expected one file listed, instead got: "
+        + toString(listOfRoot), 1, listOfRoot.length);
+    assertEquals(fs.makeQualified(filePath), listOfRoot[0].getPath());
+    outputStream.close();
+  }
+
+  /**
+   * Converts a collection of exceptions to a collection of strings by getting
+   * the stack trace on every exception.
+   */
+  private static Iterable<String> selectToString(
+      final Iterable<Throwable> collection) {
+    return new Iterable<String>() {
+      @Override
+      public Iterator<String> iterator() {
+        final Iterator<Throwable> exceptionIterator = collection.iterator();
+        return new Iterator<String>() {
+          @Override
+          public boolean hasNext() {
+            return exceptionIterator.hasNext();
+          }
+
+          @Override
+          public String next() {
+            StringWriter stringWriter = new StringWriter();
+            PrintWriter printWriter = new PrintWriter(stringWriter);
+            exceptionIterator.next().printStackTrace(printWriter);
+            printWriter.close();
+            return stringWriter.toString();
+          }
+
+          @Override
+          public void remove() {
+            exceptionIterator.remove();
+          }
+        };
+      }
+    };
+  }
+
+  /**
+   * Tests running starting multiple threads all doing various File system
+   * operations against the same FS.
+   */
+  @Test
+  public void testMultiThreadedOperation() throws Exception {
+    for (int iter = 0; iter < 10; iter++) {
+      final int numThreads = 20;
+      Thread[] threads = new Thread[numThreads];
+      final ConcurrentLinkedQueue<Throwable> exceptionsEncountered = new ConcurrentLinkedQueue<Throwable>();
+      for (int i = 0; i < numThreads; i++) {
+        final Path threadLocalFile = new Path("/myFile" + i);
+        threads[i] = new Thread(new Runnable() {
+          @Override
+          public void run() {
+            try {
+              assertTrue(!fs.exists(threadLocalFile));
+              OutputStream output = fs.create(threadLocalFile);
+              output.write(5);
+              output.close();
+              assertTrue(fs.exists(threadLocalFile));
+              assertTrue(fs.listStatus(new Path("/")).length > 0);
+            } catch (Throwable ex) {
+              exceptionsEncountered.add(ex);
+            }
+          }
+        });
+      }
+      for (Thread t : threads) {
+        t.start();
+      }
+      for (Thread t : threads) {
+        t.join();
+      }
+      assertTrue(
+          "Encountered exceptions: "
+              + StringUtils.join("\r\n", selectToString(exceptionsEncountered)),
+          exceptionsEncountered.isEmpty());
+      tearDown();
+      setUp();
+    }
+  }
+}
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
new file mode 100644
index 0000000..b4a71f6
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractEmulator.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure;
+
+import org.apache.hadoop.fs.FileSystemContractBaseTest;
+
+public class TestNativeAzureFileSystemContractEmulator extends
+    FileSystemContractBaseTest {
+  private AzureBlobStorageTestAccount testAccount;
+
+  @Override
+  protected void setUp() throws Exception {
+    testAccount = AzureBlobStorageTestAccount.createForEmulator();
+    if (testAccount != null) {
+      fs = testAccount.getFileSystem();
+    }
+  }
+
+  @Override
+  protected void tearDown() throws Exception {
+    if (testAccount != null) {
+      testAccount.cleanup();
+      testAccount = null;
+      fs = null;
+    }
+  }
+
+  @Override
+  protected void runTest() throws Throwable {
+    if (testAccount != null) {
+      super.runTest();
+    }
+  }
+}
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
new file mode 100644
index 0000000..03292f3
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractLive.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure;
+
+import org.apache.hadoop.fs.FileSystemContractBaseTest;
+
+public class TestNativeAzureFileSystemContractLive extends
+    FileSystemContractBaseTest {
+  private AzureBlobStorageTestAccount testAccount;
+
+  @Override
+  protected void setUp() throws Exception {
+    testAccount = AzureBlobStorageTestAccount.create();
+    if (testAccount != null) {
+      fs = testAccount.getFileSystem();
+    }
+  }
+
+  @Override
+  protected void tearDown() throws Exception {
+    if (testAccount != null) {
+      testAccount.cleanup();
+      testAccount = null;
+      fs = null;
+    }
+  }
+
+  @Override
+  protected void runTest() throws Throwable {
+    if (testAccount != null) {
+      super.runTest();
+    }
+  }
+}
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
new file mode 100644
index 0000000..f25055b
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemContractMocked.java
@@ -0,0 +1,31 @@
+/**
+ * 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.FileSystemContractBaseTest;
+
+public class TestNativeAzureFileSystemContractMocked extends
+    FileSystemContractBaseTest {
+
+  @Override
+  protected void setUp() throws Exception {
+    fs = AzureBlobStorageTestAccount.createMock().getFileSystem();
+  }
+
+}
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
new file mode 100644
index 0000000..d7ff0c7
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemFileNameCheck.java
@@ -0,0 +1,140 @@
+/**
+ * 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.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;
+
+/**
+ * Tests the scenario where a colon is included in the file/directory name.
+ * 
+ * NativeAzureFileSystem#create(), #mkdir(), and #rename() disallow the
+ * creation/rename of files/directories through WASB that have colons in the
+ * names.
+ */
+public class TestNativeAzureFileSystemFileNameCheck {
+  private FileSystem fs = null;
+  private AzureBlobStorageTestAccount testAccount = null;
+  private String root = null;
+
+  @Before
+  public void setUp() throws Exception {
+    testAccount = AzureBlobStorageTestAccount.createMock();
+    fs = testAccount.getFileSystem();
+    root = fs.getUri().toString();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    testAccount.cleanup();
+    root = null;
+    fs = null;
+    testAccount = null;
+  }
+
+  @Test
+  public void testCreate() throws Exception {
+    // positive test
+    Path testFile1 = new Path(root + "/testFile1");
+    assertTrue(fs.createNewFile(testFile1));
+
+    // negative test
+    Path testFile2 = new Path(root + "/testFile2:2");
+    try {
+      fs.createNewFile(testFile2);
+      fail("Should've thrown.");
+    } catch (IOException e) { // ignore
+    }
+  }
+
+  @Test
+  public void testRename() throws Exception {
+    // positive test
+    Path testFile1 = new Path(root + "/testFile1");
+    assertTrue(fs.createNewFile(testFile1));
+    Path testFile2 = new Path(root + "/testFile2");
+    fs.rename(testFile1, testFile2);
+    assertTrue(!fs.exists(testFile1) && fs.exists(testFile2));
+
+    // negative test
+    Path testFile3 = new Path(root + "/testFile3:3");
+    try {
+      fs.rename(testFile2, testFile3);
+      fail("Should've thrown.");
+    } catch (IOException e) { // ignore
+    }
+    assertTrue(fs.exists(testFile2));
+  }
+
+  @Test
+  public void testMkdirs() throws Exception {
+    // positive test
+    Path testFolder1 = new Path(root + "/testFolder1");
+    assertTrue(fs.mkdirs(testFolder1));
+
+    // negative test
+    Path testFolder2 = new Path(root + "/testFolder2:2");
+    try {
+      assertTrue(fs.mkdirs(testFolder2));
+      fail("Should've thrown.");
+    } catch (IOException e) { // ignore
+    }
+  }
+
+  @Test
+  public void testWasbFsck() throws Exception {
+    // positive test
+    Path testFolder1 = new Path(root + "/testFolder1");
+    assertTrue(fs.mkdirs(testFolder1));
+    Path testFolder2 = new Path(testFolder1, "testFolder2");
+    assertTrue(fs.mkdirs(testFolder2));
+    Path testFolder3 = new Path(testFolder1, "testFolder3");
+    assertTrue(fs.mkdirs(testFolder3));
+    Path testFile1 = new Path(testFolder2, "testFile1");
+    assertTrue(fs.createNewFile(testFile1));
+    Path testFile2 = new Path(testFolder1, "testFile2");
+    assertTrue(fs.createNewFile(testFile2));
+    assertFalse(runWasbFsck(testFolder1));
+
+    // negative test
+    InMemoryBlockBlobStore backingStore = testAccount.getMockStorage()
+        .getBackingStore();
+    backingStore.setContent(AzureBlobStorageTestAccount
+        .toMockUri("testFolder1/testFolder2/test2:2"), new byte[] { 1, 2 },
+        new HashMap<String, String>());
+    assertTrue(runWasbFsck(testFolder1));
+  }
+
+  private boolean runWasbFsck(Path p) throws Exception {
+    WasbFsck fsck = new WasbFsck(fs.getConf());
+    fsck.setMockFileSystemForTesting(fs);
+    fsck.run(new String[] { p.toString() });
+    return fsck.getPathNameWarning();
+  }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..c82cee3
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemLive.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure;
+
+/*
+ * 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();
+  }
+}
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
new file mode 100644
index 0000000..9819c18
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemMocked.java
@@ -0,0 +1,28 @@
+/**
+ * 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;
+
+public class TestNativeAzureFileSystemMocked extends
+    NativeAzureFileSystemBaseTest {
+
+  @Override
+  protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
+    return AzureBlobStorageTestAccount.createMock();
+  }
+}
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemOperationsMocked.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemOperationsMocked.java
new file mode 100644
index 0000000..a6e782a
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestNativeAzureFileSystemOperationsMocked.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.azure;
+
+import static org.junit.Assume.assumeTrue;
+
+import org.apache.hadoop.fs.FSMainOperationsBaseTest;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public class TestNativeAzureFileSystemOperationsMocked extends
+    FSMainOperationsBaseTest {
+
+  public TestNativeAzureFileSystemOperationsMocked() {
+    super("/tmp/TestNativeAzureFileSystemOperationsMocked");
+  }
+
+  @Override
+  protected FileSystem createFileSystem() throws Exception {
+    return AzureBlobStorageTestAccount.createMock().getFileSystem();
+  }
+
+  public void testListStatusThrowsExceptionForUnreadableDir() throws Exception {
+    System.out
+        .println("Skipping testListStatusThrowsExceptionForUnreadableDir since WASB"
+            + " doesn't honor directory permissions.");
+    assumeTrue(!Path.WINDOWS);
+  }
+}
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
new file mode 100644
index 0000000..b49f6ee
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java
@@ -0,0 +1,169 @@
+/**
+ * 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.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.HashMap;
+
+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.junit.After;
+import org.junit.Before;
+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 {
+  private AzureBlobStorageTestAccount testAccount;
+  private FileSystem fs;
+  private InMemoryBlockBlobStore backingStore;
+
+  @Before
+  public void setUp() throws Exception {
+    testAccount = AzureBlobStorageTestAccount.createMock();
+    fs = testAccount.getFileSystem();
+    backingStore = testAccount.getMockStorage().getBackingStore();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    testAccount.cleanup();
+    fs = null;
+    backingStore = null;
+  }
+
+  private void createEmptyBlobOutOfBand(String path) {
+    backingStore.setContent(AzureBlobStorageTestAccount.toMockUri(path),
+        new byte[] { 1, 2 }, new HashMap<String, String>());
+  }
+
+  @Test
+  public void testImplicitFolderListed() throws Exception {
+    createEmptyBlobOutOfBand("root/b");
+
+    // List the blob itself.
+    FileStatus[] obtained = fs.listStatus(new Path("/root/b"));
+    assertNotNull(obtained);
+    assertEquals(1, obtained.length);
+    assertFalse(obtained[0].isDirectory());
+    assertEquals("/root/b", obtained[0].getPath().toUri().getPath());
+
+    // List the directory
+    obtained = fs.listStatus(new Path("/root"));
+    assertNotNull(obtained);
+    assertEquals(1, obtained.length);
+    assertFalse(obtained[0].isDirectory());
+    assertEquals("/root/b", obtained[0].getPath().toUri().getPath());
+
+    // Get the directory's file status
+    FileStatus dirStatus = fs.getFileStatus(new Path("/root"));
+    assertNotNull(dirStatus);
+    assertTrue(dirStatus.isDirectory());
+    assertEquals("/root", dirStatus.getPath().toUri().getPath());
+  }
+
+  @Test
+  public void testImplicitFolderDeleted() throws Exception {
+    createEmptyBlobOutOfBand("root/b");
+    assertTrue(fs.exists(new Path("/root")));
+    assertTrue(fs.delete(new Path("/root"), true));
+    assertFalse(fs.exists(new Path("/root")));
+  }
+
+  @Test
+  public void testFileInImplicitFolderDeleted() throws Exception {
+    createEmptyBlobOutOfBand("root/b");
+    assertTrue(fs.exists(new Path("/root")));
+    assertTrue(fs.delete(new Path("/root/b"), true));
+    assertTrue(fs.exists(new Path("/root")));
+  }
+
+  @Test
+  public void testFileAndImplicitFolderSameName() throws Exception {
+    createEmptyBlobOutOfBand("root/b");
+    createEmptyBlobOutOfBand("root/b/c");
+    FileStatus[] listResult = fs.listStatus(new Path("/root/b"));
+    // File should win.
+    assertEquals(1, listResult.length);
+    assertFalse(listResult[0].isDirectory());
+    try {
+      // Trying to delete root/b/c would cause a dilemma for WASB, so
+      // it should throw.
+      fs.delete(new Path("/root/b/c"), true);
+      assertTrue("Should've thrown.", false);
+    } catch (AzureException e) {
+      assertEquals("File /root/b/c has a parent directory /root/b"
+          + " which is also a file. Can't resolve.", e.getMessage());
+    }
+  }
+
+  private static enum DeepCreateTestVariation {
+    File, Folder
+  };
+
+  /**
+   * Tests that when we create the file (or folder) x/y/z, we also create
+   * explicit folder blobs for x and x/y
+   */
+  @Test
+  public void testCreatingDeepFileCreatesExplicitFolder() throws Exception {
+    for (DeepCreateTestVariation variation : DeepCreateTestVariation.values()) {
+      switch (variation) {
+      case File:
+        assertTrue(fs.createNewFile(new Path("/x/y/z")));
+        break;
+      case Folder:
+        assertTrue(fs.mkdirs(new Path("/x/y/z")));
+        break;
+      }
+      assertTrue(backingStore
+          .exists(AzureBlobStorageTestAccount.toMockUri("x")));
+      assertTrue(backingStore.exists(AzureBlobStorageTestAccount
+          .toMockUri("x/y")));
+      fs.delete(new Path("/x"), true);
+    }
+  }
+
+  @Test
+  public void testSetPermissionOnImplicitFolder() throws Exception {
+    createEmptyBlobOutOfBand("root/b");
+    FsPermission newPermission = new FsPermission((short) 0600);
+    fs.setPermission(new Path("/root"), newPermission);
+    FileStatus newStatus = fs.getFileStatus(new Path("/root"));
+    assertNotNull(newStatus);
+    assertEquals(newPermission, newStatus.getPermission());
+  }
+
+  @Test
+  public void testSetOwnerOnImplicitFolder() throws Exception {
+    createEmptyBlobOutOfBand("root/b");
+    fs.setOwner(new Path("/root"), "newOwner", null);
+    FileStatus newStatus = fs.getFileStatus(new Path("/root"));
+    assertNotNull(newStatus);
+    assertEquals("newOwner", newStatus.getOwner());
+  }
+}
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
new file mode 100644
index 0000000..1855c3b
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperationsLive.java
@@ -0,0 +1,182 @@
+/**
+ * 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.windowsazure.storage.blob.BlobOutputStream;
+import com.microsoft.windowsazure.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);
+  }
+
+  // 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);
+  }
+}
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
new file mode 100644
index 0000000..2284d1f
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestShellDecryptionKeyProvider.java
@@ -0,0 +1,82 @@
+/**
+ * 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 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.apache.hadoop.util.Shell;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestShellDecryptionKeyProvider {
+  public static final Log LOG = LogFactory
+      .getLog(TestShellDecryptionKeyProvider.class);
+  private static File TEST_ROOT_DIR = new File(System.getProperty(
+      "test.build.data", "/tmp"), "TestShellDecryptionKeyProvider");
+
+  @Test
+  public void testScriptPathNotSpecified() throws Exception {
+    if (!Shell.WINDOWS) {
+      return;
+    }
+    ShellDecryptionKeyProvider provider = new ShellDecryptionKeyProvider();
+    Configuration conf = new Configuration();
+    String account = "testacct";
+    String key = "key";
+
+    conf.set(SimpleKeyProvider.KEY_ACCOUNT_KEY_PREFIX + account, key);
+    try {
+      provider.getStorageAccountKey(account, conf);
+      Assert
+          .fail("fs.azure.shellkeyprovider.script is not specified, we should throw");
+    } catch (KeyProviderException e) {
+      LOG.info("Received an expected exception: " + e.getMessage());
+    }
+  }
+
+  @Test
+  public void testValidScript() throws Exception {
+    if (!Shell.WINDOWS) {
+      return;
+    }
+    String expectedResult = "decretedKey";
+
+    // Create a simple script which echoes the given key plus the given
+    // expected result (so that we validate both script input and output)
+    File scriptFile = new File(TEST_ROOT_DIR, "testScript.cmd");
+    FileUtils.writeStringToFile(scriptFile, "@echo %1 " + expectedResult);
+
+    ShellDecryptionKeyProvider provider = new ShellDecryptionKeyProvider();
+    Configuration conf = new Configuration();
+    String account = "testacct";
+    String key = "key1";
+    conf.set(SimpleKeyProvider.KEY_ACCOUNT_KEY_PREFIX + account, key);
+    conf.set(ShellDecryptionKeyProvider.KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT,
+        "cmd /c " + scriptFile.getAbsolutePath());
+
+    String result = provider.getStorageAccountKey(account, conf);
+    assertEquals(key + " " + expectedResult, result);
+  }
+}
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
new file mode 100644
index 0000000..a6c3f39
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbFsck.java
@@ -0,0 +1,101 @@
+/**
+ * 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.assertNotNull;
+
+import org.apache.hadoop.conf.Configuration;
+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.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestWasbFsck {
+  private AzureBlobStorageTestAccount testAccount;
+  private FileSystem fs;
+  private InMemoryBlockBlobStore backingStore;
+
+  @Before
+  public void setUp() throws Exception {
+    testAccount = AzureBlobStorageTestAccount.createMock();
+    fs = testAccount.getFileSystem();
+    backingStore = testAccount.getMockStorage().getBackingStore();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    testAccount.cleanup();
+    fs = null;
+    backingStore = null;
+  }
+
+  /**
+   * Counts the number of temporary blobs in the backing store.
+   */
+  private int getNumTempBlobs() {
+    int count = 0;
+    for (String key : backingStore.getKeys()) {
+      if (key.contains(NativeAzureFileSystem.AZURE_TEMP_FOLDER)) {
+        count++;
+      }
+    }
+    return count;
+  }
+
+  private void runFsck(String command) throws Exception {
+    Configuration conf = fs.getConf();
+    // Set the dangling cutoff to zero, so every temp blob is considered
+    // dangling.
+    conf.setInt(NativeAzureFileSystem.AZURE_TEMP_EXPIRY_PROPERTY_NAME, 0);
+    WasbFsck fsck = new WasbFsck(conf);
+    fsck.setMockFileSystemForTesting(fs);
+    fsck.run(new String[] { AzureBlobStorageTestAccount.MOCK_WASB_URI, command });
+  }
+
+  /**
+   * Tests that we delete dangling files properly
+   */
+  @Test
+  public void testDelete() throws Exception {
+    Path danglingFile = new Path("/crashedInTheMiddle");
+
+    // Create a file and leave it dangling and try to delete it.
+    FSDataOutputStream stream = fs.create(danglingFile);
+    stream.write(new byte[] { 1, 2, 3 });
+    stream.flush();
+
+    // Now we should still only see a zero-byte file in this place
+    FileStatus fileStatus = fs.getFileStatus(danglingFile);
+    assertNotNull(fileStatus);
+    assertEquals(0, fileStatus.getLen());
+    assertEquals(1, getNumTempBlobs());
+
+    // Run WasbFsck -delete to delete the file.
+    runFsck("-delete");
+
+    // Now we should see no trace of the file.
+    assertEquals(0, getNumTempBlobs());
+    assertFalse(fs.exists(danglingFile));
+  }
+}
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
new file mode 100644
index 0000000..ea24c59
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestWasbUriAndConfiguration.java
@@ -0,0 +1,392 @@
+/**
+ * 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.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 org.apache.hadoop.conf.Configuration;
+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.Assert;
+import org.junit.Test;
+
+import com.microsoft.windowsazure.storage.blob.CloudBlobContainer;
+import com.microsoft.windowsazure.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 AzureBlobStorageTestAccount testAccount;
+
+  @After
+  public void tearDown() throws Exception {
+    if (testAccount != null) {
+      testAccount.cleanup();
+      testAccount = null;
+    }
+  }
+
+  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 {
+    // 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 {
+    // 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 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", defaultUri.toString());
+        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", "file:///");
+    try {
+      FileSystem.get(new URI("wasb:///random/path"), conf);
+      fail("Should've thrown.");
+    } catch (IllegalArgumentException e) {
+    }
+  }
+}
diff --git a/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml b/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml
new file mode 100644
index 0000000..fb2aa20
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/resources/azure-test.xml
@@ -0,0 +1,49 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<configuration xmlns:xi="http://www.w3.org/2001/XInclude">
+
+  <property> 
+    <name>fs.wasb.impl</name> 
+    <value>org.apache.hadoop.fs.azure.NativeAzureFileSystem</value> 
+  </property> 
+   
+  <property> 
+    <name>fs.wasbs.impl</name> 
+    <value>org.apache.hadoop.fs.azure.NativeAzureFileSystem</value> 
+  </property> 
+ 
+  
+  <!-- For tests against live azure, provide the following account information -->
+  <!--
+  <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>{ACCOUNTKEY}</value>
+  </property>
+  -->  
+  
+  <!-- For tests against azure-emulator -->
+  <!--
+  <property>
+    <name>fs.azure.test.emulator</name>
+    <value>true</value>
+  </property>
+  -->
+</configuration>
diff --git a/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
new file mode 100644
index 0000000..81b935b
--- /dev/null
+++ b/hadoop-tools/hadoop-azure/src/test/resources/log4j.properties
@@ -0,0 +1,23 @@
+#
+#   Licensed to the Apache Software Foundation (ASF) under one or more
+#   contributor license agreements.  See the NOTICE file distributed with
+#   this work for additional information regarding copyright ownership.
+#   The ASF licenses this file to You under the Apache License, Version 2.0
+#   (the "License"); you may not use this file except in compliance with
+#   the License.  You may obtain a copy of the License at
+#
+#       http://www.apache.org/licenses/LICENSE-2.0
+#
+#   Unless required by applicable law or agreed to in writing, software
+#   distributed under the License is distributed on an "AS IS" BASIS,
+#   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#   See the License for the specific language governing permissions and
+#   limitations under the License.
+#
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=INFO,stdout
+log4j.threshhold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
diff --git a/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridMixClasses.java b/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridMixClasses.java
index ef1265b..397494c 100644
--- a/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridMixClasses.java
+++ b/hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestGridMixClasses.java
@@ -376,7 +376,7 @@
   /*
    * test LoadSortComparator
    */
-  @Test (timeout=1000)
+  @Test (timeout=3000)
   public void testLoadJobLoadSortComparator() throws Exception {
     LoadJob.LoadSortComparator test = new LoadJob.LoadSortComparator();
 
@@ -409,7 +409,7 @@
   /*
    * test SpecGroupingComparator
    */
-  @Test (timeout=1000)
+  @Test (timeout=3000)
   public void testGridmixJobSpecGroupingComparator() throws Exception {
     GridmixJob.SpecGroupingComparator test = new GridmixJob.SpecGroupingComparator();
 
@@ -452,7 +452,7 @@
   /*
    * test CompareGridmixJob only equals and compare
    */
-  @Test (timeout=10000)
+  @Test (timeout=30000)
   public void testCompareGridmixJob() throws Exception {
     Configuration conf = new Configuration();
     Path outRoot = new Path("target");
@@ -478,7 +478,7 @@
   /*
    * test ReadRecordFactory. should read all data from inputstream
    */
-  @Test (timeout=1000)
+  @Test (timeout=3000)
   public void testReadRecordFactory() throws Exception {
 
     // RecordFactory factory, InputStream src, Configuration conf
@@ -589,7 +589,7 @@
   /*
    * test LoadRecordReader. It class reads data from some files.
    */
-  @Test (timeout=1000)
+  @Test (timeout=3000)
   public void testLoadJobLoadRecordReader() throws Exception {
     LoadJob.LoadRecordReader test = new LoadJob.LoadRecordReader();
     Configuration conf = new Configuration();
@@ -652,7 +652,7 @@
    * test LoadReducer
    */
 
-  @Test (timeout=1000)
+  @Test (timeout=3000)
   public void testLoadJobLoadReducer() throws Exception {
     LoadJob.LoadReducer test = new LoadJob.LoadReducer();
 
@@ -772,7 +772,7 @@
   /*
    * test SerialJobFactory
    */
-  @Test (timeout=40000)
+  @Test (timeout=120000)
   public void testSerialReaderThread() throws Exception {
 
     Configuration conf = new Configuration();
@@ -833,7 +833,7 @@
    * test SleepMapper
    */
   @SuppressWarnings({"unchecked", "rawtypes"})
-  @Test (timeout=10000)
+  @Test (timeout=30000)
   public void testSleepMapper() throws Exception {
     SleepJob.SleepMapper test = new SleepJob.SleepMapper();
 
@@ -878,7 +878,7 @@
   /*
    * test SleepReducer
    */
-  @Test (timeout=1000)
+  @Test (timeout=3000)
   public void testSleepReducer() throws Exception {
     Configuration conf = new Configuration();
     conf.setInt(JobContext.NUM_REDUCES, 2);
diff --git a/hadoop-tools/hadoop-tools-dist/pom.xml b/hadoop-tools/hadoop-tools-dist/pom.xml
index 46ab720..38043f7 100644
--- a/hadoop-tools/hadoop-tools-dist/pom.xml
+++ b/hadoop-tools/hadoop-tools-dist/pom.xml
@@ -85,6 +85,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-azure</artifactId>
+      <scope>compile</scope>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-sls</artifactId>
       <scope>compile</scope>
     </dependency>
diff --git a/hadoop-tools/pom.xml b/hadoop-tools/pom.xml
index 33aadae..bed0f9b 100644
--- a/hadoop-tools/pom.xml
+++ b/hadoop-tools/pom.xml
@@ -42,6 +42,7 @@
     <module>hadoop-pipes</module>
     <module>hadoop-openstack</module>
     <module>hadoop-sls</module>
+    <module>hadoop-azure</module>
   </modules>
 
   <build>
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 6fea8ea..c88560b 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -32,6 +32,10 @@
     YARN-1338. Recover localized resource cache state upon nodemanager restart 
     (Jason Lowe via junping_du)
 
+    YARN-1368. Added core functionality of recovering container state into
+    schedulers after ResourceManager Restart so as to preserve running work in
+    the cluster. (Jian He via vinodkv)
+
   IMPROVEMENTS
 
     YARN-1479. Invalid NaN values in Hadoop REST API JSON response (Chen He via
@@ -145,6 +149,15 @@
     YARN-2132. ZKRMStateStore.ZKAction#runWithRetries doesn't log the exception
     it encounters. (Vamsee Yarlagadda via kasha)
 
+    YARN-2030. Augmented RMStateStore with state machine.(Binglin Chang via jianhe)
+
+    YARN-1424. RMAppAttemptImpl should return the 
+    DummyApplicationResourceUsageReport for all invalid accesses. 
+    (Ray Chiang via kasha)
+
+    YARN-2091. Add more values to ContainerExitStatus and pass it from NM to
+    RM and then to app masters (Tsuyoshi OZAWA via bikas)
+
   OPTIMIZATIONS
 
   BUG FIXES 
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationResourceUsageReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationResourceUsageReport.java
index 8de9ff3..6e9c76f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationResourceUsageReport.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationResourceUsageReport.java
@@ -47,7 +47,7 @@
   }
 
   /**
-   * Get the number of used containers
+   * Get the number of used containers.  -1 for invalid/inaccessible reports.
    * @return the number of used containers
    */
   @Public
@@ -63,7 +63,7 @@
   public abstract void setNumUsedContainers(int num_containers);
 
   /**
-   * Get the number of reserved containers
+   * Get the number of reserved containers.  -1 for invalid/inaccessible reports.
    * @return the number of reserved containers
    */
   @Private
@@ -79,7 +79,7 @@
   public abstract void setNumReservedContainers(int num_reserved_containers);
 
   /**
-   * Get the used <code>Resource</code>
+   * Get the used <code>Resource</code>.  -1 for invalid/inaccessible reports.
    * @return the used <code>Resource</code>
    */
   @Public
@@ -91,7 +91,7 @@
   public abstract void setUsedResources(Resource resources);
 
   /**
-   * Get the reserved <code>Resource</code>
+   * Get the reserved <code>Resource</code>.  -1 for invalid/inaccessible reports.
    * @return the reserved <code>Resource</code>
    */
   @Public
@@ -103,7 +103,7 @@
   public abstract void setReservedResources(Resource reserved_resources);
 
   /**
-   * Get the needed <code>Resource</code>
+   * Get the needed <code>Resource</code>.  -1 for invalid/inaccessible reports.
    * @return the needed <code>Resource</code>
    */
   @Public
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerExitStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerExitStatus.java
index 7122578..f88fa3b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerExitStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerExitStatus.java
@@ -46,4 +46,30 @@
    * Containers preempted by the framework.
    */
   public static final int PREEMPTED = -102;
+
+  /**
+   * Container terminated because of exceeding allocated virtual memory.
+   */
+  public static final int KILLED_EXCEEDED_VMEM = -103;
+
+  /**
+   * Container terminated because of exceeding allocated physical memory.
+   */
+  public static final int KILLED_EXCEEDED_PMEM = -104;
+
+  /**
+   * Container was terminated by stop request by the app master.
+   */
+  public static final int KILLED_BY_APPMASTER = -105;
+
+  /**
+   * Container was terminated by the resource manager.
+   */
+  public static final int KILLED_BY_RESOURCEMANAGER = -106;
+
+  /**
+   * Container was terminated after the application finished.
+   */
+  public static final int KILLED_AFTER_APP_COMPLETION = -107;
+
 }
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 4a534cf..59e108a 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
@@ -318,6 +318,13 @@
   public static final String RECOVERY_ENABLED = RM_PREFIX + "recovery.enabled";
   public static final boolean DEFAULT_RM_RECOVERY_ENABLED = false;
 
+  @Private
+  public static final String RM_WORK_PRESERVING_RECOVERY_ENABLED = RM_PREFIX
+      + "work-preserving-recovery.enabled";
+  @Private
+  public static final boolean DEFAULT_RM_WORK_PRESERVING_RECOVERY_ENABLED =
+      false;
+
   /** Zookeeper interaction configs */
   public static final String RM_ZK_PREFIX = RM_PREFIX + "zk-";
 
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 c94b782..0c1628e 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
@@ -270,6 +270,14 @@
   </property>
 
   <property>
+    <description>Enable RM work preserving recovery. This configuration is private
+    to YARN for experimenting the feature.
+    </description>
+    <name>yarn.resourcemanager.work-preserving-recovery.enabled</name>
+    <value>false</value>
+  </property>
+
+  <property>
     <description>The class to use as the persistent store.
 
       If org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
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/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index 750c11a..ded2013 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -64,6 +64,7 @@
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -738,7 +739,8 @@
     } else {
       dispatcher.getEventHandler().handle(
         new ContainerKillEvent(containerID,
-          "Container killed by the ApplicationMaster."));
+            ContainerExitStatus.KILLED_BY_APPMASTER,
+            "Container killed by the ApplicationMaster."));
 
       NMAuditLogger.logSuccess(container.getUser(),    
         AuditConstants.STOP_CONTAINER, "ContainerManageImpl", containerID
@@ -887,6 +889,7 @@
           .getContainersToCleanup()) {
           this.dispatcher.getEventHandler().handle(
               new ContainerKillEvent(container,
+                  ContainerExitStatus.KILLED_BY_RESOURCEMANAGER,
                   "Container Killed by ResourceManager"));
       }
       break;
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/application/ApplicationImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
index 21d2f91..a206591 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
@@ -30,6 +30,7 @@
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.logaggregation.ContainerLogsRetentionPolicy;
@@ -375,6 +376,7 @@
       for (ContainerId containerID : app.containers.keySet()) {
         app.dispatcher.getEventHandler().handle(
             new ContainerKillEvent(containerID,
+                ContainerExitStatus.KILLED_AFTER_APP_COMPLETION,
                 "Container killed on application-finish event: " + appEvent.getDiagnostic()));
       }
       return ApplicationState.FINISHING_CONTAINERS_WAIT;
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 1b683a1..22e47e7 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
@@ -48,7 +48,6 @@
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
-import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
 import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger;
 import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEvent;
@@ -773,7 +772,7 @@
       container.cleanup();
       container.metrics.endInitingContainer();
       ContainerKillEvent killEvent = (ContainerKillEvent) event;
-      container.exitCode = ExitCode.TERMINATED.getExitCode();
+      container.exitCode = killEvent.getContainerExitStatus();
       container.diagnostics.append(killEvent.getDiagnostic()).append("\n");
       container.diagnostics.append("Container is killed before being launched.\n");
     }
@@ -817,6 +816,7 @@
               ContainersLauncherEventType.CLEANUP_CONTAINER));
       ContainerKillEvent killEvent = (ContainerKillEvent) event;
       container.diagnostics.append(killEvent.getDiagnostic()).append("\n");
+      container.exitCode = killEvent.getContainerExitStatus();
     }
   }
 
@@ -829,7 +829,10 @@
     @Override
     public void transition(ContainerImpl container, ContainerEvent event) {
       ContainerExitEvent exitEvent = (ContainerExitEvent) event;
-      container.exitCode = exitEvent.getExitCode();
+      if (container.hasDefaultExitCode()) {
+        container.exitCode = exitEvent.getExitCode();
+      }
+
       if (exitEvent.getDiagnosticInfo() != null) {
         container.diagnostics.append(exitEvent.getDiagnosticInfo())
           .append('\n');
@@ -871,7 +874,7 @@
     @Override
     public void transition(ContainerImpl container, ContainerEvent event) {
       ContainerKillEvent killEvent = (ContainerKillEvent) event;
-      container.exitCode = ExitCode.TERMINATED.getExitCode();
+      container.exitCode = killEvent.getContainerExitStatus();
       container.diagnostics.append(killEvent.getDiagnostic()).append("\n");
       container.diagnostics.append("Container is killed before being launched.\n");
       super.transition(container, event);
@@ -928,4 +931,9 @@
       this.readLock.unlock();
     }
   }
+
+  private boolean hasDefaultExitCode() {
+    return (this.exitCode == ContainerExitStatus.INVALID);
+  }
+
 }
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/ContainerKillEvent.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/ContainerKillEvent.java
index 313b6a8..9ff0f71 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/ContainerKillEvent.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/ContainerKillEvent.java
@@ -23,13 +23,21 @@
 public class ContainerKillEvent extends ContainerEvent {
 
   private final String diagnostic;
+  private final int exitStatus;
 
-  public ContainerKillEvent(ContainerId cID, String diagnostic) {
+  public ContainerKillEvent(ContainerId cID,
+      int exitStatus, String diagnostic) {
     super(cID, ContainerEventType.KILL_CONTAINER);
+    this.exitStatus = exitStatus;
     this.diagnostic = diagnostic;
   }
 
   public String getDiagnostic() {
     return this.diagnostic;
   }
+
+  public int getContainerExitStatus() {
+    return this.exitStatus;
+  }
+
 }
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/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
index b681b34..04887ed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
@@ -30,6 +30,7 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
@@ -403,6 +404,7 @@
 
             boolean isMemoryOverLimit = false;
             String msg = "";
+            int containerExitStatus = ContainerExitStatus.INVALID;
             if (isVmemCheckEnabled()
                 && isProcessTreeOverLimit(containerId.toString(),
                     currentVmemUsage, curMemUsageOfAgedProcesses, vmemLimit)) {
@@ -414,6 +416,7 @@
                   currentPmemUsage, pmemLimit,
                   pId, containerId, pTree);
               isMemoryOverLimit = true;
+              containerExitStatus = ContainerExitStatus.KILLED_EXCEEDED_VMEM;
             } else if (isPmemCheckEnabled()
                 && isProcessTreeOverLimit(containerId.toString(),
                     currentPmemUsage, curRssMemUsageOfAgedProcesses,
@@ -426,6 +429,7 @@
                   currentPmemUsage, pmemLimit,
                   pId, containerId, pTree);
               isMemoryOverLimit = true;
+              containerExitStatus = ContainerExitStatus.KILLED_EXCEEDED_PMEM;
             }
 
             if (isMemoryOverLimit) {
@@ -440,7 +444,8 @@
               }
               // kill the container
               eventDispatcher.getEventHandler().handle(
-                  new ContainerKillEvent(containerId, msg));
+                  new ContainerKillEvent(containerId,
+                      containerExitStatus, msg));
               it.remove();
               LOG.info("Removed ProcessTree with root " + pId);
             } else {
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/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index 53a3c4b..28ac14c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -31,6 +31,7 @@
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.junit.Assert;
 
 import org.apache.commons.logging.LogFactory;
@@ -68,7 +69,6 @@
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.ResourceManagerConstants;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
-import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
 import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestAuxServices.ServiceA;
@@ -348,8 +348,7 @@
         GetContainerStatusesRequest.newInstance(containerIds);
     ContainerStatus containerStatus = 
         containerManager.getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
-    int expectedExitCode = Shell.WINDOWS ? ExitCode.FORCE_KILLED.getExitCode() :
-      ExitCode.TERMINATED.getExitCode();
+    int expectedExitCode = ContainerExitStatus.KILLED_BY_APPMASTER;
     Assert.assertEquals(expectedExitCode, containerStatus.getExitStatus());
 
     // Assert that the process is not alive anymore
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 8af9518..6ad6010 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
@@ -17,6 +17,7 @@
 */
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.container;
 
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
@@ -319,7 +320,7 @@
       assertEquals(ContainerState.NEW, wc.c.getContainerState());
       wc.killContainer();
       assertEquals(ContainerState.DONE, wc.c.getContainerState());
-      assertEquals(ExitCode.TERMINATED.getExitCode(),
+      assertEquals(ContainerExitStatus.KILLED_BY_RESOURCEMANAGER,
           wc.c.cloneAndGetContainerStatus().getExitStatus());
       assertTrue(wc.c.cloneAndGetContainerStatus().getDiagnostics()
           .contains("KillRequest"));
@@ -339,7 +340,7 @@
       assertEquals(ContainerState.LOCALIZING, wc.c.getContainerState());
       wc.killContainer();
       assertEquals(ContainerState.KILLING, wc.c.getContainerState());
-      assertEquals(ExitCode.TERMINATED.getExitCode(),
+      assertEquals(ContainerExitStatus.KILLED_BY_RESOURCEMANAGER,
           wc.c.cloneAndGetContainerStatus().getExitStatus());
       assertTrue(wc.c.cloneAndGetContainerStatus().getDiagnostics()
           .contains("KillRequest"));
@@ -898,12 +899,14 @@
     }
 
     public void killContainer() {
-      c.handle(new ContainerKillEvent(cId, "KillRequest"));
+      c.handle(new ContainerKillEvent(cId,
+          ContainerExitStatus.KILLED_BY_RESOURCEMANAGER,
+          "KillRequest"));
       drainDispatcherEvents();
     }
 
     public void containerKilledOnRequest() {
-      int exitCode = ExitCode.FORCE_KILLED.getExitCode();
+      int exitCode = ContainerExitStatus.KILLED_BY_RESOURCEMANAGER;
       String diagnosticMsg = "Container completed with exit code " + exitCode;
       c.handle(new ContainerExitEvent(cId,
           ContainerEventType.CONTAINER_KILLED_ON_REQUEST, exitCode,
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/launcher/TestContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
index c8fc85a..d02f05f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher;
 
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
@@ -73,7 +74,6 @@
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
-import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
 import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
@@ -604,8 +604,7 @@
         GetContainerStatusesRequest.newInstance(containerIds);
     ContainerStatus containerStatus = 
         containerManager.getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
-    int expectedExitCode = Shell.WINDOWS ? ExitCode.FORCE_KILLED.getExitCode() :
-      ExitCode.TERMINATED.getExitCode();
+    int expectedExitCode = ContainerExitStatus.KILLED_BY_APPMASTER;
     Assert.assertEquals(expectedExitCode, containerStatus.getExitStatus());
 
     // Assert that the process is not alive anymore
@@ -717,7 +716,7 @@
     ContainerStatus containerStatus = 
         containerManager.getContainerStatuses(gcsRequest)
           .getContainerStatuses().get(0);
-    Assert.assertEquals(ExitCode.FORCE_KILLED.getExitCode(),
+    Assert.assertEquals(ContainerExitStatus.KILLED_BY_APPMASTER,
         containerStatus.getExitStatus());
 
     // Now verify the contents of the file.  Script generates a message when it
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/monitor/TestContainersMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java
index 1102ebb..b1fb3fc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor;
 
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -60,7 +61,6 @@
 import org.apache.hadoop.yarn.exceptions.YarnException;
 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.ContainerExecutor.Signal;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
@@ -270,7 +270,7 @@
         GetContainerStatusesRequest.newInstance(containerIds);
     ContainerStatus containerStatus =
         containerManager.getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
-    Assert.assertEquals(ExitCode.TERMINATED.getExitCode(),
+    Assert.assertEquals(ContainerExitStatus.KILLED_EXCEEDED_VMEM,
         containerStatus.getExitStatus());
     String expectedMsgPattern =
         "Container \\[pid=" + pid + ",containerID=" + cId
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
index 79fb5df..517e680 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
@@ -99,4 +99,6 @@
       RMApplicationHistoryWriter rmApplicationHistoryWriter);
 
   ConfigurationProvider getConfigurationProvider();
+
+  boolean isWorkPreservingRecoveryEnabled();
 }
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index 1eb4b75..1abc660 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -60,6 +60,7 @@
     = new ConcurrentHashMap<String, RMNode>();
 
   private boolean isHAEnabled;
+  private boolean isWorkPreservingRecoveryEnabled;
   private HAServiceState haServiceState =
       HAServiceProtocol.HAServiceState.INITIALIZING;
   
@@ -329,6 +330,15 @@
     }
   }
 
+  public void setWorkPreservingRecoveryEnabled(boolean enabled) {
+    this.isWorkPreservingRecoveryEnabled = enabled;
+  }
+
+  @Override
+  public boolean isWorkPreservingRecoveryEnabled() {
+    return this.isWorkPreservingRecoveryEnabled;
+  }
+
   @Override
   public RMApplicationHistoryWriter getRMApplicationHistoryWriter() {
     return rmApplicationHistoryWriter;
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 e884d29..d93c45d 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
@@ -28,6 +28,7 @@
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -43,6 +44,8 @@
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.util.resource.Resources;
 
 /**
  * Utility methods to aid serving RM data through the REST and RPC APIs
@@ -225,4 +228,13 @@
     }
   }
 
+  /**
+   * Statically defined dummy ApplicationResourceUsageREport.  Used as
+   * a return value when a valid report cannot be found.
+   */
+  public static final ApplicationResourceUsageReport
+    DUMMY_APPLICATION_RESOURCE_USAGE_REPORT =
+      BuilderUtils.newApplicationResourceUsageReport(-1, -1,
+          Resources.createResource(-1, -1), Resources.createResource(-1, -1),
+          Resources.createResource(-1, -1));
 }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index b62bd5f..724dee1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -364,9 +364,15 @@
           YarnConfiguration.DEFAULT_RM_RECOVERY_ENABLED);
 
       RMStateStore rmStore = null;
-      if(isRecoveryEnabled) {
+      if (isRecoveryEnabled) {
         recoveryEnabled = true;
-        rmStore =  RMStateStoreFactory.getStore(conf);
+        rmStore = RMStateStoreFactory.getStore(conf);
+        boolean isWorkPreservingRecoveryEnabled =
+            conf.getBoolean(
+              YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED,
+              YarnConfiguration.DEFAULT_RM_WORK_PRESERVING_RECOVERY_ENABLED);
+        rmContext
+          .setWorkPreservingRecoveryEnabled(isWorkPreservingRecoveryEnabled);
       } else {
         recoveryEnabled = false;
         rmStore = new NullRMStateStore();
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
index a59d1d5..e00eaef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
@@ -60,6 +60,7 @@
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeReconnectEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStartedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -243,11 +244,13 @@
     Resource capability = request.getResource();
     String nodeManagerVersion = request.getNMVersion();
 
-    if (!request.getNMContainerStatuses().isEmpty()) {
-      LOG.info("received container statuses on node manager register :"
-          + request.getNMContainerStatuses());
-      for (NMContainerStatus report : request.getNMContainerStatuses()) {
-        handleNMContainerStatus(report);
+    if (!rmContext.isWorkPreservingRecoveryEnabled()) {
+      if (!request.getNMContainerStatuses().isEmpty()) {
+        LOG.info("received container statuses on node manager register :"
+            + request.getNMContainerStatuses());
+        for (NMContainerStatus status : request.getNMContainerStatuses()) {
+          handleNMContainerStatus(status);
+        }
       }
     }
     RegisterNodeManagerResponse response = recordFactory
@@ -308,7 +311,7 @@
     RMNode oldNode = this.rmContext.getRMNodes().putIfAbsent(nodeId, rmNode);
     if (oldNode == null) {
       this.rmContext.getDispatcher().getEventHandler().handle(
-          new RMNodeEvent(nodeId, RMNodeEventType.STARTED));
+          new RMNodeStartedEvent(nodeId, request.getNMContainerStatuses()));
     } else {
       LOG.info("Reconnect from the node at: " + host);
       this.nmLivelinessMonitor.unregister(nodeId);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
index 1f6e175..7f4dad8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
@@ -47,6 +47,8 @@
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationStateDataProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RMStateVersionProto;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationAttemptStateData;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.RMStateVersion;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
@@ -314,7 +316,7 @@
 
   @Override
   public synchronized void storeApplicationStateInternal(ApplicationId appId,
-      ApplicationStateDataPBImpl appStateDataPB) throws Exception {
+      ApplicationStateData appStateDataPB) throws Exception {
     String appIdStr = appId.toString();
     Path appDirPath = getAppDir(rmAppRoot, appIdStr);
     fs.mkdirs(appDirPath);
@@ -334,7 +336,7 @@
 
   @Override
   public synchronized void updateApplicationStateInternal(ApplicationId appId,
-      ApplicationStateDataPBImpl appStateDataPB) throws Exception {
+      ApplicationStateData appStateDataPB) throws Exception {
     String appIdStr = appId.toString();
     Path appDirPath = getAppDir(rmAppRoot, appIdStr);
     Path nodeCreatePath = getNodePath(appDirPath, appIdStr);
@@ -354,7 +356,7 @@
   @Override
   public synchronized void storeApplicationAttemptStateInternal(
       ApplicationAttemptId appAttemptId,
-      ApplicationAttemptStateDataPBImpl attemptStateDataPB)
+      ApplicationAttemptStateData attemptStateDataPB)
       throws Exception {
     Path appDirPath =
         getAppDir(rmAppRoot, appAttemptId.getApplicationId().toString());
@@ -375,7 +377,7 @@
   @Override
   public synchronized void updateApplicationAttemptStateInternal(
       ApplicationAttemptId appAttemptId,
-      ApplicationAttemptStateDataPBImpl attemptStateDataPB)
+      ApplicationAttemptStateData attemptStateDataPB)
       throws Exception {
     Path appDirPath =
         getAppDir(rmAppRoot, appAttemptId.getApplicationId().toString());
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java
index c9f3541..a43b20d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java
@@ -32,9 +32,9 @@
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationAttemptStateData;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.RMStateVersion;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -80,7 +80,7 @@
 
   @Override
   public void storeApplicationStateInternal(ApplicationId appId,
-                                     ApplicationStateDataPBImpl appStateData)
+                                     ApplicationStateData appStateData)
       throws Exception {
     ApplicationState appState =
         new ApplicationState(appStateData.getSubmitTime(),
@@ -92,7 +92,7 @@
 
   @Override
   public void updateApplicationStateInternal(ApplicationId appId,
-      ApplicationStateDataPBImpl appStateData) throws Exception {
+      ApplicationStateData appStateData) throws Exception {
     ApplicationState updatedAppState =
         new ApplicationState(appStateData.getSubmitTime(),
           appStateData.getStartTime(),
@@ -112,7 +112,7 @@
   @Override
   public synchronized void storeApplicationAttemptStateInternal(
       ApplicationAttemptId appAttemptId,
-      ApplicationAttemptStateDataPBImpl attemptStateData)
+      ApplicationAttemptStateData attemptStateData)
       throws Exception {
     Credentials credentials = null;
     if(attemptStateData.getAppAttemptTokens() != null){
@@ -137,7 +137,7 @@
   @Override
   public synchronized void updateApplicationAttemptStateInternal(
       ApplicationAttemptId appAttemptId,
-      ApplicationAttemptStateDataPBImpl attemptStateData)
+      ApplicationAttemptStateData attemptStateData)
       throws Exception {
     Credentials credentials = null;
     if (attemptStateData.getAppAttemptTokens() != null) {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/NullRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/NullRMStateStore.java
index a12099f..6a0426c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/NullRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/NullRMStateStore.java
@@ -25,9 +25,9 @@
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationAttemptStateData;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.RMStateVersion;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
 
 @Unstable
 public class NullRMStateStore extends RMStateStore {
@@ -54,13 +54,13 @@
 
   @Override
   protected void storeApplicationStateInternal(ApplicationId appId,
-      ApplicationStateDataPBImpl appStateData) throws Exception {
+      ApplicationStateData appStateData) throws Exception {
     // Do nothing
   }
 
   @Override
   protected void storeApplicationAttemptStateInternal(ApplicationAttemptId attemptId,
-      ApplicationAttemptStateDataPBImpl attemptStateData) throws Exception {
+      ApplicationAttemptStateData attemptStateData) throws Exception {
     // Do nothing
   }
 
@@ -102,13 +102,13 @@
 
   @Override
   protected void updateApplicationStateInternal(ApplicationId appId,
-      ApplicationStateDataPBImpl appStateData) throws Exception {
+      ApplicationStateData appStateData) throws Exception {
     // Do nothing 
   }
 
   @Override
   protected void updateApplicationAttemptStateInternal(ApplicationAttemptId attemptId,
-      ApplicationAttemptStateDataPBImpl attemptStateData) throws Exception {
+      ApplicationAttemptStateData attemptStateData) throws Exception {
   }
 
   @Override
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
index fc4537c..affc6f9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.recovery;
 
-import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -31,7 +30,6 @@
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
@@ -50,6 +48,8 @@
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.RMFatalEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.RMFatalEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationAttemptStateData;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.RMStateVersion;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
@@ -61,6 +61,10 @@
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptNewSavedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUpdateSavedEvent;
+import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.SingleArcTransition;
+import org.apache.hadoop.yarn.state.StateMachine;
+import org.apache.hadoop.yarn.state.StateMachineFactory;
 
 @Private
 @Unstable
@@ -83,8 +87,163 @@
 
   public static final Log LOG = LogFactory.getLog(RMStateStore.class);
 
+  private enum RMStateStoreState {
+    DEFAULT
+  };
+
+  private static final StateMachineFactory<RMStateStore,
+                                           RMStateStoreState,
+                                           RMStateStoreEventType, 
+                                           RMStateStoreEvent>
+      stateMachineFactory = new StateMachineFactory<RMStateStore,
+                                                    RMStateStoreState,
+                                                    RMStateStoreEventType,
+                                                    RMStateStoreEvent>(
+      RMStateStoreState.DEFAULT)
+      .addTransition(RMStateStoreState.DEFAULT, RMStateStoreState.DEFAULT,
+          RMStateStoreEventType.STORE_APP, new StoreAppTransition())
+      .addTransition(RMStateStoreState.DEFAULT, RMStateStoreState.DEFAULT,
+          RMStateStoreEventType.UPDATE_APP, new UpdateAppTransition())
+      .addTransition(RMStateStoreState.DEFAULT, RMStateStoreState.DEFAULT,
+          RMStateStoreEventType.REMOVE_APP, new RemoveAppTransition())
+      .addTransition(RMStateStoreState.DEFAULT, RMStateStoreState.DEFAULT,
+          RMStateStoreEventType.STORE_APP_ATTEMPT, new StoreAppAttemptTransition())
+      .addTransition(RMStateStoreState.DEFAULT, RMStateStoreState.DEFAULT,
+          RMStateStoreEventType.UPDATE_APP_ATTEMPT, new UpdateAppAttemptTransition());
+
+  private final StateMachine<RMStateStoreState,
+                             RMStateStoreEventType,
+                             RMStateStoreEvent> stateMachine;
+
+  private static class StoreAppTransition
+      implements SingleArcTransition<RMStateStore, RMStateStoreEvent> {
+    @Override
+    public void transition(RMStateStore store, RMStateStoreEvent event) {
+      if (!(event instanceof RMStateStoreAppEvent)) {
+        // should never happen
+        LOG.error("Illegal event type: " + event.getClass());
+        return;
+      }
+      ApplicationState appState = ((RMStateStoreAppEvent) event).getAppState();
+      ApplicationId appId = appState.getAppId();
+      ApplicationStateData appStateData = ApplicationStateData
+          .newInstance(appState);
+      LOG.info("Storing info for app: " + appId);
+      try {
+        store.storeApplicationStateInternal(appId, appStateData);
+        store.notifyDoneStoringApplication(appId, null);
+      } catch (Exception e) {
+        LOG.error("Error storing app: " + appId, e);
+        store.notifyStoreOperationFailed(e);
+      }
+    };
+  }
+
+  private static class UpdateAppTransition implements
+      SingleArcTransition<RMStateStore, RMStateStoreEvent> {
+    @Override
+    public void transition(RMStateStore store, RMStateStoreEvent event) {
+      if (!(event instanceof RMStateUpdateAppEvent)) {
+        // should never happen
+        LOG.error("Illegal event type: " + event.getClass());
+        return;
+      }
+      ApplicationState appState = ((RMStateUpdateAppEvent) event).getAppState();
+      ApplicationId appId = appState.getAppId();
+      ApplicationStateData appStateData = ApplicationStateData
+          .newInstance(appState);
+      LOG.info("Updating info for app: " + appId);
+      try {
+        store.updateApplicationStateInternal(appId, appStateData);
+        store.notifyDoneUpdatingApplication(appId, null);
+      } catch (Exception e) {
+        LOG.error("Error updating app: " + appId, e);
+        store.notifyStoreOperationFailed(e);
+      }
+    };
+  }
+
+  private static class RemoveAppTransition implements
+      SingleArcTransition<RMStateStore, RMStateStoreEvent> {
+    @Override
+    public void transition(RMStateStore store, RMStateStoreEvent event) {
+      if (!(event instanceof RMStateStoreRemoveAppEvent)) {
+        // should never happen
+        LOG.error("Illegal event type: " + event.getClass());
+        return;
+      }
+      ApplicationState appState = ((RMStateStoreRemoveAppEvent) event)
+          .getAppState();
+      ApplicationId appId = appState.getAppId();
+      LOG.info("Removing info for app: " + appId);
+      try {
+        store.removeApplicationStateInternal(appState);
+      } catch (Exception e) {
+        LOG.error("Error removing app: " + appId, e);
+        store.notifyStoreOperationFailed(e);
+      }
+    };
+  }
+
+  private static class StoreAppAttemptTransition implements
+      SingleArcTransition<RMStateStore, RMStateStoreEvent> {
+    @Override
+    public void transition(RMStateStore store, RMStateStoreEvent event) {
+      if (!(event instanceof RMStateStoreAppAttemptEvent)) {
+        // should never happen
+        LOG.error("Illegal event type: " + event.getClass());
+        return;
+      }
+      ApplicationAttemptState attemptState =
+          ((RMStateStoreAppAttemptEvent) event).getAppAttemptState();
+      try {
+        ApplicationAttemptStateData attemptStateData = 
+            ApplicationAttemptStateData.newInstance(attemptState);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Storing info for attempt: " + attemptState.getAttemptId());
+        }
+        store.storeApplicationAttemptStateInternal(attemptState.getAttemptId(),
+            attemptStateData);
+        store.notifyDoneStoringApplicationAttempt(attemptState.getAttemptId(),
+            null);
+      } catch (Exception e) {
+        LOG.error("Error storing appAttempt: " + attemptState.getAttemptId(), e);
+        store.notifyStoreOperationFailed(e);
+      }
+    };
+  }
+
+  private static class UpdateAppAttemptTransition implements
+      SingleArcTransition<RMStateStore, RMStateStoreEvent> {
+    @Override
+    public void transition(RMStateStore store, RMStateStoreEvent event) {
+      if (!(event instanceof RMStateUpdateAppAttemptEvent)) {
+        // should never happen
+        LOG.error("Illegal event type: " + event.getClass());
+        return;
+      }
+      ApplicationAttemptState attemptState =
+          ((RMStateUpdateAppAttemptEvent) event).getAppAttemptState();
+      try {
+        ApplicationAttemptStateData attemptStateData = ApplicationAttemptStateData
+            .newInstance(attemptState);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Updating info for attempt: " + attemptState.getAttemptId());
+        }
+        store.updateApplicationAttemptStateInternal(attemptState.getAttemptId(),
+            attemptStateData);
+        store.notifyDoneUpdatingApplicationAttempt(attemptState.getAttemptId(),
+            null);
+      } catch (Exception e) {
+        LOG.error("Error updating appAttempt: " + attemptState.getAttemptId(), e);
+        store.notifyStoreOperationFailed(e);
+      }
+    };
+  }
+
   public RMStateStore() {
     super(RMStateStore.class.getName());
+    stateMachine = stateMachineFactory.make(this);
   }
 
   /**
@@ -390,10 +549,10 @@
    * application.
    */
   protected abstract void storeApplicationStateInternal(ApplicationId appId,
-      ApplicationStateDataPBImpl appStateData) throws Exception;
+      ApplicationStateData appStateData) throws Exception;
 
   protected abstract void updateApplicationStateInternal(ApplicationId appId,
-      ApplicationStateDataPBImpl appStateData) throws Exception;
+      ApplicationStateData appStateData) throws Exception;
   
   @SuppressWarnings("unchecked")
   /**
@@ -428,11 +587,11 @@
    */
   protected abstract void storeApplicationAttemptStateInternal(
       ApplicationAttemptId attemptId,
-      ApplicationAttemptStateDataPBImpl attemptStateData) throws Exception;
+      ApplicationAttemptStateData attemptStateData) throws Exception;
 
   protected abstract void updateApplicationAttemptStateInternal(
       ApplicationAttemptId attemptId,
-      ApplicationAttemptStateDataPBImpl attemptStateData) throws Exception;
+      ApplicationAttemptStateData attemptStateData) throws Exception;
 
   /**
    * RMDTSecretManager call this to store the state of a delegation token
@@ -596,105 +755,10 @@
 
   // Dispatcher related code
   protected void handleStoreEvent(RMStateStoreEvent event) {
-    if (event.getType().equals(RMStateStoreEventType.STORE_APP)
-        || event.getType().equals(RMStateStoreEventType.UPDATE_APP)) {
-      ApplicationState appState = null;
-      if (event.getType().equals(RMStateStoreEventType.STORE_APP)) {
-        appState = ((RMStateStoreAppEvent) event).getAppState();
-      } else {
-        assert event.getType().equals(RMStateStoreEventType.UPDATE_APP);
-        appState = ((RMStateUpdateAppEvent) event).getAppState();
-      }
-
-      Exception storedException = null;
-      ApplicationStateDataPBImpl appStateData =
-          (ApplicationStateDataPBImpl) ApplicationStateDataPBImpl
-            .newApplicationStateData(appState.getSubmitTime(),
-              appState.getStartTime(), appState.getUser(),
-              appState.getApplicationSubmissionContext(), appState.getState(),
-              appState.getDiagnostics(), appState.getFinishTime());
-
-      ApplicationId appId =
-          appState.getApplicationSubmissionContext().getApplicationId();
-
-      LOG.info("Storing info for app: " + appId);
-      try {
-        if (event.getType().equals(RMStateStoreEventType.STORE_APP)) {
-          storeApplicationStateInternal(appId, appStateData);
-          notifyDoneStoringApplication(appId, storedException);
-        } else {
-          assert event.getType().equals(RMStateStoreEventType.UPDATE_APP);
-          updateApplicationStateInternal(appId, appStateData);
-          notifyDoneUpdatingApplication(appId, storedException);
-        }
-      } catch (Exception e) {
-        LOG.error("Error storing/updating app: " + appId, e);
-        notifyStoreOperationFailed(e);
-      }
-    } else if (event.getType().equals(RMStateStoreEventType.STORE_APP_ATTEMPT)
-        || event.getType().equals(RMStateStoreEventType.UPDATE_APP_ATTEMPT)) {
-
-      ApplicationAttemptState attemptState = null;
-      if (event.getType().equals(RMStateStoreEventType.STORE_APP_ATTEMPT)) {
-        attemptState =
-            ((RMStateStoreAppAttemptEvent) event).getAppAttemptState();
-      } else {
-        assert event.getType().equals(RMStateStoreEventType.UPDATE_APP_ATTEMPT);
-        attemptState =
-            ((RMStateUpdateAppAttemptEvent) event).getAppAttemptState();
-      }
-
-      Exception storedException = null;
-      Credentials credentials = attemptState.getAppAttemptCredentials();
-      ByteBuffer appAttemptTokens = null;
-      try {
-        if (credentials != null) {
-          DataOutputBuffer dob = new DataOutputBuffer();
-          credentials.writeTokenStorageToStream(dob);
-          appAttemptTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
-        }
-        ApplicationAttemptStateDataPBImpl attemptStateData =
-            (ApplicationAttemptStateDataPBImpl) ApplicationAttemptStateDataPBImpl
-              .newApplicationAttemptStateData(attemptState.getAttemptId(),
-                attemptState.getMasterContainer(), appAttemptTokens,
-                attemptState.getStartTime(), attemptState.getState(),
-                attemptState.getFinalTrackingUrl(),
-                attemptState.getDiagnostics(),
-                attemptState.getFinalApplicationStatus());
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Storing info for attempt: " + attemptState.getAttemptId());
-        }
-        if (event.getType().equals(RMStateStoreEventType.STORE_APP_ATTEMPT)) {
-          storeApplicationAttemptStateInternal(attemptState.getAttemptId(),
-              attemptStateData);
-          notifyDoneStoringApplicationAttempt(attemptState.getAttemptId(),
-              storedException);
-        } else {
-          assert event.getType().equals(
-            RMStateStoreEventType.UPDATE_APP_ATTEMPT);
-          updateApplicationAttemptStateInternal(attemptState.getAttemptId(),
-              attemptStateData);
-          notifyDoneUpdatingApplicationAttempt(attemptState.getAttemptId(),
-              storedException);
-        }
-      } catch (Exception e) {
-        LOG.error(
-            "Error storing/updating appAttempt: " + attemptState.getAttemptId(), e);
-        notifyStoreOperationFailed(e);
-      }
-    } else if (event.getType().equals(RMStateStoreEventType.REMOVE_APP)) {
-      ApplicationState appState =
-          ((RMStateStoreRemoveAppEvent) event).getAppState();
-      ApplicationId appId = appState.getAppId();
-      LOG.info("Removing info for app: " + appId);
-      try {
-        removeApplicationStateInternal(appState);
-      } catch (Exception e) {
-        LOG.error("Error removing app: " + appId, e);
-        notifyStoreOperationFailed(e);
-      }
-    } else {
-      LOG.error("Unknown RMStateStoreEvent type: " + event.getType());
+    try {
+      this.stateMachine.doTransition(event.getType(), event);
+    } catch (InvalidStateTransitonException e) {
+      LOG.error("Can't handle this event at current state", e);
     }
   }
 
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
index 31c8885..63ae990 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
@@ -49,6 +49,8 @@
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RMStateVersionProto;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.RMZKUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationAttemptStateData;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.RMStateVersion;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
@@ -551,7 +553,7 @@
 
   @Override
   public synchronized void storeApplicationStateInternal(ApplicationId appId,
-      ApplicationStateDataPBImpl appStateDataPB) throws Exception {
+      ApplicationStateData appStateDataPB) throws Exception {
     String nodeCreatePath = getNodePath(rmAppRoot, appId.toString());
 
     if (LOG.isDebugEnabled()) {
@@ -565,7 +567,7 @@
 
   @Override
   public synchronized void updateApplicationStateInternal(ApplicationId appId,
-      ApplicationStateDataPBImpl appStateDataPB) throws Exception {
+      ApplicationStateData appStateDataPB) throws Exception {
     String nodeUpdatePath = getNodePath(rmAppRoot, appId.toString());
 
     if (LOG.isDebugEnabled()) {
@@ -587,7 +589,7 @@
   @Override
   public synchronized void storeApplicationAttemptStateInternal(
       ApplicationAttemptId appAttemptId,
-      ApplicationAttemptStateDataPBImpl attemptStateDataPB)
+      ApplicationAttemptStateData attemptStateDataPB)
       throws Exception {
     String appDirPath = getNodePath(rmAppRoot,
         appAttemptId.getApplicationId().toString());
@@ -605,7 +607,7 @@
   @Override
   public synchronized void updateApplicationAttemptStateInternal(
       ApplicationAttemptId appAttemptId,
-      ApplicationAttemptStateDataPBImpl attemptStateDataPB)
+      ApplicationAttemptStateData attemptStateDataPB)
       throws Exception {
     String appIdStr = appAttemptId.getApplicationId().toString();
     String appAttemptIdStr = appAttemptId.toString();
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationAttemptStateData.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationAttemptStateData.java
index 255800e..6af048b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationAttemptStateData.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationAttemptStateData.java
@@ -18,31 +18,73 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.recovery.records;
 
+import java.io.IOException;
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationAttemptStateDataProto;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.util.Records;
 
 /*
  * Contains the state data that needs to be persisted for an ApplicationAttempt
  */
 @Public
 @Unstable
-public interface ApplicationAttemptStateData {
-  
+public abstract class ApplicationAttemptStateData {
+  public static ApplicationAttemptStateData newInstance(
+      ApplicationAttemptId attemptId, Container container,
+      ByteBuffer attemptTokens, long startTime, RMAppAttemptState finalState,
+      String finalTrackingUrl, String diagnostics,
+      FinalApplicationStatus amUnregisteredFinalStatus) {
+    ApplicationAttemptStateData attemptStateData =
+        Records.newRecord(ApplicationAttemptStateData.class);
+    attemptStateData.setAttemptId(attemptId);
+    attemptStateData.setMasterContainer(container);
+    attemptStateData.setAppAttemptTokens(attemptTokens);
+    attemptStateData.setState(finalState);
+    attemptStateData.setFinalTrackingUrl(finalTrackingUrl);
+    attemptStateData.setDiagnostics(diagnostics);
+    attemptStateData.setStartTime(startTime);
+    attemptStateData.setFinalApplicationStatus(amUnregisteredFinalStatus);
+    return attemptStateData;
+  }
+
+  public static ApplicationAttemptStateData newInstance(
+      ApplicationAttemptState attemptState) throws IOException {
+    Credentials credentials = attemptState.getAppAttemptCredentials();
+    ByteBuffer appAttemptTokens = null;
+    if (credentials != null) {
+      DataOutputBuffer dob = new DataOutputBuffer();
+      credentials.writeTokenStorageToStream(dob);
+      appAttemptTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+    }
+    return newInstance(attemptState.getAttemptId(),
+        attemptState.getMasterContainer(), appAttemptTokens,
+        attemptState.getStartTime(), attemptState.getState(),
+        attemptState.getFinalTrackingUrl(),
+        attemptState.getDiagnostics(),
+        attemptState.getFinalApplicationStatus());
+  }
+
+  public abstract ApplicationAttemptStateDataProto getProto();
+
   /**
    * The ApplicationAttemptId for the application attempt
    * @return ApplicationAttemptId for the application attempt
    */
   @Public
   @Unstable
-  public ApplicationAttemptId getAttemptId();
+  public abstract ApplicationAttemptId getAttemptId();
   
-  public void setAttemptId(ApplicationAttemptId attemptId);
+  public abstract void setAttemptId(ApplicationAttemptId attemptId);
   
   /*
    * The master container running the application attempt
@@ -50,9 +92,9 @@
    */
   @Public
   @Unstable
-  public Container getMasterContainer();
+  public abstract Container getMasterContainer();
   
-  public void setMasterContainer(Container container);
+  public abstract void setMasterContainer(Container container);
 
   /**
    * The application attempt tokens that belong to this attempt
@@ -60,17 +102,17 @@
    */
   @Public
   @Unstable
-  public ByteBuffer getAppAttemptTokens();
+  public abstract ByteBuffer getAppAttemptTokens();
 
-  public void setAppAttemptTokens(ByteBuffer attemptTokens);
+  public abstract void setAppAttemptTokens(ByteBuffer attemptTokens);
 
   /**
    * Get the final state of the application attempt.
    * @return the final state of the application attempt.
    */
-  public RMAppAttemptState getState();
+  public abstract RMAppAttemptState getState();
 
-  public void setState(RMAppAttemptState state);
+  public abstract void setState(RMAppAttemptState state);
 
   /**
    * Get the original not-proxied <em>final tracking url</em> for the
@@ -79,34 +121,34 @@
    * @return the original not-proxied <em>final tracking url</em> for the
    *         application
    */
-  public String getFinalTrackingUrl();
+  public abstract String getFinalTrackingUrl();
 
   /**
    * Set the final tracking Url of the AM.
    * @param url
    */
-  public void setFinalTrackingUrl(String url);
+  public abstract void setFinalTrackingUrl(String url);
   /**
    * Get the <em>diagnositic information</em> of the attempt 
    * @return <em>diagnositic information</em> of the attempt
    */
-  public String getDiagnostics();
+  public abstract String getDiagnostics();
 
-  public void setDiagnostics(String diagnostics);
+  public abstract void setDiagnostics(String diagnostics);
 
   /**
    * Get the <em>start time</em> of the application.
    * @return <em>start time</em> of the application
    */
-  public long getStartTime();
+  public abstract long getStartTime();
 
-  public void setStartTime(long startTime);
+  public abstract void setStartTime(long startTime);
 
   /**
    * Get the <em>final finish status</em> of the application.
    * @return <em>final finish status</em> of the application
    */
-  public FinalApplicationStatus getFinalApplicationStatus();
+  public abstract FinalApplicationStatus getFinalApplicationStatus();
 
-  public void setFinalApplicationStatus(FinalApplicationStatus finishState);
+  public abstract void setFinalApplicationStatus(FinalApplicationStatus finishState);
 }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationStateData.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationStateData.java
index 9fce6cf..55b726f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationStateData.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationStateData.java
@@ -24,7 +24,10 @@
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationStateDataProto;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.hadoop.yarn.util.Records;
 
 /**
  * Contains all the state data that needs to be stored persistently 
@@ -32,19 +35,43 @@
  */
 @Public
 @Unstable
-public interface ApplicationStateData {
-  
+public abstract class ApplicationStateData {
+  public static ApplicationStateData newInstance(long submitTime,
+      long startTime, String user,
+      ApplicationSubmissionContext submissionContext,
+      RMAppState state, String diagnostics, long finishTime) {
+    ApplicationStateData appState = Records.newRecord(ApplicationStateData.class);
+    appState.setSubmitTime(submitTime);
+    appState.setStartTime(startTime);
+    appState.setUser(user);
+    appState.setApplicationSubmissionContext(submissionContext);
+    appState.setState(state);
+    appState.setDiagnostics(diagnostics);
+    appState.setFinishTime(finishTime);
+    return appState;
+  }
+
+  public static ApplicationStateData newInstance(
+      ApplicationState appState) {
+    return newInstance(appState.getSubmitTime(), appState.getStartTime(),
+        appState.getUser(), appState.getApplicationSubmissionContext(),
+        appState.getState(), appState.getDiagnostics(),
+        appState.getFinishTime());
+  }
+
+  public abstract ApplicationStateDataProto getProto();
+
   /**
    * The time at which the application was received by the Resource Manager
    * @return submitTime
    */
   @Public
   @Unstable
-  public long getSubmitTime();
+  public abstract long getSubmitTime();
   
   @Public
   @Unstable
-  public void setSubmitTime(long submitTime);
+  public abstract void setSubmitTime(long submitTime);
 
   /**
    * Get the <em>start time</em> of the application.
@@ -63,11 +90,11 @@
    */
   @Public
   @Unstable
-  public void setUser(String user);
+  public abstract void setUser(String user);
   
   @Public
   @Unstable
-  public String getUser();
+  public abstract String getUser();
   
   /**
    * The {@link ApplicationSubmissionContext} for the application
@@ -76,34 +103,34 @@
    */
   @Public
   @Unstable
-  public ApplicationSubmissionContext getApplicationSubmissionContext();
+  public abstract ApplicationSubmissionContext getApplicationSubmissionContext();
   
   @Public
   @Unstable
-  public void setApplicationSubmissionContext(
+  public abstract void setApplicationSubmissionContext(
       ApplicationSubmissionContext context);
 
   /**
    * Get the final state of the application.
    * @return the final state of the application.
    */
-  public RMAppState getState();
+  public abstract RMAppState getState();
 
-  public void setState(RMAppState state);
+  public abstract void setState(RMAppState state);
 
   /**
    * Get the diagnostics information for the application master.
    * @return the diagnostics information for the application master.
    */
-  public String getDiagnostics();
+  public abstract String getDiagnostics();
 
-  public void setDiagnostics(String diagnostics);
+  public abstract void setDiagnostics(String diagnostics);
 
   /**
    * The finish time of the application.
    * @return the finish time of the application.,
    */
-  public long getFinishTime();
+  public abstract long getFinishTime();
 
-  public void setFinishTime(long finishTime);
+  public abstract void setFinishTime(long finishTime);
 }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/impl/pb/ApplicationAttemptStateDataPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/impl/pb/ApplicationAttemptStateDataPBImpl.java
index 75ac2ee..e3ebe5e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/impl/pb/ApplicationAttemptStateDataPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/impl/pb/ApplicationAttemptStateDataPBImpl.java
@@ -25,10 +25,7 @@
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ProtoBase;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.proto.YarnProtos.FinalApplicationStatusProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationAttemptStateDataProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationAttemptStateDataProtoOrBuilder;
@@ -36,12 +33,10 @@
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationAttemptStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 
-public class ApplicationAttemptStateDataPBImpl
-extends ProtoBase<ApplicationAttemptStateDataProto> 
-implements ApplicationAttemptStateData {
-  private static final RecordFactory recordFactory = RecordFactoryProvider
-      .getRecordFactory(null);
+import com.google.protobuf.TextFormat;
 
+public class ApplicationAttemptStateDataPBImpl extends
+    ApplicationAttemptStateData {
   ApplicationAttemptStateDataProto proto = 
       ApplicationAttemptStateDataProto.getDefaultInstance();
   ApplicationAttemptStateDataProto.Builder builder = null;
@@ -60,7 +55,8 @@
     this.proto = proto;
     viaProto = true;
   }
-  
+
+  @Override
   public ApplicationAttemptStateDataProto getProto() {
     mergeLocalToProto();
     proto = viaProto ? proto : builder.build();
@@ -76,7 +72,8 @@
       builder.setMasterContainer(((ContainerPBImpl)masterContainer).getProto());
     }
     if(this.appAttemptTokens != null) {
-      builder.setAppAttemptTokens(convertToProtoFormat(this.appAttemptTokens));
+      builder.setAppAttemptTokens(ProtoUtils.convertToProtoFormat(
+          this.appAttemptTokens));
     }
   }
 
@@ -148,7 +145,8 @@
     if(!p.hasAppAttemptTokens()) {
       return null;
     }
-    this.appAttemptTokens = convertFromProtoFormat(p.getAppAttemptTokens());
+    this.appAttemptTokens = ProtoUtils.convertFromProtoFormat(
+        p.getAppAttemptTokens());
     return appAttemptTokens;
   }
 
@@ -249,24 +247,26 @@
     builder.setFinalApplicationStatus(convertToProtoFormat(finishState));
   }
 
-  public static ApplicationAttemptStateData newApplicationAttemptStateData(
-      ApplicationAttemptId attemptId, Container container,
-      ByteBuffer attemptTokens, long startTime, RMAppAttemptState finalState,
-      String finalTrackingUrl, String diagnostics,
-      FinalApplicationStatus amUnregisteredFinalStatus) {
-    ApplicationAttemptStateData attemptStateData =
-        recordFactory.newRecordInstance(ApplicationAttemptStateData.class);
-    attemptStateData.setAttemptId(attemptId);
-    attemptStateData.setMasterContainer(container);
-    attemptStateData.setAppAttemptTokens(attemptTokens);
-    attemptStateData.setState(finalState);
-    attemptStateData.setFinalTrackingUrl(finalTrackingUrl);
-    attemptStateData.setDiagnostics(diagnostics);
-    attemptStateData.setStartTime(startTime);
-    attemptStateData.setFinalApplicationStatus(amUnregisteredFinalStatus);
-    return attemptStateData;
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
   }
 
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+  
   private static String RM_APP_ATTEMPT_PREFIX = "RMATTEMPT_";
   public static RMAppAttemptStateProto convertToProtoFormat(RMAppAttemptState e) {
     return RMAppAttemptStateProto.valueOf(RM_APP_ATTEMPT_PREFIX + e.name());
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/impl/pb/ApplicationStateDataPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/impl/pb/ApplicationStateDataPBImpl.java
index ede8ca7..8aaf1a4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/impl/pb/ApplicationStateDataPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/impl/pb/ApplicationStateDataPBImpl.java
@@ -20,21 +20,15 @@
 
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ProtoBase;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationStateDataProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationStateDataProtoOrBuilder;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RMAppStateProto;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 
-public class ApplicationStateDataPBImpl 
-extends ProtoBase<ApplicationStateDataProto> 
-implements ApplicationStateData {
-  private static final RecordFactory recordFactory = RecordFactoryProvider
-      .getRecordFactory(null);
+import com.google.protobuf.TextFormat;
 
+public class ApplicationStateDataPBImpl extends ApplicationStateData {
   ApplicationStateDataProto proto = 
             ApplicationStateDataProto.getDefaultInstance();
   ApplicationStateDataProto.Builder builder = null;
@@ -51,7 +45,8 @@
     this.proto = proto;
     viaProto = true;
   }
-  
+
+  @Override
   public ApplicationStateDataProto getProto() {
     mergeLocalToProto();
     proto = viaProto ? proto : builder.build();
@@ -136,7 +131,7 @@
     }
     applicationSubmissionContext = 
         new ApplicationSubmissionContextPBImpl(
-                                          p.getApplicationSubmissionContext());
+            p.getApplicationSubmissionContext());
     return applicationSubmissionContext;
   }
 
@@ -200,21 +195,24 @@
     builder.setFinishTime(finishTime);
   }
 
-  public static ApplicationStateData newApplicationStateData(long submitTime,
-      long startTime, String user,
-      ApplicationSubmissionContext submissionContext, RMAppState state,
-      String diagnostics, long finishTime) {
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
 
-    ApplicationStateData appState =
-        recordFactory.newRecordInstance(ApplicationStateData.class);
-    appState.setSubmitTime(submitTime);
-    appState.setStartTime(startTime);
-    appState.setUser(user);
-    appState.setApplicationSubmissionContext(submissionContext);
-    appState.setState(state);
-    appState.setDiagnostics(diagnostics);
-    appState.setFinishTime(finishTime);
-    return appState;
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
   }
 
   private static String RM_APP_PREFIX = "RMAPP_";
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index f4f2e20..3318f15 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -71,6 +71,7 @@
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
@@ -293,11 +294,6 @@
   private final StateMachine<RMAppState, RMAppEventType, RMAppEvent>
                                                                  stateMachine;
 
-  private static final ApplicationResourceUsageReport
-    DUMMY_APPLICATION_RESOURCE_USAGE_REPORT =
-      BuilderUtils.newApplicationResourceUsageReport(-1, -1,
-          Resources.createResource(-1, -1), Resources.createResource(-1, -1),
-          Resources.createResource(-1, -1));
   private static final int DUMMY_APPLICATION_ATTEMPT_NUMBER = -1;
   
   public RMAppImpl(ApplicationId applicationId, RMContext rmContext,
@@ -498,7 +494,7 @@
       String origTrackingUrl = UNAVAILABLE;
       int rpcPort = -1;
       ApplicationResourceUsageReport appUsageReport =
-          DUMMY_APPLICATION_RESOURCE_USAGE_REPORT;
+          RMServerUtils.DUMMY_APPLICATION_RESOURCE_USAGE_REPORT;
       FinalApplicationStatus finishState = getFinalApplicationStatus();
       String diags = UNAVAILABLE;
       float progress = 0.0f;
@@ -723,29 +719,36 @@
     }
   }
 
+  // synchronously recover attempt to ensure any incoming external events
+  // to be processed after the attempt processes the recover event.
+  private void recoverAppAttempts() {
+    for (RMAppAttempt attempt : getAppAttempts().values()) {
+      attempt.handle(new RMAppAttemptEvent(attempt.getAppAttemptId(),
+        RMAppAttemptEventType.RECOVER));
+    }
+  }
+
   private static final class RMAppRecoveredTransition implements
       MultipleArcTransition<RMAppImpl, RMAppEvent, RMAppState> {
 
     @Override
     public RMAppState transition(RMAppImpl app, RMAppEvent event) {
 
-      for (RMAppAttempt attempt : app.getAppAttempts().values()) {
-        // synchronously recover attempt to ensure any incoming external events
-        // to be processed after the attempt processes the recover event.
-        attempt.handle(
-          new RMAppAttemptEvent(attempt.getAppAttemptId(),
-            RMAppAttemptEventType.RECOVER));
-      }
-
       // The app has completed.
       if (app.recoveredFinalState != null) {
+        app.recoverAppAttempts();
         new FinalTransition(app.recoveredFinalState).transition(app, event);
         return app.recoveredFinalState;
       }
 
-      // Last attempt is in final state, do not add to scheduler and just return
-      // ACCEPTED waiting for last RMAppAttempt to send finished or failed event
-      // back.
+      // Notify scheduler about the app on recovery
+      new AddApplicationToSchedulerTransition().transition(app, event);
+
+      // recover attempts
+      app.recoverAppAttempts();
+
+      // Last attempt is in final state, return ACCEPTED waiting for last
+      // RMAppAttempt to send finished or failed event back.
       if (app.currentAttempt != null
           && (app.currentAttempt.getState() == RMAppAttemptState.KILLED
               || app.currentAttempt.getState() == RMAppAttemptState.FINISHED
@@ -754,9 +757,6 @@
         return RMAppState.ACCEPTED;
       }
 
-      // Notify scheduler about the app on recovery
-      new AddApplicationToSchedulerTransition().transition(app, event);
-
       // No existent attempts means the attempt associated with this app was not
       // started or started but not yet saved.
       if (app.attempts.isEmpty()) {
@@ -1055,8 +1055,12 @@
       if (app.finishTime == 0 ) {
         app.finishTime = System.currentTimeMillis();
       }
-      app.handler.handle(new AppRemovedSchedulerEvent(app.applicationId,
-        finalState));
+      // Recovered apps that are completed were not added to scheduler, so no
+      // need to remove them from scheduler.
+      if (app.recoveredFinalState == null) {
+        app.handler.handle(new AppRemovedSchedulerEvent(app.applicationId,
+          finalState));
+      }
       app.handler.handle(
           new RMAppManagerEvent(app.applicationId,
           RMAppManagerEventType.APP_COMPLETED));
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
index e289ad5..2a1170d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
@@ -267,15 +267,17 @@
       .addTransition(RMAppAttemptState.ALLOCATED, RMAppAttemptState.FINAL_SAVING,
           RMAppAttemptEventType.CONTAINER_FINISHED,
           new FinalSavingTransition(
-            new AMContainerCrashedTransition(), RMAppAttemptState.FAILED))
+            new AMContainerCrashedBeforeRunningTransition(), RMAppAttemptState.FAILED))
 
        // Transitions from LAUNCHED State
       .addTransition(RMAppAttemptState.LAUNCHED, RMAppAttemptState.RUNNING,
           RMAppAttemptEventType.REGISTERED, new AMRegisteredTransition())
-      .addTransition(RMAppAttemptState.LAUNCHED, RMAppAttemptState.FINAL_SAVING,
+      .addTransition(RMAppAttemptState.LAUNCHED,
+          EnumSet.of(RMAppAttemptState.LAUNCHED, RMAppAttemptState.FINAL_SAVING),
           RMAppAttemptEventType.CONTAINER_FINISHED,
-          new FinalSavingTransition(
-            new AMContainerCrashedTransition(), RMAppAttemptState.FAILED))
+          new ContainerFinishedTransition(
+            new AMContainerCrashedBeforeRunningTransition(),
+            RMAppAttemptState.LAUNCHED))
       .addTransition(
           RMAppAttemptState.LAUNCHED, RMAppAttemptState.FINAL_SAVING,
           RMAppAttemptEventType.EXPIRE,
@@ -302,7 +304,9 @@
           RMAppAttemptState.RUNNING,
           EnumSet.of(RMAppAttemptState.RUNNING, RMAppAttemptState.FINAL_SAVING),
           RMAppAttemptEventType.CONTAINER_FINISHED,
-          new ContainerFinishedTransition())
+          new ContainerFinishedTransition(
+            new AMContainerCrashedAtRunningTransition(),
+            RMAppAttemptState.RUNNING))
       .addTransition(
           RMAppAttemptState.RUNNING, RMAppAttemptState.FINAL_SAVING,
           RMAppAttemptEventType.EXPIRE,
@@ -671,9 +675,7 @@
       ApplicationResourceUsageReport report =
           scheduler.getAppResourceUsageReport(this.getAppAttemptId());
       if (report == null) {
-        Resource none = Resource.newInstance(0, 0);
-        report = ApplicationResourceUsageReport.newInstance(0, 0, none, none,
-            none);
+        report = RMServerUtils.DUMMY_APPLICATION_RESOURCE_USAGE_REPORT;
       }
       return report;
     } finally {
@@ -904,6 +906,12 @@
         }
         return appAttempt.recoveredFinalState;
       } else {
+        // Add the current attempt to the scheduler.
+        if (appAttempt.rmContext.isWorkPreservingRecoveryEnabled()) {
+          appAttempt.eventHandler.handle(new AppAttemptAddedSchedulerEvent(
+            appAttempt.getAppAttemptId(), false));
+        }
+
         /*
          * Since the application attempt's final state is not saved that means
          * for AM container (previous attempt) state must be one of these.
@@ -1207,17 +1215,16 @@
     }
   }
 
-  private static final class AMContainerCrashedTransition extends
+  private static final class AMContainerCrashedBeforeRunningTransition extends
       BaseFinalTransition {
 
-    public AMContainerCrashedTransition() {
+    public AMContainerCrashedBeforeRunningTransition() {
       super(RMAppAttemptState.FAILED);
     }
 
     @Override
     public void transition(RMAppAttemptImpl appAttempt,
         RMAppAttemptEvent event) {
-
       RMAppAttemptContainerFinishedEvent finishEvent =
           ((RMAppAttemptContainerFinishedEvent)event);
 
@@ -1410,6 +1417,16 @@
       implements
       MultipleArcTransition<RMAppAttemptImpl, RMAppAttemptEvent, RMAppAttemptState> {
 
+    // The transition To Do after attempt final state is saved.
+    private BaseTransition transitionToDo;
+    private RMAppAttemptState currentState;
+
+    public ContainerFinishedTransition(BaseTransition transitionToDo,
+        RMAppAttemptState currentState) {
+      this.transitionToDo = transitionToDo;
+      this.currentState = currentState;
+    }
+
     @Override
     public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
         RMAppAttemptEvent event) {
@@ -1426,14 +1443,13 @@
             containerStatus.getContainerId())) {
         // Remember the follow up transition and save the final attempt state.
         appAttempt.rememberTargetTransitionsAndStoreState(event,
-          new ContainerFinishedFinalStateSavedTransition(),
-          RMAppAttemptState.FAILED, RMAppAttemptState.FAILED);
+          transitionToDo, RMAppAttemptState.FAILED, RMAppAttemptState.FAILED);
         return RMAppAttemptState.FINAL_SAVING;
       }
 
       // Normal container.Put it in completedcontainers list
       appAttempt.justFinishedContainers.add(containerStatus);
-      return RMAppAttemptState.RUNNING;
+      return this.currentState;
     }
   }
 
@@ -1451,7 +1467,7 @@
     }
   }
 
-  private static class ContainerFinishedFinalStateSavedTransition extends
+  private static class AMContainerCrashedAtRunningTransition extends
       BaseTransition {
     @Override
     public void
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerEventType.java
index ace4435..259d68b3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerEventType.java
@@ -33,5 +33,7 @@
   RELEASED,
 
   // Source: ContainerAllocationExpirer  
-  EXPIRE
+  EXPIRE,
+
+  RECOVER
 }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index 2921891..01db215 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -35,12 +35,14 @@
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
 import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
+import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
@@ -65,6 +67,9 @@
         RMContainerEventType.KILL)
     .addTransition(RMContainerState.NEW, RMContainerState.RESERVED,
         RMContainerEventType.RESERVED, new ContainerReservedTransition())
+    .addTransition(RMContainerState.NEW,
+        EnumSet.of(RMContainerState.RUNNING, RMContainerState.COMPLETED),
+        RMContainerEventType.RECOVER, new ContainerRecoveredTransition())
 
     // Transitions from RESERVED state
     .addTransition(RMContainerState.RESERVED, RMContainerState.RESERVED, 
@@ -341,6 +346,38 @@
     }
   }
 
+  private static final class ContainerRecoveredTransition
+      implements
+      MultipleArcTransition<RMContainerImpl, RMContainerEvent, RMContainerState> {
+    @Override
+    public RMContainerState transition(RMContainerImpl container,
+        RMContainerEvent event) {
+      NMContainerStatus report =
+          ((RMContainerRecoverEvent) event).getContainerReport();
+      if (report.getContainerState().equals(ContainerState.COMPLETE)) {
+        ContainerStatus status =
+            ContainerStatus.newInstance(report.getContainerId(),
+              report.getContainerState(), report.getDiagnostics(),
+              report.getContainerExitStatus());
+
+        new FinishedTransition().transition(container,
+          new RMContainerFinishedEvent(container.containerId, status,
+            RMContainerEventType.FINISHED));
+        return RMContainerState.COMPLETED;
+      } else if (report.getContainerState().equals(ContainerState.RUNNING)) {
+        // Tell the appAttempt
+        container.eventHandler.handle(new RMAppAttemptContainerAcquiredEvent(
+            container.getApplicationAttemptId(), container.getContainer()));
+        return RMContainerState.RUNNING;
+      } else {
+        // This can never happen.
+        LOG.warn("RMContainer received unexpected recover event with container"
+            + " state " + report.getContainerState() + " while recovering.");
+        return RMContainerState.RUNNING;
+      }
+    }
+  }
+
   private static final class ContainerReservedTransition extends
   BaseTransition {
 
@@ -398,7 +435,6 @@
       // Inform AppAttempt
       container.eventHandler.handle(new RMAppAttemptContainerFinishedEvent(
           container.appAttemptId, finishedEvent.getRemoteContainerStatus()));
-
       container.rmContext.getRMApplicationHistoryWriter()
           .containerFinished(container);
     }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerRecoverEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerRecoverEvent.java
new file mode 100644
index 0000000..338ccbd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerRecoverEvent.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.rmcontainer;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
+
+public class RMContainerRecoverEvent extends RMContainerEvent {
+
+  private final NMContainerStatus containerReport;
+
+  public RMContainerRecoverEvent(ContainerId containerId,
+      NMContainerStatus containerReport) {
+    super(containerId, RMContainerEventType.RECOVER);
+    this.containerReport = containerReport;
+  }
+
+  public NMContainerStatus getContainerReport() {
+    return containerReport;
+  }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
index 0d33796..66a7d96 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
@@ -48,6 +48,7 @@
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.ClusterMetrics;
@@ -460,13 +461,9 @@
     @Override
     public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
       // Inform the scheduler
+      RMNodeStartedEvent startEvent = (RMNodeStartedEvent) event;
+      List<NMContainerStatus> containers = null;
 
-      rmNode.context.getDispatcher().getEventHandler().handle(
-          new NodeAddedSchedulerEvent(rmNode));
-      rmNode.context.getDispatcher().getEventHandler().handle(
-          new NodesListManagerEvent(
-              NodesListManagerEventType.NODE_USABLE, rmNode));
- 
       String host = rmNode.nodeId.getHost();
       if (rmNode.context.getInactiveRMNodes().containsKey(host)) {
         // Old node rejoining
@@ -476,10 +473,17 @@
       } else {
         // Increment activeNodes explicitly because this is a new node.
         ClusterMetrics.getMetrics().incrNumActiveNodes();
+        containers = startEvent.getContainerRecoveryReports();
       }
+
+      rmNode.context.getDispatcher().getEventHandler()
+        .handle(new NodeAddedSchedulerEvent(rmNode, containers));
+      rmNode.context.getDispatcher().getEventHandler().handle(
+        new NodesListManagerEvent(
+            NodesListManagerEventType.NODE_USABLE, rmNode));
     }
   }
-  
+
   public static class ReconnectNodeTransition implements
       SingleArcTransition<RMNodeImpl, RMNodeEvent> {
 
@@ -513,7 +517,7 @@
         }
         rmNode.context.getRMNodes().put(newNode.getNodeID(), newNode);
         rmNode.context.getDispatcher().getEventHandler().handle(
-            new RMNodeEvent(newNode.getNodeID(), RMNodeEventType.STARTED));
+            new RMNodeStartedEvent(newNode.getNodeID(), null));
       }
       rmNode.context.getDispatcher().getEventHandler().handle(
           new NodesListManagerEvent(
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStartedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStartedEvent.java
new file mode 100644
index 0000000..0414347
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStartedEvent.java
@@ -0,0 +1,38 @@
+/**
+ * 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.rmnode;
+
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
+
+public class RMNodeStartedEvent extends RMNodeEvent {
+
+  private List<NMContainerStatus> containerReports;
+
+  public RMNodeStartedEvent(NodeId nodeId, List<NMContainerStatus> containerReports) {
+    super(nodeId, RMNodeEventType.STARTED);
+    this.containerReports = containerReports;
+  }
+
+  public List<NMContainerStatus> getContainerRecoveryReports() {
+    return this.containerReports;
+  }
+}
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/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index cc1cb47..9c878fe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -32,14 +32,21 @@
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerRecoverEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+@SuppressWarnings("unchecked")
 public abstract class AbstractYarnScheduler
     <T extends SchedulerApplicationAttempt, N extends SchedulerNode>
     extends AbstractService implements ResourceScheduler {
@@ -47,8 +54,7 @@
   private static final Log LOG = LogFactory.getLog(AbstractYarnScheduler.class);
 
   // Nodes in the cluster, indexed by NodeId
-  protected Map<NodeId, N> nodes =
-      new ConcurrentHashMap<NodeId, N>();
+  protected Map<NodeId, N> nodes = new ConcurrentHashMap<NodeId, N>();
 
   // Whole capacity of the cluster
   protected Resource clusterResource = Resource.newInstance(0, 0);
@@ -58,6 +64,7 @@
 
   protected RMContext rmContext;
   protected Map<ApplicationId, SchedulerApplication<T>> applications;
+
   protected final static List<Container> EMPTY_CONTAINER_LIST =
       new ArrayList<Container>();
   protected static final Allocation EMPTY_ALLOCATION = new Allocation(
@@ -169,4 +176,90 @@
     throw new YarnException(getClass().getSimpleName()
         + " does not support moving apps between queues");
   }
+
+  private void killOrphanContainerOnNode(RMNode node,
+      NMContainerStatus container) {
+    if (!container.getContainerState().equals(ContainerState.COMPLETE)) {
+      this.rmContext.getDispatcher().getEventHandler().handle(
+        new RMNodeCleanContainerEvent(node.getNodeID(),
+          container.getContainerId()));
+    }
+  }
+
+  public synchronized void recoverContainersOnNode(
+      List<NMContainerStatus> containerReports, RMNode nm) {
+    if (!rmContext.isWorkPreservingRecoveryEnabled()
+        || containerReports == null
+        || (containerReports != null && containerReports.isEmpty())) {
+      return;
+    }
+
+    for (NMContainerStatus container : containerReports) {
+      ApplicationId appId =
+          container.getContainerId().getApplicationAttemptId().getApplicationId();
+      RMApp rmApp = rmContext.getRMApps().get(appId);
+      if (rmApp == null) {
+        LOG.error("Skip recovering container " + container
+            + " for unknown application.");
+        killOrphanContainerOnNode(nm, container);
+        continue;
+      }
+
+      // Unmanaged AM recovery is addressed in YARN-1815
+      if (rmApp.getApplicationSubmissionContext().getUnmanagedAM()) {
+        LOG.info("Skip recovering container " + container + " for unmanaged AM."
+            + rmApp.getApplicationId());
+        killOrphanContainerOnNode(nm, container);
+        continue;
+      }
+
+      SchedulerApplication<T> schedulerApp = applications.get(appId);
+      if (schedulerApp == null) {
+        LOG.info("Skip recovering container  " + container
+            + " for unknown SchedulerApplication. Application current state is "
+            + rmApp.getState());
+        killOrphanContainerOnNode(nm, container);
+        continue;
+      }
+
+      LOG.info("Recovering container " + container);
+      SchedulerApplicationAttempt schedulerAttempt =
+          schedulerApp.getCurrentAppAttempt();
+
+      // create container
+      RMContainer rmContainer = recoverAndCreateContainer(container, nm);
+
+      // recover RMContainer
+      rmContainer.handle(new RMContainerRecoverEvent(container.getContainerId(),
+        container));
+
+      // recover scheduler node
+      nodes.get(nm.getNodeID()).recoverContainer(rmContainer);
+
+      // recover queue: update headroom etc.
+      Queue queue = schedulerAttempt.getQueue();
+      queue.recoverContainer(clusterResource, schedulerAttempt, rmContainer);
+
+      // recover scheduler attempt
+      schedulerAttempt.recoverContainer(rmContainer);
+    }
+  }
+
+  private RMContainer recoverAndCreateContainer(NMContainerStatus report,
+      RMNode node) {
+    Container container =
+        Container.newInstance(report.getContainerId(), node.getNodeID(),
+          node.getHttpAddress(), report.getAllocatedResource(),
+          report.getPriority(), null);
+    ApplicationAttemptId attemptId =
+        container.getId().getApplicationAttemptId();
+    RMContainer rmContainer =
+        new RMContainerImpl(container, attemptId, node.getNodeID(),
+          applications.get(attemptId.getApplicationId()).getUser(), rmContext);
+    return rmContainer;
+  }
+
+  public SchedulerNode getSchedulerNode(NodeId nodeId) {
+    return nodes.get(nodeId);
+  }
 }
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/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
index de71f71..d3d03fd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
@@ -39,6 +39,8 @@
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 /**
@@ -409,4 +411,25 @@
     //    this.requests = appInfo.getRequests();
     this.blacklist = appInfo.getBlackList();
   }
+
+  public synchronized void recoverContainer(RMContainer rmContainer) {
+    // ContainerIdCounter on recovery will be addressed in YARN-2052
+    this.containerIdCounter.incrementAndGet();
+
+    QueueMetrics metrics = queue.getMetrics();
+    if (pending) {
+      // If there was any container to recover, the application was
+      // running from scheduler's POV.
+      pending = false;
+      metrics.runAppAttempt(applicationId, user);
+    }
+
+    // Container is completed. Skip recovering resources.
+    if (rmContainer.getState().equals(RMContainerState.COMPLETED)) {
+      return;
+    }
+
+    metrics.allocateResources(user, 1, rmContainer.getAllocatedResource(),
+      false);
+  }
 }
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/Queue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Queue.java
index c51f819..0bc8ca1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Queue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Queue.java
@@ -26,6 +26,8 @@
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 
 @Evolving
 @LimitedPrivate("yarn")
@@ -60,4 +62,13 @@
   boolean hasAccess(QueueACL acl, UserGroupInformation user);
   
   public ActiveUsersManager getActiveUsersManager();
+
+  /**
+   * Recover the state of the queue for a given container.
+   * @param clusterResource the resource of the cluster
+   * @param schedulerAttempt the application for which the container was allocated
+   * @param rmContainer the container that was recovered.
+   */
+  public void recoverContainer(Resource clusterResource,
+      SchedulerApplicationAttempt schedulerAttempt, RMContainer rmContainer);
 }
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/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index ed41a6b..09cf52d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -47,6 +47,7 @@
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerReservedEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
@@ -534,5 +535,24 @@
 
     appSchedulingInfo.move(newQueue);
     this.queue = newQueue;
-  }  
+  }
+
+  public synchronized void recoverContainer(RMContainer rmContainer) {
+    // recover app scheduling info
+    appSchedulingInfo.recoverContainer(rmContainer);
+
+    if (rmContainer.getState().equals(RMContainerState.COMPLETED)) {
+      return;
+    }
+    LOG.info("SchedulerAttempt " + getApplicationAttemptId()
+      + " is recovering container " + rmContainer.getContainerId());
+    liveContainers.put(rmContainer.getContainerId(), rmContainer);
+    Resources.addTo(currentConsumption, rmContainer.getContainer()
+      .getResource());
+    // resourceLimit: updated when LeafQueue#recoverContainer#allocateResource
+    // is called.
+    // newlyAllocatedContainers.add(rmContainer);
+    // schedulingOpportunities
+    // lastScheduledContainer
+  }
 }
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 85d016b..9fb8d23 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
@@ -27,7 +27,6 @@
 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.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -35,10 +34,10 @@
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import com.google.common.base.Preconditions;
 
 /**
  * Represents a YARN Cluster Node from the viewpoint of the scheduler.
@@ -119,13 +118,10 @@
    * The Scheduler has allocated containers on this node to the given
    * application.
    * 
-   * @param applicationId
-   *          application
    * @param rmContainer
    *          allocated container
    */
-  public synchronized void allocateContainer(ApplicationId applicationId,
-      RMContainer rmContainer) {
+  public synchronized void allocateContainer(RMContainer rmContainer) {
     Container container = rmContainer.getContainer();
     deductAvailableResource(container.getResource());
     ++numContainers;
@@ -166,8 +162,8 @@
     return this.totalResourceCapability;
   }
 
-  private synchronized boolean isValidContainer(Container c) {
-    if (launchedContainers.containsKey(c.getId())) {
+  public synchronized boolean isValidContainer(ContainerId containerId) {
+    if (launchedContainers.containsKey(containerId)) {
       return true;
     }
     return false;
@@ -185,7 +181,7 @@
    *          container to be released
    */
   public synchronized void releaseContainer(Container container) {
-    if (!isValidContainer(container)) {
+    if (!isValidContainer(container.getId())) {
       LOG.error("Invalid container released " + container);
       return;
     }
@@ -274,4 +270,12 @@
     // we can only adjust available resource if total resource is changed.
     Resources.addTo(this.availableResource, deltaResource);
   }
+
+
+  public synchronized void recoverContainer(RMContainer rmContainer) {
+    if (rmContainer.getState().equals(RMContainerState.COMPLETED)) {
+      return;
+    }
+    allocateContainer(rmContainer);
+  }
 }
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/CSQueue.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/CSQueue.java
index f5090ba..ccb71e2 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/CSQueue.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/CSQueue.java
@@ -28,7 +28,6 @@
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueState;
@@ -235,15 +234,6 @@
   public ActiveUsersManager getActiveUsersManager();
   
   /**
-   * Recover the state of the queue
-   * @param clusterResource the resource of the cluster
-   * @param application the application for which the container was allocated
-   * @param container the container that was recovered.
-   */
-  public void recoverContainer(Resource clusterResource, FiCaSchedulerApp application, 
-      Container container);
-  
-  /**
    * Adds all applications in the queue and its subqueues to the given collection.
    * @param apps the collection to add the applications to
    */
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/CapacityScheduler.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/CapacityScheduler.java
index 7767445..5de407d 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/CapacityScheduler.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/CapacityScheduler.java
@@ -872,6 +872,8 @@
     {
       NodeAddedSchedulerEvent nodeAddedEvent = (NodeAddedSchedulerEvent)event;
       addNode(nodeAddedEvent.getAddedRMNode());
+      recoverContainersOnNode(nodeAddedEvent.getContainerReports(),
+        nodeAddedEvent.getAddedRMNode());
     }
     break;
     case NODE_REMOVED:
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/LeafQueue.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/LeafQueue.java
index 5ddb9a4..65938aa 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/LeafQueue.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/LeafQueue.java
@@ -59,15 +59,17 @@
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
-import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.Lock;
 import org.apache.hadoop.yarn.server.utils.Lock.NoLock;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
+import com.google.common.annotations.VisibleForTesting;
+
 @Private
 @Unstable
 public class LeafQueue implements CSQueue {
@@ -564,7 +566,8 @@
         "numContainers=" + getNumContainers();  
   }
 
-  private synchronized User getUser(String userName) {
+  @VisibleForTesting
+  public synchronized User getUser(String userName) {
     User user = users.get(userName);
     if (user == null) {
       user = new User();
@@ -1346,8 +1349,7 @@
       }
 
       // Inform the node
-      node.allocateContainer(application.getApplicationId(), 
-          allocatedContainer);
+      node.allocateContainer(allocatedContainer);
 
       LOG.info("assignedContainer" +
           " application attempt=" + application.getApplicationAttemptId() +
@@ -1446,7 +1448,7 @@
   }
 
   synchronized void allocateResource(Resource clusterResource, 
-      FiCaSchedulerApp application, Resource resource) {
+      SchedulerApplicationAttempt application, Resource resource) {
     // Update queue metrics
     Resources.addTo(usedResources, resource);
     CSQueueUtils.updateQueueStatistics(
@@ -1530,7 +1532,8 @@
     return metrics;
   }
 
-  static class User {
+  @VisibleForTesting
+  public static class User {
     Resource consumed = Resources.createResource(0, 0);
     int pendingApplications = 0;
     int activeApplications = 0;
@@ -1580,13 +1583,16 @@
 
   @Override
   public void recoverContainer(Resource clusterResource,
-      FiCaSchedulerApp application, Container container) {
+      SchedulerApplicationAttempt attempt, RMContainer rmContainer) {
+    if (rmContainer.getState().equals(RMContainerState.COMPLETED)) {
+      return;
+    }
     // Careful! Locking order is important! 
     synchronized (this) {
-      allocateResource(clusterResource, application, container.getResource());
+      allocateResource(clusterResource, attempt, rmContainer.getContainer()
+        .getResource());
     }
-    getParent().recoverContainer(clusterResource, application, container);
-
+    getParent().recoverContainer(clusterResource, attempt, rmContainer);
   }
 
   /**
@@ -1613,5 +1619,4 @@
       apps.add(app.getApplicationAttemptId());
     }
   }
-
 }
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/ParentQueue.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/ParentQueue.java
index dba92a6..d83eed3 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/ParentQueue.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/ParentQueue.java
@@ -38,7 +38,6 @@
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
@@ -49,9 +48,11 @@
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@@ -770,13 +771,16 @@
   
   @Override
   public void recoverContainer(Resource clusterResource,
-      FiCaSchedulerApp application, Container container) {
+      SchedulerApplicationAttempt attempt, RMContainer rmContainer) {
+    if (rmContainer.getState().equals(RMContainerState.COMPLETED)) {
+      return;
+    }
     // Careful! Locking order is important! 
     synchronized (this) {
-      allocateResource(clusterResource, container.getResource());
+      allocateResource(clusterResource,rmContainer.getContainer().getResource());
     }
     if (parent != null) {
-      parent.recoverContainer(clusterResource, application, container);
+      parent.recoverContainer(clusterResource, attempt, rmContainer);
     }
   }
 
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 7bab760..5227aac 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
@@ -24,6 +24,7 @@
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
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/event/NodeAddedSchedulerEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/NodeAddedSchedulerEvent.java
index c487f48..34ca6e4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/NodeAddedSchedulerEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/NodeAddedSchedulerEvent.java
@@ -18,19 +18,34 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.event;
 
+import java.util.List;
+
+import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 
 public class NodeAddedSchedulerEvent extends SchedulerEvent {
 
   private final RMNode rmNode;
+  private final List<NMContainerStatus> containerReports;
 
   public NodeAddedSchedulerEvent(RMNode rmNode) {
     super(SchedulerEventType.NODE_ADDED);
     this.rmNode = rmNode;
+    this.containerReports = null;
+  }
+
+  public NodeAddedSchedulerEvent(RMNode rmNode,
+      List<NMContainerStatus> containerReports) {
+    super(SchedulerEventType.NODE_ADDED);
+    this.rmNode = rmNode;
+    this.containerReports = containerReports;
   }
 
   public RMNode getAddedRMNode() {
     return rmNode;
   }
 
+  public List<NMContainerStatus> getContainerReports() {
+    return containerReports;
+  }
 }
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/AppSchedulable.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/AppSchedulable.java
index a1704cd..0c36c55 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/AppSchedulable.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/AppSchedulable.java
@@ -264,8 +264,7 @@
       }
 
       // Inform the node
-      node.allocateContainer(app.getApplicationId(),
-          allocatedContainer);
+      node.allocateContainer(allocatedContainer);
 
       // If this container is used to run AM, update the leaf queue's AM usage
       if (app.getLiveContainers().size() == 1 &&
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 8634f0d..21dbdc5 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
@@ -36,6 +36,7 @@
 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;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 @Private
@@ -318,4 +319,10 @@
       Resources.addTo(amResourceUsage, amResource);
     }
   }
+
+  @Override
+  public void recoverContainer(Resource clusterResource,
+      SchedulerApplicationAttempt schedulerAttempt, RMContainer rmContainer) {
+    // TODO Auto-generated method stub
+  }
 }
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/FSParentQueue.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/FSParentQueue.java
index 48db414..5ab60af 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/FSParentQueue.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/FSParentQueue.java
@@ -35,7 +35,9 @@
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.util.resource.Resources;
+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.SchedulerApplicationAttempt;
 
 @Private
 @Unstable
@@ -228,4 +230,11 @@
     // Should never be called since all applications are submitted to LeafQueues
     return null;
   }
+
+  @Override
+  public void recoverContainer(Resource clusterResource,
+      SchedulerApplicationAttempt schedulerAttempt, RMContainer rmContainer) {
+    // TODO Auto-generated method stub
+    
+  }
 }
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/fifo/FifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
index d461615..4681516 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
@@ -76,6 +76,7 @@
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.ContainersAndNMTokensAllocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
@@ -178,6 +179,17 @@
     public ActiveUsersManager getActiveUsersManager() {
       return activeUsersManager;
     }
+
+    @Override
+    public void recoverContainer(Resource clusterResource,
+        SchedulerApplicationAttempt schedulerAttempt, RMContainer rmContainer) {
+      if (rmContainer.getState().equals(RMContainerState.COMPLETED)) {
+        return;
+      }
+      increaseUsedResources(rmContainer);
+      updateAppHeadRoom(schedulerAttempt);
+      updateAvailableResourcesMetrics();
+    }
   };
 
   public FifoScheduler() {
@@ -488,7 +500,7 @@
       if (attempt == null) {
         continue;
       }
-      attempt.setHeadroom(Resources.subtract(clusterResource, usedResource));
+      updateAppHeadRoom(attempt);
     }
   }
 
@@ -659,11 +671,10 @@
             application.allocate(type, node, priority, request, container);
         
         // Inform the node
-        node.allocateContainer(application.getApplicationId(), 
-            rmContainer);
+        node.allocateContainer(rmContainer);
 
         // Update usage for this container
-        Resources.addTo(usedResource, capability);
+        increaseUsedResources(rmContainer);
       }
 
     }
@@ -707,9 +718,22 @@
       LOG.debug("Node after allocation " + rmNode.getNodeID() + " resource = "
           + node.getAvailableResource());
     }
-    
-    metrics.setAvailableResourcesToQueue(
-        Resources.subtract(clusterResource, usedResource));
+
+    updateAvailableResourcesMetrics();
+  }
+
+  private void increaseUsedResources(RMContainer rmContainer) {
+    Resources.addTo(usedResource, rmContainer.getAllocatedResource());
+  }
+
+  private void updateAppHeadRoom(SchedulerApplicationAttempt schedulerAttempt) {
+    schedulerAttempt.setHeadroom(Resources.subtract(clusterResource,
+      usedResource));
+  }
+
+  private void updateAvailableResourcesMetrics() {
+    metrics.setAvailableResourcesToQueue(Resources.subtract(clusterResource,
+      usedResource));
   }
 
   @Override
@@ -719,6 +743,9 @@
     {
       NodeAddedSchedulerEvent nodeAddedEvent = (NodeAddedSchedulerEvent)event;
       addNode(nodeAddedEvent.getAddedRMNode());
+      recoverContainersOnNode(nodeAddedEvent.getContainerReports(),
+        nodeAddedEvent.getAddedRMNode());
+
     }
     break;
     case NODE_REMOVED:
@@ -923,4 +950,8 @@
       return null;
     }
   }
+
+  public Resource getUsedResource() {
+    return usedResource;
+  }
 }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index caee228..446bbae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -21,10 +21,9 @@
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedAction;
+import java.util.List;
 import java.util.Map;
 
-import org.junit.Assert;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.security.Credentials;
@@ -47,12 +46,14 @@
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
@@ -69,6 +70,10 @@
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStartedEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
@@ -76,6 +81,7 @@
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
+import org.junit.Assert;
 
 @SuppressWarnings("unchecked")
 public class MockRM extends ResourceManager {
@@ -144,11 +150,26 @@
     }
   }
 
+  public void waitForContainerToComplete(RMAppAttempt attempt,
+      NMContainerStatus completedContainer) throws InterruptedException {
+    while (true) {
+      List<ContainerStatus> containers = attempt.getJustFinishedContainers();
+      System.out.println("Received completed containers " + containers);
+      for (ContainerStatus container : containers) {
+        if (container.getContainerId().equals(
+          completedContainer.getContainerId())) {
+          return;
+        }
+      }
+      Thread.sleep(200);
+    }
+  }
+
   public void waitForState(MockNM nm, ContainerId containerId,
       RMContainerState containerState) throws Exception {
     RMContainer container = getResourceScheduler().getRMContainer(containerId);
     int timeoutSecs = 0;
-    while(container == null && timeoutSecs++ < 20) {
+    while(container == null && timeoutSecs++ < 100) {
       nm.nodeHeartbeat(true);
       container = getResourceScheduler().getRMContainer(containerId);
       System.out.println("Waiting for container " + containerId + " to be allocated.");
@@ -333,7 +354,7 @@
   public void sendNodeStarted(MockNM nm) throws Exception {
     RMNodeImpl node = (RMNodeImpl) getRMContext().getRMNodes().get(
         nm.getNodeId());
-    node.handle(new RMNodeEvent(nm.getNodeId(), RMNodeEventType.STARTED));
+    node.handle(new RMNodeStartedEvent(nm.getNodeId(), null));
   }
   
   public void sendNodeLost(MockNM nm) throws Exception {
@@ -542,4 +563,12 @@
             .newInstance(appId));
     return response.getApplicationReport();
   }
+
+  // Explicitly reset queue metrics for testing.
+  @SuppressWarnings("static-access")
+  public void clearQueueMetrics(RMApp app) {
+    ((AbstractYarnScheduler<SchedulerApplicationAttempt, SchedulerNode>) getResourceScheduler())
+      .getSchedulerApplications().get(app.getApplicationId()).getQueue()
+      .getMetrics().clearQueueMetrics();
+  }
 }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
index 045b5b4..4b1f59c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
@@ -77,6 +77,7 @@
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -259,6 +260,28 @@
   }
 
   @Test
+  public void testGetApplicationResourceUsageReportDummy() throws YarnException,
+      IOException {
+    ApplicationAttemptId attemptId = getApplicationAttemptId(1);
+    YarnScheduler yarnScheduler = mockYarnScheduler();
+    RMContext rmContext = mock(RMContext.class);
+    mockRMContext(yarnScheduler, rmContext);
+    when(rmContext.getDispatcher().getEventHandler()).thenReturn(
+        new EventHandler<Event>() {
+          public void handle(Event event) {
+          }
+        });
+    ApplicationSubmissionContext asContext = 
+        mock(ApplicationSubmissionContext.class);
+    YarnConfiguration config = new YarnConfiguration();
+    RMAppAttemptImpl rmAppAttemptImpl = new RMAppAttemptImpl(attemptId,
+        rmContext, yarnScheduler, null, asContext, config, false);
+    ApplicationResourceUsageReport report = rmAppAttemptImpl
+        .getApplicationResourceUsageReport();
+    assertEquals(report, RMServerUtils.DUMMY_APPLICATION_RESOURCE_USAGE_REPORT);
+  }
+
+  @Test
   public void testGetApplicationAttempts() throws YarnException, IOException {
     ClientRMService rmService = createRMService();
     RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
@@ -964,6 +987,8 @@
         Arrays.asList(getApplicationAttemptId(101), getApplicationAttemptId(102)));
     when(yarnScheduler.getAppsInQueue(QUEUE_2)).thenReturn(
         Arrays.asList(getApplicationAttemptId(103)));
+    ApplicationAttemptId attemptId = getApplicationAttemptId(1);
+    when(yarnScheduler.getAppResourceUsageReport(attemptId)).thenReturn(null);
     return yarnScheduler;
   }
 }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java
index f50ae9d..b7b77a7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java
@@ -18,16 +18,16 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
-import org.junit.Assert;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -37,6 +37,7 @@
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
@@ -56,11 +57,10 @@
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
 
 public class TestFifoScheduler {
   private static final Log LOG = LogFactory.getLog(TestFifoScheduler.class);
@@ -298,7 +298,10 @@
     FifoScheduler fs = new FifoScheduler();
     fs.init(conf);
     fs.start();
+    // mock rmContext to avoid NPE.
+    RMContext context = mock(RMContext.class);
     fs.reinitialize(conf, null);
+    fs.setRMContext(context);
 
     RMNode n1 =
         MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1, "127.0.0.2");
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestMoveApplication.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestMoveApplication.java
index b80a6bc..36153de 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestMoveApplication.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestMoveApplication.java
@@ -43,10 +43,11 @@
 public class TestMoveApplication {
   private ResourceManager resourceManager = null;
   private static boolean failMove;
-  
+  private Configuration conf;
+
   @Before
   public void setUp() throws Exception {
-    Configuration conf = new YarnConfiguration();
+    conf = new YarnConfiguration();
     conf.setClass(YarnConfiguration.RM_SCHEDULER, FifoSchedulerWithMove.class,
         FifoSchedulerWithMove.class);
     conf.set(YarnConfiguration.YARN_ADMIN_ACL, " ");
@@ -119,28 +120,23 @@
     }
   }
   
-  @Test (timeout = 5000)
-  public void testMoveSuccessful() throws Exception {
-    // Submit application
-    Application application = new Application("user1", resourceManager);
-    ApplicationId appId = application.getApplicationId();
-    application.submit();
-    
-    // Wait for app to be accepted
-    RMApp app = resourceManager.rmContext.getRMApps().get(appId);
-    while (app.getState() != RMAppState.ACCEPTED) {
-      Thread.sleep(100);
-    }
-
-    ClientRMService clientRMService = resourceManager.getClientRMService();
+  @Test (timeout = 10000)
+      public
+      void testMoveSuccessful() throws Exception {
+    MockRM rm1 = new MockRM(conf);
+    rm1.start();
+    RMApp app = rm1.submitApp(1024);
+    ClientRMService clientRMService = rm1.getClientRMService();
     // FIFO scheduler does not support moves
-    clientRMService.moveApplicationAcrossQueues(
-        MoveApplicationAcrossQueuesRequest.newInstance(appId, "newqueue"));
-    
-    RMApp rmApp = resourceManager.getRMContext().getRMApps().get(appId);
+    clientRMService
+      .moveApplicationAcrossQueues(MoveApplicationAcrossQueuesRequest
+        .newInstance(app.getApplicationId(), "newqueue"));
+
+    RMApp rmApp = rm1.getRMContext().getRMApps().get(app.getApplicationId());
     assertEquals("newqueue", rmApp.getQueue());
+    rm1.stop();
   }
-  
+
   @Test
   public void testMoveRejectedByPermissions() throws Exception {
     failMove = true;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
index 3bcde8d..1da03fe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
@@ -21,15 +21,14 @@
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.when;
 
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
-import org.junit.Assert;
-
 import org.apache.hadoop.util.HostsFileReader;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -49,6 +48,7 @@
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeReconnectEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStartedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
@@ -61,6 +61,7 @@
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
@@ -160,7 +161,7 @@
   @Test (timeout = 5000)
   public void testExpiredContainer() {
     // Start the node
-    node.handle(new RMNodeEvent(null, RMNodeEventType.STARTED));
+    node.handle(new RMNodeStartedEvent(null, null));
     verify(scheduler).handle(any(NodeAddedSchedulerEvent.class));
     
     // Expire a container
@@ -188,11 +189,11 @@
   @Test (timeout = 5000)
   public void testContainerUpdate() throws InterruptedException{
     //Start the node
-    node.handle(new RMNodeEvent(null,RMNodeEventType.STARTED));
+    node.handle(new RMNodeStartedEvent(null, null));
     
     NodeId nodeId = BuilderUtils.newNodeId("localhost:1", 1);
     RMNodeImpl node2 = new RMNodeImpl(nodeId, rmContext, null, 0, 0, null, null, null);
-    node2.handle(new RMNodeEvent(null,RMNodeEventType.STARTED));
+    node2.handle(new RMNodeStartedEvent(null, null));
     
     ContainerId completedContainerIdFromNode1 = BuilderUtils.newContainerId(
         BuilderUtils.newApplicationAttemptId(
@@ -248,7 +249,7 @@
   @Test (timeout = 5000)
   public void testStatusChange(){
     //Start the node
-    node.handle(new RMNodeEvent(null,RMNodeEventType.STARTED));
+    node.handle(new RMNodeStartedEvent(null, null));
     //Add info to the queue first
     node.setNextHeartBeat(false);
 
@@ -464,7 +465,7 @@
     RMNodeImpl node = new RMNodeImpl(nodeId, rmContext,null, 0, 0,
         null, ResourceOption.newInstance(capability,
             RMNode.OVER_COMMIT_TIMEOUT_MILLIS_DEFAULT), nmVersion);
-    node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.STARTED));
+    node.handle(new RMNodeStartedEvent(node.getNodeID(), null));
     Assert.assertEquals(NodeState.RUNNING, node.getState());
     return node;
   }
@@ -495,7 +496,7 @@
     int initialUnhealthy = cm.getUnhealthyNMs();
     int initialDecommissioned = cm.getNumDecommisionedNMs();
     int initialRebooted = cm.getNumRebootedNMs();
-    node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.STARTED));
+    node.handle(new RMNodeStartedEvent(node.getNodeID(), null));
     Assert.assertEquals("Active Nodes", initialActive + 1, cm.getNumActiveNMs());
     Assert.assertEquals("Lost Nodes", initialLost, cm.getNumLostNMs());
     Assert.assertEquals("Unhealthy Nodes",
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
index 3bdb66c..9c2d87e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
@@ -84,8 +84,8 @@
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationAttemptStateData;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -612,7 +612,7 @@
 
       @Override
       public void updateApplicationStateInternal(ApplicationId appId,
-          ApplicationStateDataPBImpl appStateData) throws Exception {
+          ApplicationStateData appStateData) throws Exception {
         if (count == 0) {
           // do nothing; simulate app final state is not saved.
           LOG.info(appId + " final state is not saved.");
@@ -760,14 +760,14 @@
       @Override
       public synchronized void storeApplicationAttemptStateInternal(
           ApplicationAttemptId attemptId,
-          ApplicationAttemptStateDataPBImpl attemptStateData) throws Exception {
+          ApplicationAttemptStateData attemptStateData) throws Exception {
         // ignore attempt saving request.
       }
 
       @Override
       public synchronized void updateApplicationAttemptStateInternal(
           ApplicationAttemptId attemptId,
-          ApplicationAttemptStateDataPBImpl attemptStateData) throws Exception {
+          ApplicationAttemptStateData attemptStateData) throws Exception {
         // ignore attempt saving request.
       }
     };
@@ -1862,7 +1862,7 @@
 
     @Override
     public void updateApplicationStateInternal(ApplicationId appId,
-        ApplicationStateDataPBImpl appStateData) throws Exception {
+        ApplicationStateData appStateData) throws Exception {
       updateApp = ++count;
       super.updateApplicationStateInternal(appId, appStateData);
     }
@@ -1871,7 +1871,7 @@
     public synchronized void
         updateApplicationAttemptStateInternal(
             ApplicationAttemptId attemptId,
-            ApplicationAttemptStateDataPBImpl attemptStateData)
+            ApplicationAttemptStateData attemptStateData)
             throws Exception {
       updateAttempt = ++count;
       super.updateApplicationAttemptStateInternal(attemptId,
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
new file mode 100644
index 0000000..6693d09
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
@@ -0,0 +1,570 @@
+/**
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
+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 org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@SuppressWarnings({"rawtypes", "unchecked"})
+@RunWith(value = Parameterized.class)
+public class TestWorkPreservingRMRestart {
+
+  private YarnConfiguration conf;
+  private Class<?> schedulerClass;
+  MockRM rm1 = null;
+  MockRM rm2 = null;
+
+  @Before
+  public void setup() throws UnknownHostException {
+    Logger rootLogger = LogManager.getRootLogger();
+    rootLogger.setLevel(Level.DEBUG);
+    conf = new YarnConfiguration();
+    UserGroupInformation.setConfiguration(conf);
+    conf.set(YarnConfiguration.RECOVERY_ENABLED, "true");
+    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, schedulerClass,
+      ResourceScheduler.class);
+    conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true);
+    DefaultMetricsSystem.setMiniClusterMode(true);
+  }
+
+  @After
+  public void tearDown() {
+    if (rm1 != null) {
+      rm1.stop();
+    }
+    if (rm2 != null) {
+      rm2.stop();
+    }
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> getTestParameters() {
+    return Arrays.asList(new Object[][] { { CapacityScheduler.class },
+        { FifoScheduler.class } });
+  }
+
+  public TestWorkPreservingRMRestart(Class<?> schedulerClass) {
+    this.schedulerClass = schedulerClass;
+  }
+
+  // Test common scheduler state including SchedulerAttempt, SchedulerNode,
+  // AppSchedulingInfo can be reconstructed via the container recovery reports
+  // on NM re-registration.
+  // Also test scheduler specific changes: i.e. Queue recovery-
+  // CSQueue/FSQueue/FifoQueue recovery respectively.
+  // Test Strategy: send 3 container recovery reports(AMContainer, running
+  // container, completed container) on NM re-registration, check the states of
+  // SchedulerAttempt, SchedulerNode etc. are updated accordingly.
+  @Test(timeout = 20000)
+  public void testSchedulerRecovery() throws Exception {
+    conf.setBoolean(CapacitySchedulerConfiguration.ENABLE_USER_METRICS, true);
+    conf.set(CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS,
+      DominantResourceCalculator.class.getName());
+
+    int containerMemory = 1024;
+    Resource containerResource = Resource.newInstance(containerMemory, 1);
+
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(conf);
+    rm1 = new MockRM(conf, memStore);
+    rm1.start();
+    MockNM nm1 =
+        new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
+    nm1.registerNode();
+    RMApp app1 = rm1.submitApp(200);
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    // clear queue metrics
+    rm1.clearQueueMetrics(app1);
+
+    // Re-start RM
+    rm2 = new MockRM(conf, memStore);
+    rm2.start();
+    nm1.setResourceTrackerService(rm2.getResourceTrackerService());
+    // recover app
+    RMApp recoveredApp1 =
+        rm2.getRMContext().getRMApps().get(app1.getApplicationId());
+    RMAppAttempt loadedAttempt1 = recoveredApp1.getCurrentAppAttempt();
+    NMContainerStatus amContainer =
+        TestRMRestart.createNMContainerStatus(am1.getApplicationAttemptId(), 1,
+          ContainerState.RUNNING);
+    NMContainerStatus runningContainer =
+        TestRMRestart.createNMContainerStatus(am1.getApplicationAttemptId(), 2,
+          ContainerState.RUNNING);
+    NMContainerStatus completedContainer =
+        TestRMRestart.createNMContainerStatus(am1.getApplicationAttemptId(), 3,
+          ContainerState.COMPLETE);
+
+    nm1.registerNode(Arrays.asList(amContainer, runningContainer,
+      completedContainer));
+
+    // Wait for RM to settle down on recovering containers;
+    waitForNumContainersToRecover(2, rm2, am1.getApplicationAttemptId());
+
+    // check RMContainers are re-recreated and the container state is correct.
+    rm2.waitForState(nm1, amContainer.getContainerId(),
+      RMContainerState.RUNNING);
+    rm2.waitForState(nm1, runningContainer.getContainerId(),
+      RMContainerState.RUNNING);
+    rm2.waitForContainerToComplete(loadedAttempt1, completedContainer);
+
+    AbstractYarnScheduler scheduler =
+        (AbstractYarnScheduler) rm2.getResourceScheduler();
+    SchedulerNode schedulerNode1 = scheduler.getSchedulerNode(nm1.getNodeId());
+
+    // ********* check scheduler node state.*******
+    // 2 running containers.
+    Resource usedResources = Resources.multiply(containerResource, 2);
+    Resource nmResource =
+        Resource.newInstance(nm1.getMemory(), nm1.getvCores());
+
+    assertTrue(schedulerNode1.isValidContainer(amContainer.getContainerId()));
+    assertTrue(schedulerNode1.isValidContainer(runningContainer
+      .getContainerId()));
+    assertFalse(schedulerNode1.isValidContainer(completedContainer
+      .getContainerId()));
+    // 2 launched containers, 1 completed container
+    assertEquals(2, schedulerNode1.getNumContainers());
+
+    assertEquals(Resources.subtract(nmResource, usedResources),
+      schedulerNode1.getAvailableResource());
+    assertEquals(usedResources, schedulerNode1.getUsedResource());
+    Resource availableResources = Resources.subtract(nmResource, usedResources);
+
+    // ***** check queue state based on the underlying scheduler ********
+    Map<ApplicationId, SchedulerApplication> schedulerApps =
+        ((AbstractYarnScheduler) rm2.getResourceScheduler())
+          .getSchedulerApplications();
+    SchedulerApplication schedulerApp =
+        schedulerApps.get(recoveredApp1.getApplicationId());
+
+    if (schedulerClass.equals(CapacityScheduler.class)) {
+      checkCSQueue(rm2, schedulerApp, nmResource, nmResource, usedResources, 2);
+    } else if (schedulerClass.equals(FifoScheduler.class)) {
+      checkFifoQueue(schedulerApp, usedResources, availableResources);
+    }
+
+    // *********** check scheduler attempt state.********
+    SchedulerApplicationAttempt schedulerAttempt =
+        schedulerApp.getCurrentAppAttempt();
+    assertTrue(schedulerAttempt.getLiveContainers().contains(
+      scheduler.getRMContainer(amContainer.getContainerId())));
+    assertTrue(schedulerAttempt.getLiveContainers().contains(
+      scheduler.getRMContainer(runningContainer.getContainerId())));
+    assertEquals(schedulerAttempt.getCurrentConsumption(), usedResources);
+    assertEquals(availableResources, schedulerAttempt.getHeadroom());
+
+    // *********** check appSchedulingInfo state ***********
+    assertEquals(4, schedulerAttempt.getNewContainerId());
+  }
+
+  private void checkCSQueue(MockRM rm,
+      SchedulerApplication<SchedulerApplicationAttempt> app,
+      Resource clusterResource, Resource queueResource, Resource usedResource,
+      int numContainers)
+      throws Exception {
+    checkCSLeafQueue(rm2, app, clusterResource, queueResource, usedResource,
+      numContainers);
+
+    LeafQueue queue = (LeafQueue) app.getQueue();
+    Resource availableResources = Resources.subtract(queueResource, usedResource);
+    // ************* check Queue metrics ************
+    QueueMetrics queueMetrics = queue.getMetrics();
+    asserteMetrics(queueMetrics, 1, 0, 1, 0, 2, availableResources.getMemory(),
+      availableResources.getVirtualCores(), usedResource.getMemory(),
+      usedResource.getVirtualCores());
+
+    // ************ check user metrics ***********
+    QueueMetrics userMetrics =
+        queueMetrics.getUserMetrics(app.getUser());
+    asserteMetrics(userMetrics, 1, 0, 1, 0, 2, availableResources.getMemory(),
+      availableResources.getVirtualCores(), usedResource.getMemory(),
+      usedResource.getVirtualCores());
+  }
+
+  private void checkCSLeafQueue(MockRM rm,
+      SchedulerApplication<SchedulerApplicationAttempt> app,
+      Resource clusterResource, Resource queueResource, Resource usedResource,
+      int numContainers) {
+    LeafQueue leafQueue = (LeafQueue) app.getQueue();
+    // assert queue used resources.
+    assertEquals(usedResource, leafQueue.getUsedResources());
+    assertEquals(numContainers, leafQueue.getNumContainers());
+
+    ResourceCalculator calc =
+        ((CapacityScheduler) rm.getResourceScheduler()).getResourceCalculator();
+    float usedCapacity =
+        Resources.divide(calc, clusterResource, usedResource, queueResource);
+    // assert queue used capacity
+    assertEquals(usedCapacity, leafQueue.getUsedCapacity(), 1e-8);
+    float absoluteUsedCapacity =
+        Resources.divide(calc, clusterResource, usedResource, clusterResource);
+    // assert queue absolute capacity
+    assertEquals(absoluteUsedCapacity, leafQueue.getAbsoluteUsedCapacity(),
+      1e-8);
+    // assert user consumed resources.
+    assertEquals(usedResource, leafQueue.getUser(app.getUser())
+      .getConsumedResources());
+  }
+
+  private void checkFifoQueue(SchedulerApplication schedulerApp,
+      Resource usedResources, Resource availableResources) throws Exception {
+    FifoScheduler scheduler = (FifoScheduler) rm2.getResourceScheduler();
+    // ************ check cluster used Resources ********
+    assertEquals(usedResources, scheduler.getUsedResource());
+
+    // ************ check app headroom ****************
+    SchedulerApplicationAttempt schedulerAttempt =
+        schedulerApp.getCurrentAppAttempt();
+    assertEquals(availableResources, schedulerAttempt.getHeadroom());
+
+    // ************ check queue metrics ****************
+    QueueMetrics queueMetrics = scheduler.getRootQueueMetrics();
+    asserteMetrics(queueMetrics, 1, 0, 1, 0, 2, availableResources.getMemory(),
+      availableResources.getVirtualCores(), usedResources.getMemory(),
+      usedResources.getVirtualCores());
+  }
+
+  // create 3 container reports for AM
+  public static List<NMContainerStatus>
+      createNMContainerStatusForApp(MockAM am) {
+    List<NMContainerStatus> list =
+        new ArrayList<NMContainerStatus>();
+    NMContainerStatus amContainer =
+        TestRMRestart.createNMContainerStatus(am.getApplicationAttemptId(), 1,
+          ContainerState.RUNNING);
+    NMContainerStatus runningContainer =
+        TestRMRestart.createNMContainerStatus(am.getApplicationAttemptId(), 2,
+          ContainerState.RUNNING);
+    NMContainerStatus completedContainer =
+        TestRMRestart.createNMContainerStatus(am.getApplicationAttemptId(), 3,
+          ContainerState.COMPLETE);
+    list.add(amContainer);
+    list.add(runningContainer);
+    list.add(completedContainer);
+    return list;
+  }
+
+  private static final String R = "Default";
+  private static final String A = "QueueA";
+  private static final String B = "QueueB";
+  private static final String USER_1 = "user1";
+  private static final String USER_2 = "user2";
+
+  private void setupQueueConfiguration(CapacitySchedulerConfiguration conf) {
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] { R });
+    final String Q_R = CapacitySchedulerConfiguration.ROOT + "." + R;
+    conf.setCapacity(Q_R, 100);
+    final String Q_A = Q_R + "." + A;
+    final String Q_B = Q_R + "." + B;
+    conf.setQueues(Q_R, new String[] {A, B});
+    conf.setCapacity(Q_A, 50);
+    conf.setCapacity(Q_B, 50);
+    conf.setDouble(CapacitySchedulerConfiguration
+      .MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT, 0.5f);
+  }
+
+  // Test CS recovery with multi-level queues and multi-users:
+  // 1. setup 2 NMs each with 8GB memory;
+  // 2. setup 2 level queues: Default -> (QueueA, QueueB)
+  // 3. User1 submits 2 apps on QueueA
+  // 4. User2 submits 1 app  on QueueB
+  // 5. AM and each container has 1GB memory
+  // 6. Restart RM.
+  // 7. nm1 re-syncs back containers belong to user1
+  // 8. nm2 re-syncs back containers belong to user2.
+  // 9. Assert the parent queue and 2 leaf queues state and the metrics.
+  // 10. Assert each user's consumption inside the queue.
+  @Test (timeout = 30000)
+  public void testCapacitySchedulerRecovery() throws Exception {
+    if (!schedulerClass.equals(CapacityScheduler.class)) {
+      return;
+    }
+    conf.setBoolean(CapacitySchedulerConfiguration.ENABLE_USER_METRICS, true);
+    conf.set(CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS,
+      DominantResourceCalculator.class.getName());
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration(conf);
+    setupQueueConfiguration(csConf);
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(csConf);
+    rm1 = new MockRM(csConf, memStore);
+    rm1.start();
+    MockNM nm1 =
+        new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
+    MockNM nm2 =
+        new MockNM("127.1.1.1:4321", 8192, rm1.getResourceTrackerService());
+    nm1.registerNode();
+    nm2.registerNode();
+    RMApp app1_1 = rm1.submitApp(1024, "app1_1", USER_1, null, A);
+    MockAM am1_1 = MockRM.launchAndRegisterAM(app1_1, rm1, nm1);
+    RMApp app1_2 = rm1.submitApp(1024, "app1_2", USER_1, null, A);
+    MockAM am1_2 = MockRM.launchAndRegisterAM(app1_2, rm1, nm2);
+
+    RMApp app2 = rm1.submitApp(1024, "app2", USER_2, null, B);
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
+
+    // clear queue metrics
+    rm1.clearQueueMetrics(app1_1);
+    rm1.clearQueueMetrics(app1_2);
+    rm1.clearQueueMetrics(app2);
+
+    // Re-start RM
+    rm2 = new MockRM(csConf, memStore);
+    rm2.start();
+    nm1.setResourceTrackerService(rm2.getResourceTrackerService());
+    nm2.setResourceTrackerService(rm2.getResourceTrackerService());
+
+    List<NMContainerStatus> am1_1Containers =
+        createNMContainerStatusForApp(am1_1);
+    List<NMContainerStatus> am1_2Containers =
+        createNMContainerStatusForApp(am1_2);
+    am1_1Containers.addAll(am1_2Containers);
+    nm1.registerNode(am1_1Containers);
+
+    List<NMContainerStatus> am2Containers =
+        createNMContainerStatusForApp(am2);
+    nm2.registerNode(am2Containers);
+
+    // Wait for RM to settle down on recovering containers;
+    waitForNumContainersToRecover(2, rm2, am1_1.getApplicationAttemptId());
+    waitForNumContainersToRecover(2, rm2, am1_2.getApplicationAttemptId());
+    waitForNumContainersToRecover(2, rm2, am1_2.getApplicationAttemptId());
+
+    // Calculate each queue's resource usage.
+    Resource containerResource = Resource.newInstance(1024, 1);
+    Resource nmResource =
+        Resource.newInstance(nm1.getMemory(), nm1.getvCores());
+    Resource clusterResource = Resources.multiply(nmResource, 2);
+    Resource q1Resource = Resources.multiply(clusterResource, 0.5);
+    Resource q2Resource = Resources.multiply(clusterResource, 0.5);
+    Resource q1UsedResource = Resources.multiply(containerResource, 4);
+    Resource q2UsedResource = Resources.multiply(containerResource, 2);
+    Resource totalUsedResource = Resources.add(q1UsedResource, q2UsedResource);
+    Resource q1availableResources =
+        Resources.subtract(q1Resource, q1UsedResource);
+    Resource q2availableResources =
+        Resources.subtract(q2Resource, q2UsedResource);
+    Resource totalAvailableResource =
+        Resources.add(q1availableResources, q2availableResources);
+
+    Map<ApplicationId, SchedulerApplication> schedulerApps =
+        ((AbstractYarnScheduler) rm2.getResourceScheduler())
+          .getSchedulerApplications();
+    SchedulerApplication schedulerApp1_1 =
+        schedulerApps.get(app1_1.getApplicationId());
+
+    // assert queue A state.
+    checkCSLeafQueue(rm2, schedulerApp1_1, clusterResource, q1Resource,
+      q1UsedResource, 4);
+    QueueMetrics queue1Metrics = schedulerApp1_1.getQueue().getMetrics();
+    asserteMetrics(queue1Metrics, 2, 0, 2, 0, 4,
+      q1availableResources.getMemory(), q1availableResources.getVirtualCores(),
+      q1UsedResource.getMemory(), q1UsedResource.getVirtualCores());
+
+    // assert queue B state.
+    SchedulerApplication schedulerApp2 =
+        schedulerApps.get(app2.getApplicationId());
+    checkCSLeafQueue(rm2, schedulerApp2, clusterResource, q2Resource,
+      q2UsedResource, 2);
+    QueueMetrics queue2Metrics = schedulerApp2.getQueue().getMetrics();
+    asserteMetrics(queue2Metrics, 1, 0, 1, 0, 2,
+      q2availableResources.getMemory(), q2availableResources.getVirtualCores(),
+      q2UsedResource.getMemory(), q2UsedResource.getVirtualCores());
+
+    // assert parent queue state.
+    LeafQueue leafQueue = (LeafQueue) schedulerApp2.getQueue();
+    ParentQueue parentQueue = (ParentQueue) leafQueue.getParent();
+    checkParentQueue(parentQueue, 6, totalUsedResource, (float) 6 / 16,
+      (float) 6 / 16);
+    asserteMetrics(parentQueue.getMetrics(), 3, 0, 3, 0, 6,
+      totalAvailableResource.getMemory(),
+      totalAvailableResource.getVirtualCores(), totalUsedResource.getMemory(),
+      totalUsedResource.getVirtualCores());
+  }
+
+  private void checkParentQueue(ParentQueue parentQueue, int numContainers,
+      Resource usedResource, float UsedCapacity, float absoluteUsedCapacity) {
+    assertEquals(numContainers, parentQueue.getNumContainers());
+    assertEquals(usedResource, parentQueue.getUsedResources());
+    assertEquals(UsedCapacity, parentQueue.getUsedCapacity(), 1e-8);
+    assertEquals(absoluteUsedCapacity, parentQueue.getAbsoluteUsedCapacity(), 1e-8);
+  }
+
+  // Test RM shuts down, in the meanwhile, AM fails. Restarted RM scheduler
+  // should not recover the containers that belong to the failed AM.
+  @Test(timeout = 20000)
+  public void testAMfailedBetweenRMRestart() throws Exception {
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(conf);
+    rm1 = new MockRM(conf, memStore);
+    rm1.start();
+    MockNM nm1 =
+        new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
+    nm1.registerNode();
+    RMApp app1 = rm1.submitApp(200);
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    rm2 = new MockRM(conf, memStore);
+    rm2.start();
+    nm1.setResourceTrackerService(rm2.getResourceTrackerService());
+
+    NMContainerStatus amContainer =
+        TestRMRestart.createNMContainerStatus(am1.getApplicationAttemptId(), 1,
+          ContainerState.COMPLETE);
+    NMContainerStatus runningContainer =
+        TestRMRestart.createNMContainerStatus(am1.getApplicationAttemptId(), 2,
+          ContainerState.RUNNING);
+    NMContainerStatus completedContainer =
+        TestRMRestart.createNMContainerStatus(am1.getApplicationAttemptId(), 3,
+          ContainerState.COMPLETE);
+    nm1.registerNode(Arrays.asList(amContainer, runningContainer,
+      completedContainer));
+    rm2.waitForState(am1.getApplicationAttemptId(), RMAppAttemptState.FAILED);
+    // Wait for RM to settle down on recovering containers;
+    Thread.sleep(3000);
+
+    AbstractYarnScheduler scheduler =
+        (AbstractYarnScheduler) rm2.getResourceScheduler();
+    // Previous AM failed, The failed AM should once again release the
+    // just-recovered containers.
+    assertNull(scheduler.getRMContainer(runningContainer.getContainerId()));
+    assertNull(scheduler.getRMContainer(completedContainer.getContainerId()));
+  }
+
+  // Apps already completed before RM restart. Restarted RM scheduler should not
+  // recover containers for completed apps.
+  @Test(timeout = 20000)
+  public void testContainersNotRecoveredForCompletedApps() throws Exception {
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(conf);
+    rm1 = new MockRM(conf, memStore);
+    rm1.start();
+    MockNM nm1 =
+        new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
+    nm1.registerNode();
+    RMApp app1 = rm1.submitApp(200);
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+    MockRM.finishAMAndVerifyAppState(app1, rm1, nm1, am1);
+
+    rm2 = new MockRM(conf, memStore);
+    rm2.start();
+    nm1.setResourceTrackerService(rm2.getResourceTrackerService());
+    NMContainerStatus runningContainer =
+        TestRMRestart.createNMContainerStatus(am1.getApplicationAttemptId(), 2,
+          ContainerState.RUNNING);
+    NMContainerStatus completedContainer =
+        TestRMRestart.createNMContainerStatus(am1.getApplicationAttemptId(), 3,
+          ContainerState.COMPLETE);
+    nm1.registerNode(Arrays.asList(runningContainer, completedContainer));
+    RMApp recoveredApp1 =
+        rm2.getRMContext().getRMApps().get(app1.getApplicationId());
+    assertEquals(RMAppState.FINISHED, recoveredApp1.getState());
+
+    // Wait for RM to settle down on recovering containers;
+    Thread.sleep(3000);
+
+    AbstractYarnScheduler scheduler =
+        (AbstractYarnScheduler) rm2.getResourceScheduler();
+
+    // scheduler should not recover containers for finished apps.
+    assertNull(scheduler.getRMContainer(runningContainer.getContainerId()));
+    assertNull(scheduler.getRMContainer(completedContainer.getContainerId()));
+  }
+
+  private void asserteMetrics(QueueMetrics qm, int appsSubmitted,
+      int appsPending, int appsRunning, int appsCompleted,
+      int allocatedContainers, int availableMB, int availableVirtualCores,
+      int allocatedMB, int allocatedVirtualCores) {
+    assertEquals(appsSubmitted, qm.getAppsSubmitted());
+    assertEquals(appsPending, qm.getAppsPending());
+    assertEquals(appsRunning, qm.getAppsRunning());
+    assertEquals(appsCompleted, qm.getAppsCompleted());
+    assertEquals(allocatedContainers, qm.getAllocatedContainers());
+    assertEquals(availableMB, qm.getAvailableMB());
+    assertEquals(availableVirtualCores, qm.getAvailableVirtualCores());
+    assertEquals(allocatedMB, qm.getAllocatedMB());
+    assertEquals(allocatedVirtualCores, qm.getAllocatedVirtualCores());
+  }
+
+  private void waitForNumContainersToRecover(int num, MockRM rm,
+      ApplicationAttemptId attemptId) throws Exception {
+    AbstractYarnScheduler scheduler =
+        (AbstractYarnScheduler) rm.getResourceScheduler();
+    SchedulerApplicationAttempt attempt =
+        scheduler.getApplicationAttempt(attemptId);
+    while (attempt == null) {
+      System.out.println("Wait for scheduler attempt " + attemptId
+          + " to be created");
+      Thread.sleep(200);
+      attempt = scheduler.getApplicationAttempt(attemptId);
+    }
+    while (attempt.getLiveContainers().size() < num) {
+      System.out.println("Wait for " + num + " containers to recover.");
+      Thread.sleep(200);
+    }
+  }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
index 792b73e..da25c5b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
@@ -24,7 +24,6 @@
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.junit.Assert;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -37,6 +36,7 @@
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.RMStateVersion;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.RMStateVersionPBImpl;
@@ -213,9 +213,8 @@
           try {
             store.storeApplicationStateInternal(
                 ApplicationId.newInstance(100L, 1),
-                (ApplicationStateDataPBImpl) ApplicationStateDataPBImpl
-                    .newApplicationStateData(111, 111, "user", null,
-                        RMAppState.ACCEPTED, "diagnostics", 333));
+                ApplicationStateData.newInstance(111, 111, "user", null,
+                    RMAppState.ACCEPTED, "diagnostics", 333));
           } catch (Exception e) {
             // TODO 0 datanode exception will not be retried by dfs client, fix
             // that separately.
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
index e89b71b..0fd3c3c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
@@ -51,6 +51,7 @@
 import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationState;
@@ -921,6 +922,7 @@
     assertAppState(RMAppState.NEW, app);
     ApplicationReport report = app.createAndGetApplicationReport(null, true);
     Assert.assertNotNull(report.getApplicationResourceUsageReport());
+    Assert.assertEquals(report.getApplicationResourceUsageReport(),RMServerUtils.DUMMY_APPLICATION_RESOURCE_USAGE_REPORT);
     report = app.createAndGetApplicationReport("clientuser", true);
     Assert.assertNotNull(report.getApplicationResourceUsageReport());
   }